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

[27/50] [abbrv] incubator-geode git commit: GEODE-11 Added the findKeys method to LuceneQuery

GEODE-11 Added the findKeys method to LuceneQuery

Added the findKeys method to LuceneQuery that returns the list of keys. Added
unit and intergration to test findKeys. Added an integration test for
pagination.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 14437b72e1600233173586444e8a9d4f7ac88335
Parents: d0e61ec
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Fri Jun 17 16:17:19 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:41 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |   8 ++
 .../cache/lucene/internal/LuceneQueryImpl.java  |  36 +++++-
 .../lucene/LuceneQueriesIntegrationTest.java    |  41 +++++++
 .../LuceneQueryImplIntegrationTest.java         | 120 -------------------
 .../internal/LuceneQueryImplJUnitTest.java      | 116 ++++++++++++++++++
 .../cache/lucene/test/LuceneTestUtilities.java  |   8 +-
 6 files changed, 199 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index be47677..9915116 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import java.util.Collection;
+
 import com.gemstone.gemfire.annotations.Experimental;
 
 /**
@@ -27,6 +29,11 @@ import com.gemstone.gemfire.annotations.Experimental;
 @Experimental
 public interface LuceneQuery<K, V> {
   /**
+   * Execute search and return keys.
+   */
+  public Collection<K> findKeys() throws LuceneQueryException;
+
+  /**
    * Execute the search and get results. 
    */
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException;
@@ -45,4 +52,5 @@ public interface LuceneQuery<K, V> {
    * Get projected fields setting of current query. 
    */
   public String[] getProjectedFieldNames();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index 44fa446..cd49f72 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -19,7 +19,13 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
@@ -28,6 +34,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
@@ -57,20 +64,36 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
+  public Collection<K> findKeys() throws LuceneQueryException {
+    TopEntries entries = findTopEntries();
+    final List<EntryScore> hits = entries.getHits();
+
+    return hits.stream()
+      .map(hit -> (K) hit.getKey())
+      .collect(Collectors.toList());
+  }
+
+  @Override
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException {
+    TopEntries entries = findTopEntries();
+
+    return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+  }
+
+  private TopEntries findTopEntries() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
 
-    ResultCollector<TopEntriesCollector, TopEntries> rc = (ResultCollector<TopEntriesCollector, TopEntries>) FunctionService.onRegion(region)
+    ResultCollector<TopEntriesCollector, TopEntries> rc = (ResultCollector<TopEntriesCollector, TopEntries>) onRegion()
         .withArgs(context)
         .withCollector(collector)
         .execute(LuceneFunction.ID);
-    
+
     //TODO provide a timeout to the user?
+    TopEntries entries;
     try {
-      TopEntries entries = rc.getResult();
-      return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+      entries = rc.getResult();
     } catch(FunctionException e) {
       if(e.getCause() instanceof LuceneQueryException) {
         throw new LuceneQueryException(e);
@@ -78,6 +101,11 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
         throw e;
       }
     }
+    return entries;
+  }
+
+  protected Execution onRegion() {
+    return FunctionService.onRegion(region);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index dbb92cf..cf1b5d2 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -17,8 +17,13 @@
 package com.gemstone.gemfire.cache.lucene;
 
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -110,6 +115,42 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
+  public void shouldPaginateResults() throws Exception {
+
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+    Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
+      .create(REGION_NAME);
+    final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    //Put two values with some of the same tokens
+    String value1 = "one three";
+    String value2 = "one two three";
+    String value3 = "one@three";
+    region.put("A", new TestObject(value1, value1));
+    region.put("B", new TestObject(value2, value2));
+    region.put("C", new TestObject(value3, value3));
+
+    index.waitUntilFlushed(60000);
+    final LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .setPageSize(2)
+      .create(INDEX_NAME, REGION_NAME,
+      "one", "field1");
+
+    final PageableLuceneQueryResults<Object, Object> pages = query.findPages();
+    assertTrue(pages.hasNextPage());
+    assertEquals(3, pages.size());
+    final List<LuceneResultStruct<Object, Object>> page1 = pages.getNextPage();
+    final List<LuceneResultStruct<Object, Object>> page2 = pages.getNextPage();
+    List<LuceneResultStruct<Object, Object>> allEntries=new ArrayList<>();
+    allEntries.addAll(page1);
+    allEntries.addAll(page2);
+
+    assertEquals(region.keySet(), allEntries.stream().map(entry -> entry.getKey()).collect(Collectors.toSet()));
+    assertEquals(region.values(), allEntries.stream().map(entry -> entry.getValue()).collect(Collectors.toSet()));
+
+  }
+
+  @Test()
   public void shouldTokenizeUsingMyCharacterAnalyser() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     // not to specify field1's analyzer, it should use standard analyzer

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
deleted file mode 100644
index 62f4623..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ /dev/null
@@ -1,120 +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 com.gemstone.gemfire.cache.lucene.internal;
-
-import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.FunctionAdapter;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
-import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-@Category(IntegrationTest.class)
-public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
-  private static int LIMIT = 123;
-  private Region<Object, Object> region;
-
-  @Before
-  public void createRegion() {
-    region = cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
-  }
-
-  @After
-  public void removeFunction() {
-    FunctionService.unregisterFunction(LuceneFunction.ID);
-  }
-
-  @Test
-  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
-    // Register a fake function to observe the function invocation
-    FunctionService.unregisterFunction(LuceneFunction.ID);
-    TestLuceneFunction function = new TestLuceneFunction();
-    FunctionService.registerFunction(function);
-
-    StringQueryProvider provider = new StringQueryProvider(null, DEFAULT_FIELD);
-    LuceneQueryImpl<Object, Object> query = new LuceneQueryImpl<>("index", region, provider, null, LIMIT, 20);
-    PageableLuceneQueryResults<Object, Object> results = query.findPages();
-
-    assertTrue(function.wasInvoked);
-    assertEquals(2f * LIMIT, results.getMaxScore(), 0.01);
-    int resultCount = 0;
-    while (results.hasNextPage()) {
-      List<LuceneResultStruct<Object, Object>> nextPage = results.getNextPage();
-      resultCount += nextPage.size();
-      if (results.hasNextPage()) {
-        assertEquals(20, nextPage.size());
-      }
-    }
-    assertEquals(LIMIT, resultCount);
-
-    LuceneFunctionContext<? extends IndexResultCollector> funcArgs = function.args;
-    assertEquals(provider.getQueryString(), ((StringQueryProvider) funcArgs.getQueryProvider()).getQueryString());
-    assertEquals("index", funcArgs.getIndexName());
-    assertEquals(LIMIT, funcArgs.getLimit());
-  }
-
-  private static class TestLuceneFunction extends FunctionAdapter {
-    private static final long serialVersionUID = 1L;
-    private boolean wasInvoked;
-    private LuceneFunctionContext<? extends IndexResultCollector> args;
-
-    @Override
-    public void execute(FunctionContext context) {
-      this.args = (LuceneFunctionContext<?>) context.getArguments();
-      TopEntriesCollectorManager manager = (TopEntriesCollectorManager) args.getCollectorManager();
-
-      assertEquals(LIMIT, manager.getLimit());
-
-      wasInvoked = true;
-      TopEntriesCollector lastResult = new TopEntriesCollector(null, 2 * LIMIT);
-      // put more than LIMIT entries. The resultCollector should trim the results
-      for (int i = LIMIT * 2; i >= 0; i--) {
-        lastResult.collect(i, i * 1f);
-      }
-      assertEquals(LIMIT * 2, lastResult.getEntries().getHits().size());
-
-      context.getResultSender().lastResult(lastResult);
-    }
-
-    @Override
-    public String getId() {
-      return LuceneFunction.ID;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
new file mode 100644
index 0000000..1b36e37
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.FunctionAdapter;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LuceneQueryImplJUnitTest {
+  private static int LIMIT = 123;
+  private LuceneQueryImpl<Object, Object> query;
+  private Execution execution;
+  private LuceneQueryProvider provider;
+
+  @Before
+  public void createMocks() {
+    Region region = mock(Region.class);
+    execution = mock(Execution.class);
+    ResultCollector<TopEntriesCollector, TopEntries> collector = mock(ResultCollector.class);
+    provider = mock(LuceneQueryProvider.class);
+
+    when(execution.withArgs(any())).thenReturn(execution);
+    when(execution.withCollector(any())).thenReturn(execution);
+    when(execution.execute(anyString())).thenReturn((ResultCollector) collector);
+
+    TopEntries entries = new TopEntries();
+    entries.addHit(new EntryScore("hi", 5));
+    when(collector.getResult()).thenReturn(entries);
+
+
+    query = new LuceneQueryImpl<Object, Object>("index", region, provider, null, LIMIT, 20) {
+      @Override protected Execution onRegion() {
+        return execution;
+      }
+    };
+  }
+
+  @Test
+  public void shouldReturnKeysFromFindKeys() throws LuceneQueryException {
+    Collection<Object> results = query.findKeys();
+    assertEquals(Collections.singletonList("hi"), results);
+  }
+
+  @Test
+  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
+    PageableLuceneQueryResults<Object, Object> results = query.findPages();
+
+    verify(execution).execute(eq(LuceneFunction.ID));
+    ArgumentCaptor<LuceneFunctionContext> captor = ArgumentCaptor.forClass(LuceneFunctionContext.class);
+    verify(execution).withArgs(captor.capture());
+    LuceneFunctionContext context = captor.getValue();
+    assertEquals(LIMIT, context.getLimit());
+    assertEquals(provider, context.getQueryProvider());
+    assertEquals("index", context.getIndexName());
+
+    assertEquals(5, results.getMaxScore(), 0.01);
+    final List<LuceneResultStruct<Object, Object>> page = results.getNextPage();
+    assertEquals(1, page.size());
+    LuceneResultStruct element = page.iterator().next();
+    assertEquals("hi", element.getKey());
+    assertEquals(5, element.getScore(), 0.01);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 2084e11..06c3c7d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -21,6 +21,7 @@ package com.gemstone.gemfire.cache.lucene.test;
 import static org.junit.Assert.*;
 
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -89,12 +90,7 @@ public class LuceneTestUtilities {
    */
   public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) throws LuceneQueryException {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
-    Set<K> actualKeySet = new HashSet<>();
-    final PageableLuceneQueryResults<K, Object> results = query.findPages();
-    while(results.hasNextPage()) {
-      results.getNextPage().stream()
-        .forEach(struct -> actualKeySet.add(struct.getKey()));
-    }
+    Set<K> actualKeySet = new HashSet<>(query.findKeys());
     assertEquals(expectedKeySet, actualKeySet);
   }