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 2016/07/20 17:53:54 UTC

incubator-geode git commit: GEODE-11:Gfsh command for lucene query

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 3f6acdc70 -> 603bae8cf


GEODE-11:Gfsh command for lucene query

Added a gfsh command to execute a basic search operation on the lucene index. Added junit and dunit tests to verify.

This closes #213


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

Branch: refs/heads/develop
Commit: 603bae8cfae734602a1ccc77128bdf7965e265de
Parents: 3f6acdc
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Tue Jul 19 17:39:47 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Wed Jul 20 10:53:04 2016 -0700

----------------------------------------------------------------------
 .../lucene/internal/cli/LuceneCliStrings.java   |  14 ++-
 .../internal/cli/LuceneIndexCommands.java       |  86 +++++++++++++--
 .../lucene/internal/cli/LuceneQueryInfo.java    |  55 ++++++++++
 .../internal/cli/LuceneSearchResults.java       |  50 +++++++++
 .../functions/LuceneSearchIndexFunction.java    |  90 +++++++++++++++
 .../cli/LuceneIndexCommandsDUnitTest.java       |  59 +++++++++-
 .../cli/LuceneIndexCommandsJUnitTest.java       |  31 ++++++
 .../LuceneSearchIndexFunctionJUnitTest.java     | 109 +++++++++++++++++++
 8 files changed, 481 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneCliStrings.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneCliStrings.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneCliStrings.java
index 393ab22..2f61cf9 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneCliStrings.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneCliStrings.java
@@ -57,6 +57,16 @@ public class LuceneCliStrings {
   public static final String LUCENE_DESCRIBE_INDEX__NAME__HELP = "Name of the lucene index to describe.";
   public static final String LUCENE_DESCRIBE_INDEX__REGION_HELP = "Name/Path of the region where the lucene index to be described exists.";
 
-
-
+  
+  //Search lucene index commands
+  public static final String LUCENE_SEARCH_INDEX = "lucene search";
+  public static final String LUCENE_SEARCH_INDEX__HELP = "Search lucene index";
+  public static final String LUCENE_SEARCH_INDEX__ERROR_MESSAGE = "An error occurred while searching lucene index across the Geode cluster: %1$s";
+  public static final String LUCENE_SEARCH_INDEX__NAME__HELP = "Name of the lucene index to search.";
+  public static final String LUCENE_SEARCH_INDEX__REGION_HELP = "Name/Path of the region where the lucene index exists.";
+  public static final String LUCENE_SEARCH_INDEX__QUERY_STRING="queryStrings";
+  public static final String LUCENE_SEARCH_INDEX__QUERY_STRING__HELP="Query string to search the lucene index";
+  public static final String LUCENE_SEARCH_INDEX__DEFAULT_FIELD="defaultField";
+  public static final String LUCENE_SEARCH_INDEX__DEFAULT_FIELD__HELP="Default field to search in";
+  public static final String LUCENE_SEARCH_INDEX__NO_RESULTS_MESSAGE="No results";
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommands.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommands.java
index bb2b2c9..5942d19 100755
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommands.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommands.java
@@ -16,15 +16,11 @@
  */
 package com.gemstone.gemfire.cache.lucene.internal.cli;
 
-import static com.gemstone.gemfire.cache.operations.OperationContext.*;
-
 import java.util.ArrayList;
-
-import java.util.HashMap;
+import java.util.Comparator;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.stream.Collectors;
 
 import org.apache.geode.security.GeodePermission.Operation;
@@ -38,9 +34,12 @@ import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionAdapter;
 import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneCreateIndexFunction;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneDescribeIndexFunction;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneListIndexFunction;
+import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneSearchIndexFunction;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
@@ -52,8 +51,6 @@ import com.gemstone.gemfire.management.internal.cli.commands.AbstractCommandsSup
 import com.gemstone.gemfire.management.internal.cli.functions.CliFunctionResult;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResultException;
-import com.gemstone.gemfire.management.internal.cli.result.ErrorResultData;
-import com.gemstone.gemfire.management.internal.cli.result.InfoResultData;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
@@ -70,6 +67,7 @@ import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 public class LuceneIndexCommands extends AbstractCommandsSupport {
   private static final LuceneCreateIndexFunction createIndexFunction = new LuceneCreateIndexFunction();
   private static final LuceneDescribeIndexFunction describeIndexFunction = new LuceneDescribeIndexFunction();
+  private static final LuceneSearchIndexFunction searchIndexFunction = new LuceneSearchIndexFunction();
 
   @CliCommand(value = LuceneCliStrings.LUCENE_LIST_INDEX, help = LuceneCliStrings.LUCENE_LIST_INDEX__HELP)
   @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA })
@@ -251,8 +249,80 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
     return funcResults.stream().filter(indexDetails -> indexDetails != null).collect(Collectors.toList());
   }
 
+  @CliCommand(value = LuceneCliStrings.LUCENE_SEARCH_INDEX, help = LuceneCliStrings.LUCENE_SEARCH_INDEX__HELP)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA })
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result searchIndex(
+    @CliOption(key = LuceneCliStrings.LUCENE__INDEX_NAME,
+      mandatory=true,
+      help = LuceneCliStrings.LUCENE_SEARCH_INDEX__NAME__HELP) final String indexName,
+
+    @CliOption (key = LuceneCliStrings.LUCENE__REGION_PATH,
+      mandatory = true,
+      optionContext = ConverterHint.REGIONPATH,
+      help = LuceneCliStrings.LUCENE_SEARCH_INDEX__REGION_HELP) final String regionPath,
+
+    @CliOption (key = LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING,
+      mandatory = true,
+      help = LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING__HELP) final String queryString,
+
+    @CliOption (key = LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD,
+      mandatory = true,
+      help = LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD__HELP) final String defaultField) {
+    try {
+
+      LuceneQueryInfo queryInfo=new LuceneQueryInfo(indexName,regionPath,queryString, defaultField);
+      return getSearchResults(queryInfo);
+
+    }
+    catch (FunctionInvocationTargetException ignore) {
+      return ResultBuilder.createGemFireErrorResult(CliStrings.format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN,
+        LuceneCliStrings.LUCENE_SEARCH_INDEX));
+    }
+    catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    }
+    catch (Throwable t) {
+      SystemFailure.checkFailure();
+      getCache().getLogger().error(t);
+      return ResultBuilder.createGemFireErrorResult(String.format(LuceneCliStrings.LUCENE_SEARCH_INDEX__ERROR_MESSAGE,
+        toString(t, isDebugging())));
+    }
+  }
+
+  private Result getSearchResults(final LuceneQueryInfo queryInfo) throws Exception {
+    GeodeSecurityUtil.authorizeRegionManage(queryInfo.getRegionPath());
+    final String[] groups = {};
+    final ResultCollector<?, ?> rc = this.executeFunctionOnGroups(searchIndexFunction, groups, queryInfo);
+    final List<Set<LuceneSearchResults>> functionResults = (List<Set<LuceneSearchResults>>) rc.getResult();
+
+    List<LuceneSearchResults> results = functionResults.stream()
+      .flatMap(set -> set.stream())
+      .sorted()
+      .collect(Collectors.toList());
+    if (results.size() != 0) {
+      final TabularResultData data = ResultBuilder.createTabularResultData();
+      for (LuceneSearchResults struct : results) {
+        data.accumulate("key", struct.getKey());
+        data.accumulate("value", struct.getValue());
+        data.accumulate("score", struct.getScore());
+      }
+      return ResultBuilder.buildResult(data);
+    }
+    else {
+      return ResultBuilder.createInfoResult(LuceneCliStrings.LUCENE_SEARCH_INDEX__NO_RESULTS_MESSAGE);
+    }
+    //@TODO : Pagination
+  }
+
   protected ResultCollector<?, ?> executeFunctionOnGroups(FunctionAdapter function, String[]groups, final LuceneIndexInfo indexInfo) throws CommandResultException {
     final Set<DistributedMember> targetMembers = CliUtil.findAllMatchingMembers(groups, null);
     return CliUtil.executeFunction(function, indexInfo, targetMembers);
   }
+
+  protected ResultCollector<?, ?> executeFunctionOnGroups(FunctionAdapter function, String[]groups, final LuceneQueryInfo queryInfo) throws CommandResultException {
+    final Set<DistributedMember> targetMembers = CliUtil.findAllMatchingMembers(groups, null);
+    return CliUtil.executeFunction(function, queryInfo, targetMembers);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneQueryInfo.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneQueryInfo.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneQueryInfo.java
new file mode 100644
index 0000000..2f86064
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneQueryInfo.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.cli;
+
+import java.io.Serializable;
+
+public class LuceneQueryInfo implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private String indexName;
+  private String regionPath;
+  private String queryString;
+  private String defaultField;
+
+  public LuceneQueryInfo(final String indexName,
+                         final String regionPath,
+                         final String queryString,
+                         final String defaultField)
+  {
+    this.indexName = indexName;
+    this.regionPath = regionPath;
+    this.queryString = queryString;
+    this.defaultField = defaultField;
+  }
+
+  public String getIndexName() {
+    return indexName;
+  }
+
+  public String getRegionPath() {
+    return regionPath;
+  }
+
+  public String getQueryString() {
+    return queryString;
+  }
+
+  public String getDefaultField() {
+    return defaultField;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneSearchResults.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneSearchResults.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneSearchResults.java
new file mode 100644
index 0000000..0748f0a
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneSearchResults.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.cli;
+
+import java.io.Serializable;
+
+public class LuceneSearchResults<K,V> implements Comparable<LuceneSearchResults>, Serializable {
+
+  private String key;
+  private String value;
+  private float score;
+
+  public LuceneSearchResults(final String key, final String value, final float score) {
+    this.key = key;
+    this.value = value;
+    this.score = score;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public float getScore() {
+    return score;
+  }
+
+  @Override
+  public int compareTo(final LuceneSearchResults searchResults) {
+    return getScore() < searchResults.getScore() ? -1 : 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunction.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunction.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunction.java
new file mode 100755
index 0000000..a76a741
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunction.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.cli.functions;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.execute.FunctionAdapter;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneIndexDetails;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneIndexInfo;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneQueryInfo;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneSearchResults;
+import com.gemstone.gemfire.cache.query.RegionNotFoundException;
+import com.gemstone.gemfire.internal.InternalEntity;
+
+/**
+ * The LuceneSearchIndexFunction class is a function used to collect the information on a particular lucene index.
+ * </p>
+ * @see Cache
+ * @see com.gemstone.gemfire.cache.execute.Function
+ * @see FunctionAdapter
+ * @see FunctionContext
+ * @see InternalEntity
+ * @see LuceneIndexDetails
+ * @see LuceneIndexInfo
+ */
+@SuppressWarnings("unused")
+public class LuceneSearchIndexFunction<K,V> extends FunctionAdapter implements InternalEntity {
+
+  protected Cache getCache() {
+    return CacheFactory.getAnyInstance();
+  }
+
+  public String getId() {
+    return LuceneSearchIndexFunction.class.getName();
+  }
+
+  public void execute(final FunctionContext context)   {
+
+    Set<LuceneSearchResults> result=new HashSet<>();
+    final Cache cache = getCache();
+    final LuceneQueryInfo queryInfo = (LuceneQueryInfo) context.getArguments();
+
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    try {
+      if (cache.getRegion(queryInfo.getRegionPath())!=null) {
+        final LuceneQuery<K, V> query = luceneService.createLuceneQueryFactory().create(
+          queryInfo.getIndexName(), queryInfo.getRegionPath(), queryInfo.getQueryString(), queryInfo.getDefaultField());
+        PageableLuceneQueryResults pageableLuceneQueryResults = query.findPages();
+        while (pageableLuceneQueryResults.hasNext()) {
+          List<LuceneResultStruct> page = pageableLuceneQueryResults.next();
+          page.stream()
+            .forEach(searchResult ->
+              result.add(new LuceneSearchResults<K,V>(searchResult.getKey().toString(),searchResult.getValue().toString(),searchResult.getScore())));
+        }
+      }
+      context.getResultSender().lastResult(result);
+    }
+    catch (LuceneQueryException e) {
+      context.getResultSender().lastResult(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
index 8114f93..422b340 100755
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
@@ -48,6 +48,7 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -235,7 +236,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     entries.put("A",new TestObject("field1:value1","field2:value2","field3:value3"));
     entries.put("B",new TestObject("ABC","EFG","HIJ"));
 
-    putEntries(vm1,entries);
+    putEntries(vm1,entries,2);
     queryAndVerify(vm1, "field1:value1", "field1", Collections.singletonList("A"));
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
@@ -250,7 +251,55 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(Collections.singletonList("2"), data.retrieveAllValues("Documents"));
   }
 
+  @Test
+  public void searchShouldReturnCorrectResults() throws Exception {
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    createIndex(vm1);
+    Map<String,TestObject> entries=new HashMap<>();
+    entries.put("A",new TestObject("field1:value1 ","field2:value2","field3:value3"));
+    entries.put("B",new TestObject("ABC","EFG","HIJ"));
+    entries.put("C",new TestObject("field1:value1","QWE","RTY"));
+    entries.put("D",new TestObject("ABC","EFG","HIJ"));
+    entries.put("E",new TestObject("field1 :value1","ABC","EFG"));
+    entries.put("F",new TestObject("ABC","EFG","HIJ"));
+    entries.put("G",new TestObject("field1: value1","JKR","POW"));
+    entries.put("H",new TestObject("ABC","EFG","H2J"));
+    putEntries(vm1,entries,8);
+
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_SEARCH_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME,INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH,REGION_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING,"field1:value1");
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD,"field1");
+    TabularResultData data = (TabularResultData) executeCommandAndGetResult(csb).getResultData();
+    assertEquals(4,data.retrieveAllValues("key").size());
+  }
+
+  @Test
+  public void searchShouldReturnNoResults() throws Exception {
+    final VM vm1 = Host.getHost(0).getVM(1);
 
+    createIndex(vm1);
+    Map<String,TestObject> entries=new HashMap<>();
+    entries.put("A",new TestObject("field1:value1 ","field2:value2","field3:value3"));
+    entries.put("B",new TestObject("ABC","EFG","HIJ"));
+    entries.put("C",new TestObject("field1:value1","QWE","RTY"));
+    entries.put("D",new TestObject("ABC","EFG","HIJ"));
+    entries.put("E",new TestObject("field1 :value1","ABC","EFG"));
+    entries.put("F",new TestObject("ABC","EFG","HIJ"));
+    entries.put("G",new TestObject("field1: value1","JKR","POW"));
+    entries.put("H",new TestObject("ABC","EFG","H2J"));
+    putEntries(vm1,entries,8);
+
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_SEARCH_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME,INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH,REGION_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING,"NotAnExistingValue");
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD,"field1");
+    String resultAsString = executeCommandAndLogResult(csb);
+    assertTrue(resultAsString.contains(LuceneCliStrings.LUCENE_SEARCH_INDEX__NO_RESULTS_MESSAGE));
+  }
   private void createRegion() {
     getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
   }
@@ -292,7 +341,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     System.out.println(text + ":" + resultAsString);
   }
 
-  private void putEntries(final VM vm1, Map<String,TestObject> entries) {
+  private void putEntries(final VM vm1, Map<String,TestObject> entries, int countOfDocuments) {
     Cache cache=getCache();
     vm1.invoke(()-> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());
@@ -301,7 +350,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       luceneService.getIndex(INDEX_NAME,REGION_NAME).waitUntilFlushed(60000);
       LuceneIndexImpl index=(LuceneIndexImpl) luceneService.getIndex(INDEX_NAME,REGION_NAME);
       Awaitility.await().atMost(65, TimeUnit.SECONDS).until(() ->
-        assertEquals(2,index.getIndexStats().getDocuments()));
+        assertEquals(countOfDocuments,index.getIndexStats().getDocuments()));
 
     });
   }
@@ -325,5 +374,9 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       this.field2=value2;
       this.field3=value3;
     }
+
+    public String toString() {
+      return "field1="+field1+" field2="+field2+" field3="+field3;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
index e482ee4..493e76a 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
@@ -24,12 +24,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.payloads.FloatEncoder;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.junit.Assert;
 import org.junit.Test;
@@ -42,6 +44,7 @@ import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexStats;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneCreateIndexFunction;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneDescribeIndexFunction;
 import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneListIndexFunction;
+import com.gemstone.gemfire.cache.lucene.internal.cli.functions.LuceneSearchIndexFunction;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
 import com.gemstone.gemfire.internal.util.CollectionUtils;
@@ -220,7 +223,31 @@ public class LuceneIndexCommandsJUnitTest {
     assertEquals(Arrays.asList("10"), data.retrieveAllValues("Commits"));
     assertEquals(Arrays.asList("5"), data.retrieveAllValues("Updates"));
     assertEquals(Arrays.asList("1"), data.retrieveAllValues("Documents"));
+  }
+
+  @Test
+  public void testSearchIndex() throws CommandResultException {
+
+    final Cache mockCache = mock(Cache.class, "Cache");
+    final ResultCollector mockResultCollector = mock(ResultCollector.class, "ResultCollector");
+    final LuceneIndexCommands commands=spy(createIndexCommands(mockCache,null));
 
+    final List<Set<LuceneSearchResults>> queryResultsList = new ArrayList<>();
+    HashSet<LuceneSearchResults> queryResults = new HashSet<>();
+    queryResults.add(createQueryResults("A","Result1",Float.valueOf("1.3")));
+    queryResults.add(createQueryResults("B","Result1",Float.valueOf("1.2")));
+    queryResults.add(createQueryResults("C","Result1",Float.valueOf("1.1")));
+    queryResultsList.add(queryResults);
+    doReturn(mockResultCollector).when(commands).executeFunctionOnGroups(isA(LuceneSearchIndexFunction.class),any(),any(LuceneQueryInfo.class));
+    doReturn(queryResultsList).when(mockResultCollector).getResult();
+
+    CommandResult result = (CommandResult) commands.searchIndex("index","region","Result1","field1");
+
+    TabularResultData data = (TabularResultData) result.getResultData();
+
+    assertEquals(Arrays.asList("C","B","A"), data.retrieveAllValues("key"));
+    assertEquals(Arrays.asList("Result1","Result1","Result1"), data.retrieveAllValues("value"));
+    assertEquals(Arrays.asList("1.1","1.2","1.3"), data.retrieveAllValues("score"));
   }
 
 
@@ -249,6 +276,10 @@ public class LuceneIndexCommandsJUnitTest {
     return new LuceneIndexDetails(indexName, regionPath, searchableFields, fieldAnalyzers,null);
   }
 
+  private LuceneSearchResults createQueryResults(final String key, final String value, final float score) {
+    return new LuceneSearchResults(key,value,score);
+  }
+
   private static class LuceneTestIndexCommands extends LuceneIndexCommands {
 
     private final Cache cache;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/603bae8c/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunctionJUnitTest.java
new file mode 100644
index 0000000..88f608c
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/cli/functions/LuceneSearchIndexFunctionJUnitTest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.cli.functions;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneService;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneResultStructImpl;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneQueryInfo;
+import com.gemstone.gemfire.cache.lucene.internal.cli.LuceneSearchResults;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.fake.Fakes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+@Category(UnitTest.class)
+
+public class LuceneSearchIndexFunctionJUnitTest {
+  
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testExecute() throws LuceneQueryException {
+    FunctionContext context = mock(FunctionContext.class);
+    ResultSender resultSender = mock(ResultSender.class);
+    GemFireCacheImpl cache = Fakes.cache();
+
+    LuceneQueryInfo queryInfo = createMockQueryInfo("index","region","field1:region1","field1");
+    InternalLuceneService service = getMockLuceneService("A","Value","1.2");
+    Region mockRegion=mock(Region.class);
+
+    LuceneSearchIndexFunction function = spy(LuceneSearchIndexFunction.class);
+
+    doReturn(queryInfo).when(context).getArguments();
+    doReturn(resultSender).when(context).getResultSender();
+    doReturn(cache).when(function).getCache();
+
+    when(cache.getService(eq(InternalLuceneService.class))).thenReturn(service);
+    when(cache.getRegion(queryInfo.getRegionPath())).thenReturn(mockRegion);
+
+    function.execute(context);
+    ArgumentCaptor<Set> resultCaptor  = ArgumentCaptor.forClass(Set.class);
+    verify(resultSender).lastResult(resultCaptor.capture());
+    Set<LuceneSearchResults> result = resultCaptor.getValue();
+
+    assertEquals(1,result.size());
+    for (LuceneSearchResults searchResult: result) {
+      assertEquals("A",searchResult.getKey());
+      assertEquals("Value",searchResult.getValue());
+      assertEquals(1.2,searchResult.getScore(),.1);
+    }
+  }
+  private InternalLuceneService getMockLuceneService(String resultKey, String resultValue, String resultScore) throws LuceneQueryException{
+    InternalLuceneService service=mock(InternalLuceneService.class);
+    LuceneQueryFactory mockQueryFactory = mock(LuceneQueryFactory.class);
+    LuceneQuery mockQuery=mock(LuceneQuery.class);
+    PageableLuceneQueryResults pageableLuceneQueryResults = mock(PageableLuceneQueryResults.class);
+    LuceneResultStruct<String,String> resultStruct = new LuceneResultStructImpl(resultKey,resultValue,Float.valueOf(resultScore));
+    List<LuceneResultStruct<String,String>> queryResults= new ArrayList<>();
+    queryResults.add(resultStruct);
+
+    doReturn(mockQueryFactory).when(service).createLuceneQueryFactory();
+    doReturn(mockQuery).when(mockQueryFactory).create(any(),any(),any(),any());
+    when(mockQuery.findPages()).thenReturn(pageableLuceneQueryResults);
+    when(pageableLuceneQueryResults.hasNext()).thenReturn(true).thenReturn(false);
+    when(pageableLuceneQueryResults.next()).thenReturn(queryResults);
+
+    return service;
+  }
+
+  private LuceneQueryInfo createMockQueryInfo(final String index, final String region, final String query, final String field) {
+    LuceneQueryInfo queryInfo = mock(LuceneQueryInfo.class);
+    when(queryInfo.getIndexName()).thenReturn(index);
+    when(queryInfo.getRegionPath()).thenReturn(region);
+    when(queryInfo.getQueryString()).thenReturn(query);
+    when(queryInfo.getDefaultField()).thenReturn(field);
+    return queryInfo;
+  }
+
+}