You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2017/09/21 15:57:19 UTC

[geode] branch feature/GEODE-3239 updated: GEODE-3569: save the serializer class name into LuceneIndexCreationProfile and compare between members when creating index addCacheServiceProfile should use serializer as paramter set default class name add junit and dunit tests for fromData/toData add backward compatibility for fromData/toData add backward compatibility for LuceneIndexCreationProfile

This is an automated email from the ASF dual-hosted git repository.

zhouxj pushed a commit to branch feature/GEODE-3239
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/feature/GEODE-3239 by this push:
     new 00f2afd  GEODE-3569: save the serializer class name into LuceneIndexCreationProfile and compare between members when creating index             addCacheServiceProfile should use serializer as paramter             set default class name             add junit and dunit tests for fromData/toData             add backward compatibility for fromData/toData             add backward compatibility for LuceneIndexCreationProfile
00f2afd is described below

commit 00f2afdea997f57ab6351531f2e2fe4496e03371
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Fri Sep 15 15:30:06 2017 -0700

    GEODE-3569: save the serializer class name into LuceneIndexCreationProfile
    and compare between members when creating index
                addCacheServiceProfile should use serializer as paramter
                set default class name
                add junit and dunit tests for fromData/toData
                add backward compatibility for fromData/toData
                add backward compatibility for LuceneIndexCreationProfile
---
 .../geode/internal/i18n/LocalizedStrings.java      |  3 ++
 .../internal/LuceneIndexCreationProfile.java       | 41 +++++++++++++--
 .../lucene/internal/LuceneRegionListener.java      |  2 +-
 .../cache/lucene/internal/LuceneServiceImpl.java   |  4 +-
 .../geode/cache/lucene/DummyLuceneSerializer.java  | 29 +++++++++++
 .../cache/lucene/LuceneIndexCreationDUnitTest.java | 48 ++++++++++++++++++
 .../LuceneIndexCreationProfileJUnitTest.java       | 59 +++++++++++++++++++---
 .../cache/lucene/test/LuceneTestUtilities.java     |  2 +
 8 files changed, 176 insertions(+), 12 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index 3a32db8..ab3b8eb 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -7611,6 +7611,9 @@ public class LocalizedStrings {
   public static final StringId LuceneService_CANNOT_CREATE_INDEX_0_ON_REGION_1_WITH_ANALYZER_2_ON_FIELD_3_BECAUSE_ANOTHER_MEMBER_DEFINES_THE_SAME_INDEX_WITH_ANALYZER_4_ON_THAT_FIELD =
       new StringId(6631,
           "Cannot create Lucene index {0} on region {1} with analyzer {2} on field {3} because another member defines the same index with analyzer {4} on that field.");
+  public static final StringId LuceneService_CANNOT_CREATE_INDEX_0_ON_REGION_1_WITH_SERIALIZER_2_BECAUSE_ANOTHER_MEMBER_DEFINES_THE_SAME_INDEX_WITH_DIFFERENT_SERIALIZER_3 =
+      new StringId(6632,
+          "Cannot create Lucene index {0} on region {1} with serializer {2} because another member defines the same index with different serializer {3}.");
 
   public static final StringId AbstractDistributionConfig_CLUSTER_SSL_ALIAS_0 = new StringId(6633,
       "SSL communication uses the this alias when determining the key to use from the keystore for SSL. Defaults to \"{0}\".");
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
index 28d84e5..e88045d 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
@@ -16,6 +16,10 @@ package org.apache.geode.cache.lucene.internal;
 
 import org.apache.geode.DataSerializable;
 import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.lucene.LuceneSerializer;
+import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
+import org.apache.geode.internal.Version;
+import org.apache.geode.internal.VersionedDataSerializable;
 import org.apache.geode.internal.cache.CacheServiceProfile;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.lucene.analysis.Analyzer;
@@ -26,7 +30,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.*;
 
-public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSerializable {
+public class LuceneIndexCreationProfile implements CacheServiceProfile, VersionedDataSerializable {
 
   private String indexName;
 
@@ -36,18 +40,23 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
 
   private Map<String, String> fieldAnalyzers;
 
+  private String serializerClass = HeterogeneousLuceneSerializer.class.getSimpleName();
+
   private String regionPath;
 
   /* Used by DataSerializer */
   public LuceneIndexCreationProfile() {}
 
   public LuceneIndexCreationProfile(String indexName, String regionPath, String[] fieldNames,
-      Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers) {
+      Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers, LuceneSerializer serializer) {
     this.indexName = indexName;
     this.regionPath = regionPath;
     this.fieldNames = fieldNames;
     this.analyzerClass = analyzer.getClass().getSimpleName();
     initializeFieldAnalyzers(fieldAnalyzers);
+    if (serializer != null) {
+      this.serializerClass = serializer.getClass().getSimpleName();
+    }
   }
 
   public String getIndexName() {
@@ -66,6 +75,10 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
     return this.fieldAnalyzers;
   }
 
+  public String getSerializerClass() {
+    return this.serializerClass;
+  }
+
   protected void initializeFieldAnalyzers(Map<String, Analyzer> fieldAnalyzers) {
     this.fieldAnalyzers = new HashMap<>();
     for (String field : fieldNames) {
@@ -129,6 +142,13 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
         }
       }
     }
+
+    if (!getSerializerClass().equals(remoteProfile.getSerializerClass())) {
+      return LocalizedStrings.LuceneService_CANNOT_CREATE_INDEX_0_ON_REGION_1_WITH_SERIALIZER_2_BECAUSE_ANOTHER_MEMBER_DEFINES_THE_SAME_INDEX_WITH_DIFFERENT_SERIALIZER_3
+          .toString(getIndexName(), regionPath, getSerializerClass(),
+              remoteProfile.getSerializerClass());
+    }
+
     return result;
   }
 
@@ -143,6 +163,11 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
 
   @Override
   public void toData(DataOutput out) throws IOException {
+    toDataPre_GEODE_1_3_0_0(out);
+    DataSerializer.writeString(this.serializerClass, out);
+  }
+
+  public void toDataPre_GEODE_1_3_0_0(DataOutput out) throws IOException {
     DataSerializer.writeString(this.indexName, out);
     DataSerializer.writeString(this.regionPath, out);
     DataSerializer.writeStringArray(this.fieldNames, out);
@@ -152,6 +177,11 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    fromDataPre_GEODE_1_3_0_0(in);
+    this.serializerClass = DataSerializer.readString(in);
+  }
+
+  public void fromDataPre_GEODE_1_3_0_0(DataInput in) throws IOException, ClassNotFoundException {
     this.indexName = DataSerializer.readString(in);
     this.regionPath = DataSerializer.readString(in);
     this.fieldNames = DataSerializer.readStringArray(in);
@@ -164,10 +194,15 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, DataSeri
         .append(this.indexName).append("; regionPath=").append(this.regionPath)
         .append("; fieldNames=").append(Arrays.toString(this.fieldNames)).append("; analyzerClass=")
         .append(this.analyzerClass).append("; fieldAnalyzers=").append(this.fieldAnalyzers)
-        .append("]").toString();
+        .append("; serializer=").append(this.serializerClass).append("]").toString();
   }
 
   public String getRegionPath() {
     return this.regionPath;
   }
+
+  @Override
+  public Version[] getSerializationVersions() {
+    return new Version[] {Version.GEODE_130};
+  }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
index 8866249..70e7e0b 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
@@ -103,7 +103,7 @@ public class LuceneRegionListener implements RegionListener {
 
       // Add index creation profile
       internalRegionArgs.addCacheServiceProfile(new LuceneIndexCreationProfile(this.indexName,
-          this.regionPath, this.fields, this.analyzer, this.fieldAnalyzers));
+          this.regionPath, this.fields, this.analyzer, this.fieldAnalyzers, serializer));
 
       luceneIndex = this.service.beforeDataRegionCreated(this.indexName, this.regionPath, attrs,
           this.analyzer, this.fieldAnalyzers, aeqId, serializer, this.fields);
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
index 60e196c..6af9a07 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
@@ -206,8 +206,8 @@ public class LuceneServiceImpl implements InternalLuceneService {
       regionPath = "/" + regionPath;
     }
 
-    registerDefinedIndex(indexName, regionPath,
-        new LuceneIndexCreationProfile(indexName, regionPath, fields, analyzer, fieldAnalyzers));
+    registerDefinedIndex(indexName, regionPath, new LuceneIndexCreationProfile(indexName,
+        regionPath, fields, analyzer, fieldAnalyzers, serializer));
 
     Region region = cache.getRegion(regionPath);
     if (region != null) {
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/DummyLuceneSerializer.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/DummyLuceneSerializer.java
new file mode 100644
index 0000000..2a31768
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/DummyLuceneSerializer.java
@@ -0,0 +1,29 @@
+/*
+ * 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 org.apache.geode.cache.lucene;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+
+public class DummyLuceneSerializer implements LuceneSerializer {
+
+  @Override
+  public Collection<Document> toDocuments(Object value) {
+    return Collections.emptyList();
+  }
+
+}
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
index b83ca44..e41cf85 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.cache.lucene;
 
+import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
 import org.apache.geode.test.dunit.SerializableRunnableIF;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -324,6 +325,28 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
     dataStore2.invoke(() -> initDataStore(createIndex2, regionType));
   }
 
+  @Test
+  @Parameters("PARTITION")
+  public void verifyDifferentSerializerShouldFail(RegionTestableType regionType) {
+    SerializableRunnableIF createIndex1 = getIndexWithDefaultSerializer();
+    dataStore1.invoke(() -> initDataStore(createIndex1, regionType));
+
+    SerializableRunnableIF createIndex2 = getIndexWithDummySerializer();
+    dataStore2.invoke(() -> initDataStore(createIndex2, regionType,
+        CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_SERIALIZER));
+  }
+
+  @Test
+  @Parameters("PARTITION")
+  public void verifyDifferentSerializerShouldFail2(RegionTestableType regionType) {
+    SerializableRunnableIF createIndex1 = getHeterogeneousLuceneSerializerCreationProfile();
+    dataStore1.invoke(() -> initDataStore(createIndex1, regionType));
+
+    SerializableRunnableIF createIndex2 = getIndexWithDummySerializer();
+    dataStore2.invoke(() -> initDataStore(createIndex2, regionType,
+        CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_SERIALIZER));
+  }
+
   protected String getXmlFileForTest(String testName) {
     return TestUtil.getResourcePath(getClass(),
         getClassSimpleName() + "." + testName + ".cache.xml");
@@ -462,4 +485,29 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
+
+  protected SerializableRunnableIF getIndexWithDummySerializer() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields(new String[] {"field1", "field2"})
+          .setLuceneSerializer(new DummyLuceneSerializer()).create(INDEX_NAME, REGION_NAME);
+    };
+  }
+
+  protected SerializableRunnableIF getIndexWithDefaultSerializer() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields(new String[] {"field1", "field2"})
+          .create(INDEX_NAME, REGION_NAME);
+    };
+  }
+
+  protected SerializableRunnableIF getHeterogeneousLuceneSerializerCreationProfile() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields(new String[] {"field1", "field2"})
+          .setLuceneSerializer(new HeterogeneousLuceneSerializer(new String[] {"field1", "field2"}))
+          .create(INDEX_NAME, REGION_NAME);
+    };
+  }
 }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
index b378ca5..c653c2f 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfileJUnitTest.java
@@ -15,8 +15,14 @@
 package org.apache.geode.cache.lucene.internal;
 
 import org.apache.geode.CopyHelper;
+import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.lucene.DummyLuceneSerializer;
+import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
+import org.apache.geode.internal.HeapDataOutputStream;
+import org.apache.geode.internal.Version;
 import org.apache.geode.test.junit.categories.UnitTest;
+
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
 import org.apache.lucene.analysis.Analyzer;
@@ -27,6 +33,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -54,10 +63,44 @@ public class LuceneIndexCreationProfileJUnitTest {
     return $(new Object[] {getOneFieldLuceneIndexCreationProfile()},
         new Object[] {getTwoFieldLuceneIndexCreationProfile()},
         new Object[] {getTwoAnalyzersLuceneIndexCreationProfile()},
+        new Object[] {getDummySerializerCreationProfile()},
         new Object[] {getNullField1AnalyzerLuceneIndexCreationProfile()});
   }
 
   @Test
+  @Parameters(method = "getProfileWithSerializer")
+  public void toDataFromDataShouldContainSerializer(LuceneIndexCreationProfile profile,
+      String expectedSerializerCLassName) throws IOException, ClassNotFoundException {
+    HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+    DataSerializer.writeObject(profile, hdos);
+    byte[] outputArray = hdos.toByteArray();
+    ByteArrayInputStream bais = new ByteArrayInputStream(outputArray);
+    LuceneIndexCreationProfile profile2 = DataSerializer.readObject(new DataInputStream(bais));
+    assertEquals(expectedSerializerCLassName, profile2.getSerializerClass());
+  }
+
+  private Object[] getProfileWithSerializer() {
+    return $(new Object[] {getDefaultSerializerCreationProfile(), "HeterogeneousLuceneSerializer"},
+        new Object[] {getDummySerializerCreationProfile(), "DummyLuceneSerializer"}, new Object[] {
+            getHeterogeneousLuceneSerializerCreationProfile(), "HeterogeneousLuceneSerializer"});
+  }
+
+  private LuceneIndexCreationProfile getDefaultSerializerCreationProfile() {
+    return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME, new String[] {"field1"},
+        new StandardAnalyzer(), null, null);
+  }
+
+  private LuceneIndexCreationProfile getDummySerializerCreationProfile() {
+    return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME, new String[] {"field1"},
+        new StandardAnalyzer(), null, new DummyLuceneSerializer());
+  }
+
+  private LuceneIndexCreationProfile getHeterogeneousLuceneSerializerCreationProfile() {
+    return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME, new String[] {"field1"},
+        new StandardAnalyzer(), null, new HeterogeneousLuceneSerializer(new String[] {"field1"}));
+  }
+
+  @Test
   @Parameters(method = "getCheckCompatibilityProfiles")
   public void testCheckCompatibility(LuceneIndexCreationProfile myProfile,
       LuceneIndexCreationProfile otherProfile, String expectedResult) {
@@ -80,6 +123,10 @@ public class LuceneIndexCreationProfileJUnitTest {
         new Object[] {getNullField2AnalyzerLuceneIndexCreationProfile(),
             getNullField1AnalyzerLuceneIndexCreationProfile(),
             CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS_1},
+        new Object[] {getDefaultSerializerCreationProfile(), getDummySerializerCreationProfile(),
+            CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_SERIALIZER},
+        new Object[] {getDefaultSerializerCreationProfile(),
+            getHeterogeneousLuceneSerializerCreationProfile(), null},
         new Object[] {getNullField1AnalyzerLuceneIndexCreationProfile(),
             getNullField2AnalyzerLuceneIndexCreationProfile(),
             LuceneTestUtilities.CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_ANALYZERS_2});
@@ -87,12 +134,12 @@ public class LuceneIndexCreationProfileJUnitTest {
 
   private LuceneIndexCreationProfile getOneFieldLuceneIndexCreationProfile() {
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME, new String[] {"field1"},
-        new StandardAnalyzer(), null);
+        new StandardAnalyzer(), null, null);
   }
 
   private LuceneIndexCreationProfile getTwoFieldLuceneIndexCreationProfile() {
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME,
-        new String[] {"field1", "field2"}, new StandardAnalyzer(), null);
+        new String[] {"field1", "field2"}, new StandardAnalyzer(), null, null);
   }
 
   private LuceneIndexCreationProfile getOneAnalyzerLuceneIndexCreationProfile(Analyzer analyzer) {
@@ -100,7 +147,7 @@ public class LuceneIndexCreationProfileJUnitTest {
     fieldAnalyzers.put("field1", analyzer);
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME,
         new String[] {"field1", "field2"}, getPerFieldAnalyzerWrapper(fieldAnalyzers),
-        fieldAnalyzers);
+        fieldAnalyzers, null);
   }
 
   private LuceneIndexCreationProfile getTwoAnalyzersLuceneIndexCreationProfile() {
@@ -109,7 +156,7 @@ public class LuceneIndexCreationProfileJUnitTest {
     fieldAnalyzers.put("field2", new KeywordAnalyzer());
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME,
         new String[] {"field1", "field2"}, getPerFieldAnalyzerWrapper(fieldAnalyzers),
-        fieldAnalyzers);
+        fieldAnalyzers, null);
   }
 
   private LuceneIndexCreationProfile getNullField1AnalyzerLuceneIndexCreationProfile() {
@@ -118,7 +165,7 @@ public class LuceneIndexCreationProfileJUnitTest {
     fieldAnalyzers.put("field2", new KeywordAnalyzer());
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME,
         new String[] {"field1", "field2"}, getPerFieldAnalyzerWrapper(fieldAnalyzers),
-        fieldAnalyzers);
+        fieldAnalyzers, null);
   }
 
   private LuceneIndexCreationProfile getNullField2AnalyzerLuceneIndexCreationProfile() {
@@ -127,7 +174,7 @@ public class LuceneIndexCreationProfileJUnitTest {
     fieldAnalyzers.put("field2", null);
     return new LuceneIndexCreationProfile(INDEX_NAME, REGION_NAME,
         new String[] {"field1", "field2"}, getPerFieldAnalyzerWrapper(fieldAnalyzers),
-        fieldAnalyzers);
+        fieldAnalyzers, null);
   }
 
   private Analyzer getPerFieldAnalyzerWrapper(Map<String, Analyzer> fieldAnalyzers) {
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
index 17f4dea..307b649 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
@@ -79,6 +79,8 @@ public class LuceneTestUtilities {
       "Cannot create Lucene index index2 on region /region because it is not defined in another member.";
   public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_3 =
       "Cannot create Lucene index index on region /region because it is not defined in another member.";
+  public static final String CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_SERIALIZER =
+      "Cannot create Lucene index index on region /region with serializer DummyLuceneSerializer because another member defines the same index with different serializer HeterogeneousLuceneSerializer.";
 
   public static String Quarter1 = "Q1";
   public static String Quarter2 = "Q2";

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].