You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by hi...@apache.org on 2016/06/07 19:07:03 UTC

[44/55] [abbrv] incubator-geode git commit: GEODE-11: Added support for lucene index profile exchange

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index 33cb3d3..98233b0 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -84,11 +84,10 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   @Override
   public void beforeCreate(Extensible<Region<?, ?>> source, Cache cache) {
     LuceneServiceImpl service = (LuceneServiceImpl) LuceneServiceProvider.get(cache);
-    if (this.fieldAnalyzers == null) {
-      service.createIndex(getName(), getRegionPath(), getFieldNames());
-    } else {
-      service.createIndex(getName(), getRegionPath(), this.fieldAnalyzers);
-    }
+    Analyzer analyzer = this.fieldAnalyzers == null
+        ? new StandardAnalyzer()
+        : new PerFieldAnalyzerWrapper(new StandardAnalyzer(), this.fieldAnalyzers);
+    service.createIndex(getName(), getRegionPath(), analyzer, this.fieldAnalyzers, getFieldNames());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneDUnitTest.java
new file mode 100644
index 0000000..565ec4f
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneDUnitTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.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;
+
+public abstract class LuceneDUnitTest extends JUnit4CacheTestCase {
+  protected VM dataStore1;
+  protected VM dataStore2;
+
+  @Override
+  public void postSetUp() throws Exception {
+    Host host = Host.getHost(0);
+    dataStore1 = host.getVM(0);
+    dataStore2 = host.getVM(1);
+  }
+
+  protected abstract void initDataStore(SerializableRunnableIF createIndex) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.java
new file mode 100644
index 0000000..f51c848
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.java
@@ -0,0 +1,281 @@
+/*
+ * 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 com.gemstone.gemfire.util.test.TestUtil;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.$;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
+
+  @Override
+  protected void initDataStore(SerializableRunnableIF createIndex) throws Exception {
+    createIndex.run();
+    getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+  }
+
+  @Test
+  public void verifyDifferentFieldsFails() {
+    SerializableRunnableIF createIndex1 = getFieldsIndexWithOneField();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getFieldsIndexWithTwoFields();
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_FIELDS));
+  }
+
+  @Test
+  public void verifyDifferentFieldAnalyzerSizesFails1() {
+    SerializableRunnableIF createIndex1 = getAnalyzersIndexWithTwoFields();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getAnalyzersIndexWithOneField();
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD2));
+  }
+
+  @Test
+  public void verifyDifferentFieldAnalyzerSizesFails2() {
+    SerializableRunnableIF createIndex1 = getAnalyzersIndexWithOneField();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getAnalyzersIndexWithTwoFields();
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZER_SIZES_1));
+  }
+
+  @Test
+  public void verifyDifferentFieldAnalyzersFails1() {
+    SerializableRunnableIF createIndex1 = getAnalyzersIndexWithOneField(StandardAnalyzer.class);
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getAnalyzersIndexWithOneField(KeywordAnalyzer.class);
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS));
+  }
+
+  @Test
+  public void verifyDifferentFieldAnalyzersFails2() {
+    SerializableRunnableIF createIndex1 = getAnalyzersIndexWithNullField2();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getAnalyzersIndexWithNullField1();
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD1));
+  }
+
+  @Test
+  public void verifyDifferentFieldAnalyzersFails3() {
+    SerializableRunnableIF createIndex1 = getAnalyzersIndexWithNullField1();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = getAnalyzersIndexWithNullField2();
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_EXISTING_MEMBER));
+  }
+
+  @Test
+  public void verifyDifferentIndexNamesFails() {
+    SerializableRunnableIF createIndex1 = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME+"1", REGION_NAME, "field1");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME+"2", REGION_NAME, "field1");
+    };
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_NAMES));
+  }
+
+  @Test
+  public void verifyDifferentIndexesFails1() {
+    SerializableRunnableIF createIndex1 = getFieldsIndexWithOneField();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = () -> {/*Do nothing*/};
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_1));
+  }
+
+  @Test
+  public void verifyDifferentIndexesFails2() {
+    SerializableRunnableIF createIndex1 = getFieldsIndexWithOneField();
+    dataStore1.invoke(() -> initDataStore(createIndex1));
+
+    SerializableRunnableIF createIndex2 = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1");
+      luceneService.createIndex(INDEX_NAME+"2", REGION_NAME, "field2");
+    };
+    dataStore2.invoke(() -> initDataStore(createIndex2, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_2));
+  }
+
+  @Test
+  @Parameters(method = "getIndexes")
+  public void verifySameIndexesSucceeds(SerializableRunnableIF createIndex) {
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    dataStore2.invoke(() -> initDataStore(createIndex));
+  }
+
+  private final Object[] getIndexes() {
+    return $(
+        new Object[] { getFieldsIndexWithOneField() },
+        new Object[] { getFieldsIndexWithTwoFields() },
+        new Object[] { get2FieldsIndexes() },
+        new Object[] { getAnalyzersIndexWithOneField() },
+        new Object[] { getAnalyzersIndexWithTwoFields() },
+        new Object[] { getAnalyzersIndexWithNullField1() }
+    );
+  }
+
+  @Test
+  @Parameters(method = "getXmlAndExceptionMessages")
+  public void verifyXml(String cacheXmlFileBaseName, String exceptionMessage) {
+    dataStore1.invoke(() -> initCache(getXmlFileForTest(cacheXmlFileBaseName + ".1")));
+    dataStore2.invoke(() -> initCache(getXmlFileForTest(cacheXmlFileBaseName + ".2"), exceptionMessage));
+  }
+
+  private final Object[] getXmlAndExceptionMessages() {
+    return $(
+        new Object[] { "verifyDifferentFieldsFails", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_FIELDS },
+        new Object[] { "verifyDifferentFieldAnalyzerSizesFails1", CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD2 },
+        new Object[] { "verifyDifferentFieldAnalyzerSizesFails2", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZER_SIZES_1 },
+        new Object[] { "verifyDifferentFieldAnalyzersFails1", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS },
+        // Currently setting a null analyzer is not a valid xml configuration: <lucene:field name="field2" analyzer="null"/>
+        //new Object[] { "verifyDifferentFieldAnalyzersFails2", CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD1 },
+        //new Object[] { "verifyDifferentFieldAnalyzersFails3", CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_EXISTING_MEMBER },
+        new Object[] { "verifyDifferentIndexNamesFails", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_NAMES },
+        new Object[] { "verifyDifferentIndexesFails1", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_1 },
+        new Object[] { "verifyDifferentIndexesFails2", CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_2 }
+    );
+  }
+
+  private String getXmlFileForTest(String testName) {
+    return TestUtil.getResourcePath(getClass(), getClass().getSimpleName() + "." + testName + ".cache.xml");
+  }
+
+  private void initDataStore(SerializableRunnableIF createIndex, String message) throws Exception {
+    createIndex.run();
+    try {
+      getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+      fail("Should not have been able to create index");
+    } catch (IllegalStateException e) {
+      assertEquals(message, e.getMessage());
+    }
+  }
+
+  private void initCache(String cacheXmlFileName) throws FileNotFoundException {
+    getCache().loadCacheXml(new FileInputStream(cacheXmlFileName));
+  }
+
+  private void initCache(String cacheXmlFileName, String message) throws FileNotFoundException {
+    try {
+      getCache().loadCacheXml(new FileInputStream(cacheXmlFileName));
+      fail("Should not have been able to create cache");
+    } catch (IllegalStateException e) {
+      assertEquals(message, e.getMessage());
+    }
+  }
+
+  private SerializableRunnableIF getFieldsIndexWithOneField() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1");
+    };
+  }
+
+  private SerializableRunnableIF getFieldsIndexWithTwoFields() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+    };
+  }
+
+  private SerializableRunnableIF get2FieldsIndexes() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME+"_1", REGION_NAME, "field1");
+      luceneService.createIndex(INDEX_NAME+"_2", REGION_NAME, "field2");
+    };
+  }
+
+  private SerializableRunnableIF getAnalyzersIndexWithNullField1() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      Map<String, Analyzer> analyzers = new HashMap<>();
+      analyzers.put("field1", null);
+      analyzers.put("field2", new KeywordAnalyzer());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    };
+  }
+
+  private SerializableRunnableIF getAnalyzersIndexWithNullField2() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      Map<String, Analyzer> analyzers = new HashMap<>();
+      analyzers.put("field1", new KeywordAnalyzer());
+      analyzers.put("field2", null);
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    };
+  }
+
+  private SerializableRunnableIF getAnalyzersIndexWithOneField(Class<? extends Analyzer> analyzerClass) {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      Map<String, Analyzer> analyzers = new HashMap<>();
+      analyzers.put("field1", analyzerClass.newInstance());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    };
+  }
+
+  private SerializableRunnableIF getAnalyzersIndexWithOneField() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      Map<String, Analyzer> analyzers = new HashMap<>();
+      analyzers.put("field1", new KeywordAnalyzer());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    };
+  }
+
+  private SerializableRunnableIF getAnalyzersIndexWithTwoFields() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      Map<String, Analyzer> analyzers = new HashMap<>();
+      analyzers.put("field1", new KeywordAnalyzer());
+      analyzers.put("field2", new KeywordAnalyzer());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/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
index 821be17..92d8e8b 100644
--- 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
@@ -28,16 +28,10 @@ import java.util.Map;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneEventListener;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexImpl;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
-import com.gemstone.gemfire.internal.logging.LogService;
 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;
 
@@ -47,23 +41,17 @@ import org.junit.Test;
   * of different regions types and topologies.
   *
   */
-public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
+public abstract class LuceneQueriesBase extends LuceneDUnitTest {
 
   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);
+    super.postSetUp();
+    accessor = Host.getHost(0).getVM(3);
   }
 
-  protected abstract void initDataStore(SerializableRunnableIF createIndex) throws Exception;
-
   protected abstract void initAccessor(SerializableRunnableIF createIndex) throws Exception;
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
new file mode 100644
index 0000000..ecb48af
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.CopyHelper;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.$;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+@Category(UnitTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexCreationProfileJUnitTest {
+
+  @Test
+  @Parameters(method = "getSerializationProfiles")
+  public void testSerialization(LuceneIndexCreationProfile profile) {
+    LuceneServiceImpl.registerDataSerializables();
+    LuceneIndexCreationProfile copy = CopyHelper.deepCopy(profile);
+    assertEquals(profile.getIndexName(), copy.getIndexName());
+    assertEquals(profile.getAnalyzerClass(), copy.getAnalyzerClass());
+    assertArrayEquals(profile.getFieldNames(), copy.getFieldNames());
+    assertEquals(profile.getFieldAnalyzers(), copy.getFieldAnalyzers());
+  }
+
+  private final Object[] getSerializationProfiles() {
+    return $(
+        new Object[] { getOneFieldLuceneIndexCreationProfile() },
+        new Object[] { getTwoFieldLuceneIndexCreationProfile() },
+        new Object[] { getTwoAnalyzersLuceneIndexCreationProfile() },
+        new Object[] { getNullField1AnalyzerLuceneIndexCreationProfile() }
+    );
+  }
+
+  @Test
+  @Parameters(method = "getCheckCompatibilityProfiles")
+  public void testCheckCompatibility(LuceneIndexCreationProfile myProfile, LuceneIndexCreationProfile otherProfile, String expectedResult) {
+    assertEquals(expectedResult, otherProfile.checkCompatibility("/"+REGION_NAME, myProfile));
+  }
+
+  private final Object[] getCheckCompatibilityProfiles() {
+    return $(
+        new Object[] {
+            getOneFieldLuceneIndexCreationProfile(),
+            getTwoFieldLuceneIndexCreationProfile(),
+            CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_FIELDS
+        },
+        new Object[] {
+            getTwoAnalyzersLuceneIndexCreationProfile(),
+            getOneAnalyzerLuceneIndexCreationProfile(new KeywordAnalyzer()),
+            CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD2
+        },
+        new Object[] {
+            getOneAnalyzerLuceneIndexCreationProfile(new KeywordAnalyzer()),
+            getTwoAnalyzersLuceneIndexCreationProfile(),
+            CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZER_SIZES_2
+        },
+        new Object[] {
+            getOneAnalyzerLuceneIndexCreationProfile(new StandardAnalyzer()),
+            getOneAnalyzerLuceneIndexCreationProfile(new KeywordAnalyzer()),
+            CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS
+        },
+        new Object[] {
+            getNullField2AnalyzerLuceneIndexCreationProfile(),
+            getNullField1AnalyzerLuceneIndexCreationProfile(),
+            CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD1
+        },
+        new Object[] {
+            getNullField1AnalyzerLuceneIndexCreationProfile(),
+            getNullField2AnalyzerLuceneIndexCreationProfile(),
+            CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_EXISTING_MEMBER
+        }
+    );
+  }
+
+  private LuceneIndexCreationProfile getOneFieldLuceneIndexCreationProfile() {
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1" }, new StandardAnalyzer(), null);
+  }
+
+  private LuceneIndexCreationProfile getTwoFieldLuceneIndexCreationProfile() {
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1", "field2" }, new StandardAnalyzer(), null);
+  }
+
+  private LuceneIndexCreationProfile getOneAnalyzerLuceneIndexCreationProfile(Analyzer analyzer) {
+    Map<String, Analyzer> fieldAnalyzers = new HashMap<>();
+    fieldAnalyzers.put("field1", analyzer);
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1", "field2" }, getPerFieldAnalyzerWrapper(fieldAnalyzers), fieldAnalyzers);
+  }
+
+  private LuceneIndexCreationProfile getTwoAnalyzersLuceneIndexCreationProfile() {
+    Map<String, Analyzer> fieldAnalyzers = new HashMap<>();
+    fieldAnalyzers.put("field1", new KeywordAnalyzer());
+    fieldAnalyzers.put("field2", new KeywordAnalyzer());
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1", "field2" }, getPerFieldAnalyzerWrapper(fieldAnalyzers), fieldAnalyzers);
+  }
+
+  private LuceneIndexCreationProfile getNullField1AnalyzerLuceneIndexCreationProfile() {
+    Map<String, Analyzer> fieldAnalyzers = new HashMap<>();
+    fieldAnalyzers.put("field1", null);
+    fieldAnalyzers.put("field2", new KeywordAnalyzer());
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1", "field2" }, getPerFieldAnalyzerWrapper(fieldAnalyzers), fieldAnalyzers);
+  }
+
+  private LuceneIndexCreationProfile getNullField2AnalyzerLuceneIndexCreationProfile() {
+    Map<String, Analyzer> fieldAnalyzers = new HashMap<>();
+    fieldAnalyzers.put("field1", new KeywordAnalyzer());
+    fieldAnalyzers.put("field2", null);
+    return new LuceneIndexCreationProfile(INDEX_NAME, new String[] { "field1", "field2" }, getPerFieldAnalyzerWrapper(fieldAnalyzers), fieldAnalyzers);
+  }
+
+  private Analyzer getPerFieldAnalyzerWrapper(Map<String, Analyzer> fieldAnalyzers) {
+    return new PerFieldAnalyzerWrapper(new StandardAnalyzer(), fieldAnalyzers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 860dacf..0cf8953 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -43,6 +43,17 @@ public class LuceneTestUtilities {
   public static final String INDEX_NAME = "index";
   public static final String REGION_NAME = "region";
 
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_FIELDS = "Cannot create Lucene index index on region /region with fields [field1, field2] because another member defines the same index with fields [field1].";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD2 = "Cannot create Lucene index index on region /region with no analyzer on field field2 because another member defines the same index with analyzer org.apache.lucene.analysis.core.KeywordAnalyzer on that field.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZER_SIZES_1 = "Cannot create Lucene index index on region /region with field analyzers {field2=class org.apache.lucene.analysis.core.KeywordAnalyzer, field1=class org.apache.lucene.analysis.core.KeywordAnalyzer} because another member defines the same index with field analyzers {field1=class org.apache.lucene.analysis.core.KeywordAnalyzer}.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZER_SIZES_2 = "Cannot create Lucene index index on region /region with field analyzers {field1=class org.apache.lucene.analysis.core.KeywordAnalyzer, field2=class org.apache.lucene.analysis.core.KeywordAnalyzer} because another member defines the same index with field analyzers {field1=class org.apache.lucene.analysis.core.KeywordAnalyzer}.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS = "Cannot create Lucene index index on region /region with analyzer org.apache.lucene.analysis.core.KeywordAnalyzer on field field1 because another member defines the same index with analyzer org.apache.lucene.analysis.standard.StandardAnalyzer on that field.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_FIELD1 = "Cannot create Lucene index index on region /region with no analyzer on field field1 because another member defines the same index with analyzer org.apache.lucene.analysis.core.KeywordAnalyzer on that field.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_NO_ANALYZER_EXISTING_MEMBER = "Cannot create Lucene index index on region /region with analyzer org.apache.lucene.analysis.core.KeywordAnalyzer on field field1 because another member defines the same index with no analyzer on that field.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_NAMES = "Cannot create Region /region with [index2#_region] async event ids because another cache has the same region defined with [index1#_region] async event ids";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_1 = "Cannot create Region /region with [] async event ids because another cache has the same region defined with [index#_region] async event ids";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_2 = "Cannot create Region /region with [index#_region, index2#_region] async event ids because another cache has the same region defined with [index#_region] async event ids";
+
   public static void verifyInternalRegions(LuceneService luceneService, Cache cache, Consumer<LocalRegion> verify) {
     // Get index
     LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.1.cache.xml
new file mode 100755
index 0000000..40fc80e
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.1.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+      <lucene:field name="field2" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.2.cache.xml
new file mode 100755
index 0000000..ec278b1
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails1.2.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.1.cache.xml
new file mode 100755
index 0000000..ec278b1
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.2.cache.xml
new file mode 100755
index 0000000..40fc80e
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzerSizesFails2.2.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+      <lucene:field name="field2" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.1.cache.xml
new file mode 100755
index 0000000..12ef799
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.standard.StandardAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.2.cache.xml
new file mode 100755
index 0000000..ec278b1
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails1.2.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.1.cache.xml
new file mode 100755
index 0000000..5550ad6
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.1.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+      <lucene:field name="field2" analyzer="null"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.2.cache.xml
new file mode 100755
index 0000000..ceed0ec
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails2.2.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="null"/>
+      <lucene:field name="field2" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.1.cache.xml
new file mode 100755
index 0000000..ceed0ec
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.1.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="null"/>
+      <lucene:field name="field2" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.2.cache.xml
new file mode 100755
index 0000000..5550ad6
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldAnalyzersFails3.2.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1" analyzer="org.apache.lucene.analysis.core.KeywordAnalyzer"/>
+      <lucene:field name="field2" analyzer="null"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.1.cache.xml
new file mode 100755
index 0000000..b0870ee
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.2.cache.xml
new file mode 100755
index 0000000..77936e2
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentFieldsFails.2.cache.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1"/>
+      <lucene:field name="field2"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.1.cache.xml
new file mode 100755
index 0000000..38afdbd
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index1">
+      <lucene:field name="field1"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.2.cache.xml
new file mode 100755
index 0000000..83e532d
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexNamesFails.2.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index2">
+      <lucene:field name="field1"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.1.cache.xml
new file mode 100755
index 0000000..b0870ee
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.2.cache.xml
new file mode 100755
index 0000000..3666c4f
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails1.2.cache.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT"/>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.1.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.1.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.1.cache.xml
new file mode 100755
index 0000000..b0870ee
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.1.cache.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1"/>
+    </lucene:index>
+  </region>
+ 
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c742c4e5/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.2.cache.xml
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.2.cache.xml b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.2.cache.xml
new file mode 100755
index 0000000..7aca855
--- /dev/null
+++ b/geode-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationDUnitTest.verifyDifferentIndexesFails2.2.cache.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<cache
+    xmlns="http://geode.apache.org/schema/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://geode.apache.org/schema/cache
+        http://geode.apache.org/schema/cache/cache-1.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="1.0">
+
+  <region name="region" refid="PARTITION_REDUNDANT">
+    <lucene:index name="index">
+      <lucene:field name="field1"/>
+    </lucene:index>
+    <lucene:index name="index2">
+      <lucene:field name="field2"/>
+    </lucene:index>
+  </region>
+ 
+</cache>