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

geode git commit: GEODE-2595: Change LuceneService.createIndex to use a factory

Repository: geode
Updated Branches:
  refs/heads/develop 492d01add -> b74a0bd4f


GEODE-2595: Change LuceneService.createIndex to use a factory

Changing LuceneService.createIndex to createIndexFactory and
using a factory pattern to create the index.

This allows us to introduce new options to the index create without
breaking backwards compatibility in the future.


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

Branch: refs/heads/develop
Commit: b74a0bd4fc867d80d2ac13066321f7547fb3688e
Parents: 492d01a
Author: Dan Smith <up...@apache.org>
Authored: Wed Mar 8 17:15:44 2017 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Thu Mar 9 17:50:43 2017 -0800

----------------------------------------------------------------------
 .../geode/cache/lucene/LuceneIndexFactory.java  | 86 ++++++++++++++++++++
 .../geode/cache/lucene/LuceneService.java       | 57 +++++--------
 .../lucene/internal/LuceneIndexFactory.java     | 25 ------
 .../lucene/internal/LuceneIndexFactoryImpl.java | 66 +++++++++++++++
 .../lucene/internal/LuceneIndexImplFactory.java | 25 ++++++
 .../lucene/internal/LuceneRawIndexFactory.java  |  2 +-
 .../lucene/internal/LuceneServiceImpl.java      | 18 ++--
 .../functions/LuceneCreateIndexFunction.java    | 16 ++--
 .../lucene/LuceneIndexCreationDUnitTest.java    | 32 ++++----
 .../LuceneIndexCreationIntegrationTest.java     | 19 ++---
 ...IndexCreationPersistenceIntegrationTest.java |  9 +-
 .../lucene/LuceneIndexDestroyDUnitTest.java     |  6 +-
 .../LuceneIndexMaintenanceIntegrationTest.java  | 21 +++--
 .../cache/lucene/LuceneQueriesDUnitTest.java    |  8 +-
 .../lucene/LuceneQueriesIntegrationTest.java    | 25 +++---
 ...LuceneQueriesPersistenceIntegrationTest.java |  2 +-
 .../geode/cache/lucene/PaginationDUnitTest.java |  6 +-
 .../geode/cache/lucene/RebalanceDUnitTest.java  | 21 +----
 .../RebalanceWithRedundancyDUnitTest.java       |  9 +-
 .../lucene/internal/LuceneIndexFactorySpy.java  |  4 +-
 .../LuceneIndexRecoveryHAIntegrationTest.java   |  3 +-
 .../internal/LuceneServiceImplJUnitTest.java    |  2 +-
 .../cli/LuceneIndexCommandsDUnitTest.java       | 15 +---
 .../LuceneCreateIndexFunctionJUnitTest.java     | 23 +++---
 .../DumpDirectoryFilesIntegrationTest.java      |  3 +-
 .../management/LuceneManagementDUnitTest.java   |  3 +-
 .../IndexRepositoryImplPerformanceTest.java     |  2 +-
 ...neIndexXmlGeneratorIntegrationJUnitTest.java |  2 +-
 .../cache/lucene/test/LuceneTestUtilities.java  | 10 +--
 29 files changed, 313 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexFactory.java
new file mode 100644
index 0000000..2472ec0
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.geode.annotations.Experimental;
+import org.apache.lucene.analysis.Analyzer;
+
+import java.util.Map;
+
+/**
+ * A factory for creating a lucene index on the current member. Obtain a factory from
+ * {@link LuceneService#createIndexFactory()}.
+ *
+ * Configure the index using the add methods, and then call {@link #create(String, String)} to
+ * create the index.
+ */
+@Experimental
+public interface LuceneIndexFactory {
+
+  /**
+   * Add a field to be indexed
+   * 
+   * @param name A field of the object to index. Only fields listed here will be stored in the
+   *        index. Fields should map to PDX fieldNames if the object is serialized with PDX, or to
+   *        java fields on the object otherwise. The special field name
+   *        {@link LuceneService#REGION_VALUE_FIELD} indicates that the entire value should be
+   *        stored as a single field in the index.
+   */
+  LuceneIndexFactory addField(String name);
+
+  /**
+   * Set the list of fields to be indexed.
+   * 
+   * @param fields Fields of the object to index. Only fields listed here will be stored in the
+   *        index. Fields should map to PDX fieldNames if the object is serialized with PDX, or to
+   *        java fields on the object otherwise. The special field name
+   *        {@link LuceneService#REGION_VALUE_FIELD} indicates that the entire value should be
+   *        stored as a single field in the index.
+   */
+  LuceneIndexFactory setFields(String... fields);
+
+  /**
+   * Add a field to be indexed, using the specified analyzer.
+   *
+   * @param name A field of the object to index. Only fields listed here will be stored in the
+   *        index. Fields should map to PDX fieldNames if the object is serialized with PDX, or to
+   *        java fields on the object otherwise. The special field name
+   *        {@link LuceneService#REGION_VALUE_FIELD} indicates that the entire value should be
+   *        stored as a single field in the index.
+   * @param analyzer The analyzer to use for this this field. Analyzers are used by Lucene to
+   *        tokenize your field into individual words.
+   */
+  LuceneIndexFactory addField(String name, Analyzer analyzer);
+
+  /**
+   * Set the list of fields to be indexed.
+   * 
+   * @param fieldMap Fields of the object to index, with the analyzer to be used for each field.
+   *        Only fields listed here will be stored in the index. Fields should map to PDX fieldNames
+   *        if the object is serialized with PDX, or to java fields on the object otherwise. The
+   *        special field name {@link LuceneService#REGION_VALUE_FIELD} indicates that the entire
+   *        value should be stored as a single field in the index.
+   */
+  LuceneIndexFactory setFields(Map<String, Analyzer> fieldMap);
+
+  /**
+   * Create the index on this member.
+   * 
+   * @param indexName name of the index.
+   * @param regionPath The region to index. The entries added to this region will be indexes.
+   */
+  void create(String indexName, String regionPath);
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
index f035442..24ca902 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
@@ -14,17 +14,17 @@
  */
 package org.apache.geode.cache.lucene;
 
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
+import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.GemFireCache;
+import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
-import org.apache.geode.annotations.Experimental;
-import org.apache.geode.cache.GemFireCache;
-import org.apache.geode.cache.lucene.internal.LuceneIndexCreationProfile;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 /**
  *
@@ -37,27 +37,26 @@ import org.apache.geode.cache.lucene.internal.LuceneIndexCreationProfile;
  * </p>
  * <p>
  * Lucene indexes can be created using gfsh, xml, or the java API. Below is an example of creating a
- * Lucene index with the java API. The Lucene index created on each member that will host data for
- * the region.
+ * Lucene index with the java API. The Lucene index should be created on each member that has the
+ * region that is being indexed.
  * </p>
  * 
  * <pre>
  * {
  *   &#64;code
- *   LuceneIndex index =
- *       luceneService.createIndex(indexName, regionName, "field1", "field2", "field3");
+*       luceneService.createCreateIndexFactory()
+ *        .addField("name")
+ *        .addField("zipcode")
+ *        .addField("email", new KeywordAnalyzer())
+ *        .create(indexName, regionName);
  * }
  * </pre>
  * <p>
- * You can also specify what {@link Analyzer} to use for each field.
+ * You can also specify what {@link Analyzer} to use for each field. In the example above, email is
+ * being tokenized with the KeywordAnalyzer so it is treated as a single word. The default analyzer
+ * if non is specified is the {@link StandardAnalyzer}
  * </p>
  * 
- * <pre>
- * {
- *   &#64;code
- *   LuceneIndex index = luceneService.createIndex(indexName, regionName, analyzerPerField);
- * }
- * </pre>
  *
  * Indexes should be created on all peers that host the region being indexed. Clients do not need to
  * define the index, they can directly execute queries using this service.
@@ -107,27 +106,9 @@ public interface LuceneService {
   public String REGION_VALUE_FIELD = "__REGION_VALUE_FIELD";
 
   /**
-   * Create a Lucene index using default analyzer.
-   * 
-   * @param fields The fields of the object to index. Only fields listed here will be stored in the
-   *        index. Fields should map to PDX fieldNames if the object is serialized with PDX, or to
-   *        java fields on the object otherwise. The special field name {@link #REGION_VALUE_FIELD}
-   *        indicates that the entire value should be stored as a single field in the index.
-   */
-  public void createIndex(String indexName, String regionPath, String... fields);
-
-  /**
-   * Create a Lucene index using specified {@link Analyzer} per field. Analyzers are used by Lucene
-   * to tokenize your field into individual words.
-   * 
-   * @param indexName index name
-   * @param regionPath region name
-   * @param analyzerPerField A map of fields to analyzers. See
-   *        {@link #createIndex(String, String, String...)} for details on valid values for fields.
-   *        Each field will be tokenized using the provided Analyzer.
+   * Get a factory for creating a lucene index on this member.
    */
-  public void createIndex(String indexName, String regionPath,
-      Map<String, Analyzer> analyzerPerField);
+  public LuceneIndexFactory createIndexFactory();
 
   /**
    * Destroy the Lucene index

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactory.java
deleted file mode 100755
index fe1fc52..0000000
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactory.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.internal;
-
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-
-public class LuceneIndexFactory {
-  public LuceneIndexFactory() {}
-
-  public LuceneIndexImpl create(String indexName, String regionPath, GemFireCacheImpl cache) {
-    return new LuceneIndexForPartitionedRegion(indexName, regionPath, cache);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
new file mode 100644
index 0000000..23f2206
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
@@ -0,0 +1,66 @@
+/*
+ * 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.internal;
+
+import org.apache.geode.cache.lucene.LuceneIndexFactory;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public class LuceneIndexFactoryImpl implements org.apache.geode.cache.lucene.LuceneIndexFactory {
+  private final LuceneServiceImpl service;
+  private final Map<String, Analyzer> fields = new LinkedHashMap<String, Analyzer>();
+
+
+  public LuceneIndexFactoryImpl(final LuceneServiceImpl luceneService) {
+    this.service = luceneService;
+  }
+
+  @Override
+  public LuceneIndexFactory addField(final String name) {
+    return addField(name, new StandardAnalyzer());
+  }
+
+  @Override
+  public LuceneIndexFactory setFields(final String... fields) {
+    this.fields.clear();
+    for (String field : fields) {
+      addField(field);
+    }
+    return this;
+  }
+
+  @Override
+  public LuceneIndexFactory addField(final String name, final Analyzer analyzer) {
+    fields.put(name, analyzer);
+    return this;
+  }
+
+  @Override
+  public LuceneIndexFactory setFields(final Map<String, Analyzer> fieldMap) {
+    this.fields.clear();
+    this.fields.putAll(fieldMap);
+    return this;
+  }
+
+  @Override
+  public void create(final String indexName, final String regionPath) {
+    service.createIndex(indexName, regionPath, fields);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImplFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImplFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImplFactory.java
new file mode 100755
index 0000000..5da2732
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImplFactory.java
@@ -0,0 +1,25 @@
+/*
+ * 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.internal;
+
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+
+public class LuceneIndexImplFactory {
+  public LuceneIndexImplFactory() {}
+
+  public LuceneIndexImpl create(String indexName, String regionPath, GemFireCacheImpl cache) {
+    return new LuceneIndexForPartitionedRegion(indexName, regionPath, cache);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
index 825786c..cf445c0 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
@@ -16,7 +16,7 @@ package org.apache.geode.cache.lucene.internal;
 
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 
-public class LuceneRawIndexFactory extends LuceneIndexFactory {
+public class LuceneRawIndexFactory extends LuceneIndexImplFactory {
   public LuceneIndexImpl create(String indexName, String regionPath, GemFireCacheImpl cache) {
     return new LuceneRawIndex(indexName, regionPath, cache);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
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 a212094..41d2991 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
@@ -69,7 +69,7 @@ import org.apache.geode.internal.logging.LogService;
  * @since GemFire 8.5
  */
 public class LuceneServiceImpl implements InternalLuceneService {
-  public static LuceneIndexFactory luceneIndexFactory = new LuceneIndexFactory();
+  public static LuceneIndexImplFactory luceneIndexFactory = new LuceneIndexImplFactory();
   private static final Logger logger = LogService.getLogger();
 
   private GemFireCacheImpl cache;
@@ -82,6 +82,11 @@ public class LuceneServiceImpl implements InternalLuceneService {
   }
 
   @Override
+  public org.apache.geode.cache.lucene.LuceneIndexFactory createIndexFactory() {
+    return new LuceneIndexFactoryImpl(this);
+  }
+
+  @Override
   public Cache getCache() {
     return this.cache;
   }
@@ -127,17 +132,6 @@ public class LuceneServiceImpl implements InternalLuceneService {
     return getUniqueIndexName(indexName, regionPath) + regionSuffix;
   }
 
-  @Override
-  public void createIndex(String indexName, String regionPath, String... fields) {
-    if (fields == null || fields.length == 0) {
-      throw new IllegalArgumentException("At least one field must be indexed");
-    }
-    StandardAnalyzer analyzer = new StandardAnalyzer();
-
-    createIndex(indexName, regionPath, analyzer, null, fields);
-  }
-
-  @Override
   public void createIndex(String indexName, String regionPath,
       Map<String, Analyzer> fieldAnalyzers) {
     if (fieldAnalyzers == null || fieldAnalyzers.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction.java
index 075bbd7..f1b9c82 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction.java
@@ -15,15 +15,11 @@
 
 package org.apache.geode.cache.lucene.internal.cli.functions;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.FunctionAdapter;
 import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
@@ -73,18 +69,20 @@ public class LuceneCreateIndexFunction extends FunctionAdapter implements Intern
       String[] fields = indexInfo.getSearchableFieldNames();
       String[] analyzerName = indexInfo.getFieldAnalyzers();
 
+      final LuceneIndexFactory indexFactory = service.createIndexFactory();
       if (analyzerName == null || analyzerName.length == 0) {
-        service.createIndex(indexInfo.getIndexName(), indexInfo.getRegionPath(), fields);
+        for (String field : fields) {
+          indexFactory.addField(field);
+        }
       } else {
         if (analyzerName.length != fields.length)
           throw new Exception("Mismatch in lengths of fields and analyzers");
-        Map<String, Analyzer> fieldAnalyzer = new HashMap<>();
         for (int i = 0; i < fields.length; i++) {
           Analyzer analyzer = toAnalyzer(analyzerName[i]);
-          fieldAnalyzer.put(fields[i], analyzer);
+          indexFactory.addField(fields[i], analyzer);
         }
-        service.createIndex(indexInfo.getIndexName(), indexInfo.getRegionPath(), fieldAnalyzer);
       }
+      indexFactory.create(indexInfo.getIndexName(), indexInfo.getRegionPath());
 
       // TODO - update cluster configuration by returning a valid XmlEntity
       XmlEntity xmlEntity = null;

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
----------------------------------------------------------------------
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 60f8b65..a16646e 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
@@ -186,13 +186,13 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
   public void verifyDifferentIndexNamesFails(RegionTestableType regionType) {
     SerializableRunnableIF createIndex1 = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME + "1", REGION_NAME, "field1");
+      luceneService.createIndexFactory().addField("field1").create(INDEX_NAME + "1", REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex1, regionType));
 
     SerializableRunnableIF createIndex2 = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME + "2", REGION_NAME, "field1");
+      luceneService.createIndexFactory().addField("field1").create(INDEX_NAME + "2", REGION_NAME);
     };
     dataStore2.invoke(
         () -> initDataStore(createIndex2, regionType, CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_NAMES));
@@ -219,8 +219,8 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
 
     SerializableRunnableIF createIndex2 = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1");
-      luceneService.createIndex(INDEX_NAME + "2", REGION_NAME, "field2");
+      luceneService.createIndexFactory().addField("field1").create(INDEX_NAME, REGION_NAME);
+      luceneService.createIndexFactory().addField("field2").create(INDEX_NAME + "2", REGION_NAME);
     };
     dataStore2.invoke(() -> initDataStore(createIndex2, regionType,
         CANNOT_CREATE_LUCENE_INDEX_DIFFERENT_INDEXES_2));
@@ -362,22 +362,23 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
   protected SerializableRunnableIF getFieldsIndexWithOneField() {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1");
+      luceneService.createIndexFactory().addField("field1").create(INDEX_NAME, REGION_NAME);
     };
   }
 
   protected SerializableRunnableIF getFieldsIndexWithTwoFields() {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+      luceneService.createIndexFactory().addField("field1").addField("field2").create(INDEX_NAME,
+          REGION_NAME);
     };
   }
 
   protected SerializableRunnableIF get2FieldsIndexes() {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME + "_1", REGION_NAME, "field1");
-      luceneService.createIndex(INDEX_NAME + "_2", REGION_NAME, "field2");
+      luceneService.createIndexFactory().addField("field1").create(INDEX_NAME + "_1", REGION_NAME);
+      luceneService.createIndexFactory().addField("field2").create(INDEX_NAME + "_2", REGION_NAME);
     };
   }
 
@@ -385,7 +386,8 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
       for (int count = 1; count <= numberOfIndexes; count++) {
-        luceneService.createIndex(INDEX_NAME + "_" + count, REGION_NAME, "field" + count);
+        luceneService.createIndexFactory().addField("field" + count)
+            .create(INDEX_NAME + "_" + count, REGION_NAME);
       }
     };
   }
@@ -410,7 +412,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", null);
       analyzers.put("field2", new KeywordAnalyzer());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 
@@ -420,7 +422,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", new KeywordAnalyzer());
       analyzers.put("field2", null);
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 
@@ -430,7 +432,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", analyzerClass.newInstance());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 
@@ -439,7 +441,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", new KeywordAnalyzer());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 
@@ -449,7 +451,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", new KeywordAnalyzer());
       analyzers.put("field2", new KeywordAnalyzer());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 
@@ -459,7 +461,7 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
       Map<String, Analyzer> analyzers = new HashMap<>();
       analyzers.put("field1", new StandardAnalyzer());
       analyzers.put("field2", new KeywordAnalyzer());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
index b696676..80e77e9 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -15,9 +15,7 @@
 
 package org.apache.geode.cache.lucene;
 
-import static org.apache.geode.cache.RegionShortcut.*;
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
-import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
@@ -28,9 +26,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import java.util.stream.Collector;
 import java.util.stream.Collectors;
 
 import org.apache.geode.cache.EvictionAction;
@@ -42,7 +38,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.internal.LuceneIndexCreationProfile;
-import org.apache.geode.cache.lucene.internal.LuceneIndexFactory;
+import org.apache.geode.cache.lucene.internal.LuceneIndexImplFactory;
 import org.apache.geode.cache.lucene.internal.LuceneRawIndex;
 import org.apache.geode.cache.lucene.internal.LuceneRawIndexFactory;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
@@ -52,7 +48,6 @@ import org.apache.geode.internal.cache.BucketNotFoundException;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
@@ -90,7 +85,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     final RecordingAnalyzer field2Analyzer = new RecordingAnalyzer();
     analyzers.put("field1", field1Analyzer);
     analyzers.put("field2", field2Analyzer);
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
@@ -166,7 +161,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     analyzers.put("field2", field2Analyzer);
     LuceneServiceImpl.luceneIndexFactory = new LuceneRawIndexFactory();
     try {
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+      luceneService.createIndexFactory().setFields(analyzers).create(INDEX_NAME, REGION_NAME);
       Region region = createRegion();
       final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
       assertTrue(index instanceof LuceneRawIndex);
@@ -176,7 +171,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
       assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
       assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
     } finally {
-      LuceneServiceImpl.luceneIndexFactory = new LuceneIndexFactory();
+      LuceneServiceImpl.luceneIndexFactory = new LuceneIndexImplFactory();
     }
   }
 
@@ -225,8 +220,10 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   @Test
   public void shouldReturnAllDefinedIndexes() {
     LuceneServiceImpl luceneServiceImpl = (LuceneServiceImpl) luceneService;
-    luceneServiceImpl.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2", "field3");
-    luceneServiceImpl.createIndex("index2", "region2", "field4", "field5", "field6");
+    luceneServiceImpl.createIndexFactory().setFields("field1", "field2", "field3")
+        .create(INDEX_NAME, REGION_NAME);
+    luceneServiceImpl.createIndexFactory().setFields("field4", "field5", "field6").create("index2",
+        "region2");
     final Collection<LuceneIndexCreationProfile> indexList =
         luceneServiceImpl.getAllDefinedIndexes();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index 4b27ee6..8f65574 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -164,8 +164,10 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   @Test
   @Parameters(method = "getRegionShortcuts")
   public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws Exception {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME + "_1", REGION_NAME, "field1");
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME + "_2", REGION_NAME, "field2");
+    LuceneServiceProvider.get(this.cache).createIndexFactory().setFields("field1")
+        .create(INDEX_NAME + "_1", REGION_NAME);
+    LuceneServiceProvider.get(this.cache).createIndexFactory().setFields("field2")
+        .create(INDEX_NAME + "_2", REGION_NAME);
     Region region = cache.createRegionFactory(shortcut).create(REGION_NAME);
     region.put("key1", new TestObject());
     verifyQueryResultSize(INDEX_NAME + "_1", REGION_NAME, "field1:world", DEFAULT_FIELD, 1);
@@ -175,7 +177,8 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   @Test
   @Parameters(method = "getRegionShortcuts")
   public void shouldCreateInternalRegionsForIndex(RegionShortcut shortcut) {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+    luceneService.createIndexFactory().setFields("field1", "field2").create(INDEX_NAME,
+        REGION_NAME);
 
     // Create partitioned region
     createRegion(REGION_NAME, shortcut);

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
index 496cdeb..cf79248 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
@@ -331,15 +331,15 @@ public class LuceneIndexDestroyDUnitTest extends LuceneDUnitTest {
   private SerializableRunnableIF createIndex(String indexName, String field) {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(indexName, REGION_NAME, field);
+      luceneService.createIndexFactory().setFields(field).create(INDEX_NAME, REGION_NAME);
     };
   }
 
   private SerializableRunnableIF createIndexes() {
     return () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME + "0", REGION_NAME, "text");
-      luceneService.createIndex(INDEX_NAME + "1", REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME + "0", REGION_NAME);
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME + "1", REGION_NAME);
     };
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index 5ac01b8..00abb99 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -43,7 +43,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void indexIsNotUpdatedIfTransactionHasNotCommittedYet() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     region.put("object-1", new TestObject("title 1", "hello world"));
@@ -69,7 +70,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void indexIsUpdatedAfterTransactionHasCommitted() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     region.put("object-1", new TestObject("title 1", "hello world"));
@@ -96,7 +98,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void indexIsNotUpdatedAfterTransactionRollback() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     region.put("object-1", new TestObject("title 1", "hello world"));
@@ -123,7 +126,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void statsAreUpdatedAfterACommit() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     region.put("object-1", new TestObject("title 1", "hello world"));
@@ -146,7 +150,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void indexShouldBeUpdatedWithRegionExpirationDestroyOperation() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     // Configure PR with expiration operation set to destroy
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
@@ -173,7 +178,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void nullValuesShouldNotCauseAnException() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     // Configure PR with expiration operation set to destroy
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
@@ -192,7 +198,8 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
 
   @Test
   public void entriesFlushedToIndexAfterWaitForFlushCalled() throws InterruptedException {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     LuceneTestUtilities.pauseSender(cache);

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
index 5db2ed4..d259205 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
@@ -50,7 +50,7 @@ public class LuceneQueriesDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().addField("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));
@@ -68,7 +68,7 @@ public class LuceneQueriesDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().addField("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));
@@ -85,7 +85,7 @@ public class LuceneQueriesDUnitTest extends LuceneQueriesAccessorBase {
   public void canQueryWithCustomLuceneQueryObject(RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().addField("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));
@@ -113,7 +113,7 @@ public class LuceneQueriesDUnitTest extends LuceneQueriesAccessorBase {
       throws InterruptedException {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().addField("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
index 6420307..5b85c19 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -44,7 +44,6 @@ import org.apache.lucene.search.Query;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.contrib.java.lang.system.SystemOutRule;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
@@ -54,10 +53,8 @@ import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.test.TestObject;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.PartitionAttributesImpl;
 import org.apache.geode.pdx.JSONFormatter;
 import org.apache.geode.pdx.PdxInstance;
-import org.apache.geode.pdx.internal.AutoSerializableManager.ObjectArrayField;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -76,7 +73,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", new StandardAnalyzer());
     fields.put("field2", new KeywordAnalyzer());
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    luceneService.createIndexFactory().setFields(fields).create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -134,7 +131,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   public void shouldQueryUsingIntRangeQueryProvider() throws Exception {
     // Note: range query on numeric field has some limitations. But IntRangeQueryProvider
     // provided basic functionality
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, LuceneService.REGION_VALUE_FIELD);
+    luceneService.createIndexFactory().setFields(LuceneService.REGION_VALUE_FIELD)
+        .create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -153,7 +151,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   public void queryParserCannotQueryByRange() throws Exception {
     // Note: range query on numeric field has some limitations. But IntRangeQueryProvider
     // provided basic functionality
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, LuceneService.REGION_VALUE_FIELD);
+    luceneService.createIndexFactory().setFields(LuceneService.REGION_VALUE_FIELD)
+        .create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -208,7 +207,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
 
   private LuceneQuery<Object, Object> addValuesAndCreateQuery(int pagesize)
       throws InterruptedException {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+    luceneService.createIndexFactory().setFields("field1", "field2").create(INDEX_NAME,
+        REGION_NAME);
     region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -236,7 +236,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     // Note: fields has to contain "field1", otherwise, field1 will not be tokenized
     fields.put("field1", null);
     fields.put("field2", new MyCharacterAnalyzer());
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    luceneService.createIndexFactory().setFields(fields).create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -261,7 +261,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", null);
     fields.put("field2", null);
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    luceneService.createIndexFactory().setFields(fields).create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -279,7 +279,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     fields.put("name", null);
     fields.put("lastName", null);
     fields.put("address", null);
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    luceneService.createIndexFactory().setFields(fields).create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -297,7 +297,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
 
   @Test()
   public void shouldAllowQueryOnRegionWithStringValue() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, LuceneService.REGION_VALUE_FIELD);
+    luceneService.createIndexFactory().setFields(LuceneService.REGION_VALUE_FIELD)
+        .create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -310,7 +311,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   @Test()
   public void throwFunctionExceptionWhenGivenBadQuery() throws Exception {
     LuceneService luceneService = LuceneServiceProvider.get(cache);
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
 
     // Create a query that throws an exception

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
index 5fe2df5..f7bb5ab 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
@@ -63,7 +63,7 @@ public class LuceneQueriesPersistenceIntegrationTest extends LuceneIntegrationTe
     String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
 
     LuceneService service = LuceneServiceProvider.get(cache);
-    service.createIndex(INDEX_NAME, REGION_NAME, Type1.fields);
+    service.createIndexFactory().setFields(Type1.fields).create(INDEX_NAME, REGION_NAME);
 
     RegionFactory<String, Type1> regionFactory =
         cache.createRegionFactory(RegionShortcut.PARTITION);

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/PaginationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/PaginationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/PaginationDUnitTest.java
index 3ff86f7..ad6727e 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/PaginationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/PaginationDUnitTest.java
@@ -64,7 +64,7 @@ public class PaginationDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
 
 
@@ -104,7 +104,7 @@ public class PaginationDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
 
 
@@ -145,7 +145,7 @@ public class PaginationDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
 
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceDUnitTest.java
index 0142b37..ad5595b 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceDUnitTest.java
@@ -15,34 +15,21 @@
 
 package org.apache.geode.cache.lucene;
 
-import static org.apache.geode.cache.lucene.test.IndexRepositorySpy.doOnce;
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
 
 import java.util.stream.IntStream;
 
-import org.apache.geode.cache.lucene.internal.LuceneIndexFactorySpy;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.InitialImageOperation;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.partitioned.BecomePrimaryBucketMessage;
-import org.apache.geode.internal.cache.partitioned.BecomePrimaryBucketMessage.BecomePrimaryBucketResponse;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.junit.After;
 import org.junit.Test;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.control.RebalanceOperation;
-import org.apache.geode.cache.control.RebalanceResults;
-import org.apache.geode.cache.lucene.test.IndexRepositorySpy;
 import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
-import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.test.dunit.SerializableRunnableIF;
-import org.apache.geode.test.dunit.VM;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
@@ -125,7 +112,7 @@ public class RebalanceDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) throws InterruptedException {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     accessor.invoke(() -> initAccessor(createIndex, regionTestType));
@@ -147,7 +134,7 @@ public class RebalanceDUnitTest extends LuceneQueriesAccessorBase {
       RegionTestableType regionTestType) throws InterruptedException {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     accessor.invoke(() -> initAccessor(createIndex, regionTestType));
@@ -168,7 +155,7 @@ public class RebalanceDUnitTest extends LuceneQueriesAccessorBase {
   protected void putEntriesAndValidateQueryResults(RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     accessor.invoke(() -> initAccessor(createIndex, regionTestType));
@@ -181,7 +168,7 @@ public class RebalanceDUnitTest extends LuceneQueriesAccessorBase {
 
     assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
 
-    // dataStore3.invoke(() -> initDataStore(createIndex, regionType));
+    // dataStore3.invoke(() -> initDataStore(create, regionType));
     executeTextSearch(accessor, "world", "text", NUM_BUCKETS);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceWithRedundancyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceWithRedundancyDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceWithRedundancyDUnitTest.java
index 8ee14cb..7406162 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceWithRedundancyDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/RebalanceWithRedundancyDUnitTest.java
@@ -28,15 +28,10 @@ import org.apache.geode.cache.lucene.test.IndexRegionSpy;
 import org.apache.geode.cache.lucene.test.IndexRepositorySpy;
 import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InitialImageOperation;
 import org.apache.geode.internal.cache.InitialImageOperation.GIITestHook;
 import org.apache.geode.internal.cache.InitialImageOperation.GIITestHookType;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.partitioned.BecomePrimaryBucketMessage;
-import org.apache.geode.internal.cache.partitioned.BecomePrimaryBucketMessage.BecomePrimaryBucketResponse;
 import org.apache.geode.test.dunit.SerializableRunnableIF;
-import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 import org.awaitility.Awaitility;
@@ -120,7 +115,7 @@ public class RebalanceWithRedundancyDUnitTest extends LuceneQueriesAccessorBase
       RegionTestableType regionTestType) throws InterruptedException {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     accessor.invoke(() -> initAccessor(createIndex, regionTestType));
@@ -157,7 +152,7 @@ public class RebalanceWithRedundancyDUnitTest extends LuceneQueriesAccessorBase
   private void putEntriesAndValidateResultsWithRedundancy(RegionTestableType regionTestType) {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
     };
     dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
     dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
index 5c6d256..b3db8ec 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
@@ -26,7 +26,7 @@ import org.mockito.stubbing.Answer;
 
 import java.util.function.Consumer;
 
-public class LuceneIndexFactorySpy extends LuceneIndexFactory {
+public class LuceneIndexFactorySpy extends LuceneIndexImplFactory {
 
   public static LuceneIndexFactorySpy injectSpy() {
     LuceneIndexFactorySpy factory = new LuceneIndexFactorySpy();
@@ -35,7 +35,7 @@ public class LuceneIndexFactorySpy extends LuceneIndexFactory {
   }
 
   public static void remove() {
-    LuceneServiceImpl.luceneIndexFactory = new LuceneIndexFactory();
+    LuceneServiceImpl.luceneIndexFactory = new LuceneIndexImplFactory();
   }
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
index b50db98..f853baf 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
@@ -31,7 +31,6 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.io.IOException;
@@ -77,7 +76,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
   public void recoverRepoInANewNode()
       throws BucketNotFoundException, IOException, InterruptedException {
     LuceneServiceImpl service = (LuceneServiceImpl) LuceneServiceProvider.get(cache);
-    service.createIndex("index1", "/userRegion", indexedFields);
+    service.createIndexFactory().setFields(indexedFields).create("index1", "/userRegion");
     PartitionAttributes<String, String> attrs =
         new PartitionAttributesFactory().setTotalNumBuckets(1).create();
     RegionFactory<String, String> regionfactory =

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index d6ec545..35923e7 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -52,7 +52,7 @@ public class LuceneServiceImplJUnitTest {
   @Test
   public void shouldThrowIllegalArgumentExceptionIfFieldsAreMissing() {
     thrown.expect(IllegalArgumentException.class);
-    service.createIndex("index", "region");
+    service.createIndexFactory().create("index", "region");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
index efc11ab..a3e2a90 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
@@ -15,7 +15,6 @@
 package org.apache.geode.cache.lucene.internal.cli;
 
 import org.apache.geode.cache.*;
-import org.apache.geode.cache.Region.Entry;
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneQuery;
 import org.apache.geode.cache.lucene.LuceneService;
@@ -24,7 +23,6 @@ import org.apache.geode.cache.lucene.internal.LuceneIndexCreationProfile;
 import org.apache.geode.cache.lucene.internal.LuceneIndexImpl;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
 import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.commands.CliCommandTestBase;
@@ -39,33 +37,24 @@ import org.awaitility.Awaitility;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.search.Collector;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
 import static org.apache.geode.test.dunit.Assert.*;
-import static junitparams.JUnitParamsRunner.$;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
 
 import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
 
 @Category(DistributedTest.class)
 @RunWith(JUnitParamsRunner.class)
@@ -623,7 +612,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       fieldAnalyzers.put("field1", new StandardAnalyzer());
       fieldAnalyzers.put("field2", new KeywordAnalyzer());
       fieldAnalyzers.put("field3", null);
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, fieldAnalyzers);
+      luceneService.createIndexFactory().setFields(fieldAnalyzers).create(INDEX_NAME, REGION_NAME);
       createRegion();
     });
   }
@@ -635,7 +624,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       fieldAnalyzers.put("field1", new StandardAnalyzer());
       fieldAnalyzers.put("field2", new KeywordAnalyzer());
       fieldAnalyzers.put("field3", null);
-      luceneService.createIndex(INDEX_NAME, REGION_NAME, fieldAnalyzers);
+      luceneService.createIndexFactory().setFields(fieldAnalyzers).create(INDEX_NAME, REGION_NAME);
     });
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunctionJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunctionJUnitTest.java
index d974730..5f062d9 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunctionJUnitTest.java
@@ -22,7 +22,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.analysis.Analyzer;
+import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.junit.Before;
@@ -50,6 +50,7 @@ public class LuceneCreateIndexFunctionJUnitTest {
   FunctionContext context;
   ResultSender resultSender;
   CliFunctionResult expectedResult;
+  private LuceneIndexFactory factory;
 
   @Before
   public void prepare() {
@@ -58,7 +59,8 @@ public class LuceneCreateIndexFunctionJUnitTest {
     member = ds.getDistributedMember().getId();
     service = mock(InternalLuceneService.class);
     when(cache.getService(InternalLuceneService.class)).thenReturn(service);
-    doNothing().when(service).createIndex(anyString(), anyString(), anyMap());
+    factory = mock(LuceneIndexFactory.class);
+    when(service.createIndexFactory()).thenReturn(factory);
 
     context = mock(FunctionContext.class);
     resultSender = mock(ResultSender.class);
@@ -87,12 +89,11 @@ public class LuceneCreateIndexFunctionJUnitTest {
     function.execute(context);
 
     ArgumentCaptor<Map> analyzersCaptor = ArgumentCaptor.forClass(Map.class);
-    verify(service).createIndex(eq("index1"), eq("/region1"), analyzersCaptor.capture());
-    Map<String, Analyzer> analyzerPerField = analyzersCaptor.getValue();
-    assertEquals(3, analyzerPerField.size());
-    assertTrue(analyzerPerField.get("field1") instanceof StandardAnalyzer);
-    assertTrue(analyzerPerField.get("field2") instanceof KeywordAnalyzer);
-    assertTrue(analyzerPerField.get("field3") instanceof StandardAnalyzer);
+    verify(service).createIndexFactory();
+    verify(factory).addField(eq("field1"), isA(StandardAnalyzer.class));
+    verify(factory).addField(eq("field2"), isA(KeywordAnalyzer.class));
+    verify(factory).addField(eq("field3"), isA(StandardAnalyzer.class));
+    verify(factory).create(eq("index1"), eq("/region1"));
 
     ArgumentCaptor<Set> resultCaptor = ArgumentCaptor.forClass(Set.class);
     verify(resultSender).lastResult(resultCaptor.capture());
@@ -113,8 +114,10 @@ public class LuceneCreateIndexFunctionJUnitTest {
     doReturn(cache).when(function).getCache();
     function.execute(context);
 
-    verify(service).createIndex(eq("index1"), eq("/region1"), eq("field1"), eq("field2"),
-        eq("field3"));
+    verify(factory).addField(eq("field1"));
+    verify(factory).addField(eq("field2"));
+    verify(factory).addField(eq("field3"));
+    verify(factory).create(eq("index1"), eq("/region1"));
 
     ArgumentCaptor<Set> resultCaptor = ArgumentCaptor.forClass(Set.class);
     verify(resultSender).lastResult(resultCaptor.capture());

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
index eb6c882..93f1aff 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
@@ -45,7 +45,8 @@ public class DumpDirectoryFilesIntegrationTest extends LuceneIntegrationTest {
 
   @Test
   public void shouldDumpReadableLuceneIndexFile() throws Exception {
-    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
 
     Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
     region.put(0, new TestObject("title 1", "hello world"));

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/management/LuceneManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/management/LuceneManagementDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/management/LuceneManagementDUnitTest.java
index 36a8e55..79c011d 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/management/LuceneManagementDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/management/LuceneManagementDUnitTest.java
@@ -184,7 +184,8 @@ public class LuceneManagementDUnitTest extends ManagementTestBase {
   private void createIndexes(String regionName, int numIndexes) {
     LuceneService luceneService = LuceneServiceProvider.get(getCache());
     for (int i = 0; i < numIndexes; i++) {
-      luceneService.createIndex(INDEX_NAME + "_" + i, regionName, "field" + i);
+      luceneService.createIndexFactory().setFields("field" + i).create(INDEX_NAME + "_" + i,
+          regionName);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index e3e2787..9067376 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -160,7 +160,7 @@ public class IndexRepositoryImplPerformanceTest {
       public void init() throws Exception {
         cache = new CacheFactory().set(MCAST_PORT, "0").set(LOG_LEVEL, "warning").create();
         service = LuceneServiceProvider.get(cache);
-        service.createIndex("index", "/region", "text");
+        service.createIndexFactory().addField("test").create("index", "/region");
         region =
             cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
                 .setPartitionAttributes(

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
index 8df05b6..361b7a9 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexXmlGeneratorIntegrationJUnitTest.java
@@ -52,7 +52,7 @@ public class LuceneIndexXmlGeneratorIntegrationJUnitTest {
   public void generateWithFields() {
     cache = new CacheFactory().set(MCAST_PORT, "0").create();
     LuceneService service = LuceneServiceProvider.get(cache);
-    service.createIndex("index", "region", "a", "b", "c");
+    service.createIndexFactory().setFields("a", "b", "c").create("index", "region");
     cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/b74a0bd4/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
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 329dee9..e2d3abc 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
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.cache.lucene.test;
 
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
 import static org.junit.Assert.*;
 
 import java.util.ArrayList;
@@ -36,6 +35,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.asyncqueue.AsyncEventQueue;
 import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import org.apache.geode.cache.lucene.LuceneIndex;
+import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.geode.cache.lucene.LuceneQuery;
 import org.apache.geode.cache.lucene.LuceneQueryException;
 import org.apache.geode.cache.lucene.PageableLuceneQueryResults;
@@ -43,13 +43,8 @@ import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.LuceneIndexForPartitionedRegion;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
-import org.apache.geode.cache.persistence.PartitionOfflineException;
-import org.apache.geode.internal.cache.ForceReattemptException;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
-import org.apache.geode.pdx.JSONFormatter;
-import org.apache.geode.pdx.PdxInstance;
-import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.VM;
 
 public class LuceneTestUtilities {
@@ -186,7 +181,8 @@ public class LuceneTestUtilities {
   }
 
   public static void createIndex(Cache cache, String... fieldNames) {
-    LuceneServiceProvider.get(cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
+    final LuceneIndexFactory indexFactory = LuceneServiceProvider.get(cache).createIndexFactory();
+    indexFactory.setFields(fieldNames).create(INDEX_NAME, REGION_NAME);
   }
 
   public static void verifyIndexFinishFlushing(Cache cache, String indexName, String regionName)