Repository: incubator-geode Updated Branches: refs/heads/feature/GEODE-11 026271b0c -> 38ff05e58
GEODE-11: Rename lucene function classes * Instead of using LuceneQuery or LuceneSearch as prefix, just use Lucene as a prefix. * LuceneSearchFunctionArgs would be used by search function executor and function result collector to setup a common context. Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/7160e88c Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/7160e88c Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/7160e88c Branch: refs/heads/feature/GEODE-11 Commit: 7160e88c1b6b57f6422e6fbd13d1c1f798546e61 Parents: 026271b Author: Ashvin Agrawal <[email protected]> Authored: Fri Sep 11 16:04:57 2015 -0700 Committer: Ashvin Agrawal <[email protected]> Committed: Fri Sep 11 16:15:43 2015 -0700 ---------------------------------------------------------------------- .../lucene/internal/LuceneServiceImpl.java | 4 +- .../internal/distributed/LuceneFunction.java | 90 +++++ .../distributed/LuceneFunctionContext.java | 58 +++ .../distributed/LuceneQueryFunction.java | 90 ----- .../distributed/LuceneSearchFunctionArgs.java | 53 --- .../TopEntriesFunctionCollector.java | 19 +- .../internal/LuceneServiceImplJUnitTest.java | 6 +- .../distributed/LuceneFunctionJUnitTest.java | 359 +++++++++++++++++++ .../LuceneQueryFunctionJUnitTest.java | 359 ------------------- .../TopEntriesFunctionCollectorJUnitTest.java | 11 +- 10 files changed, 530 insertions(+), 519 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java index bcc2c07..47e6424 100644 --- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java +++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java @@ -11,7 +11,7 @@ import com.gemstone.gemfire.cache.execute.FunctionService; import com.gemstone.gemfire.cache.lucene.LuceneIndex; import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; import com.gemstone.gemfire.cache.lucene.LuceneService; -import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneQueryFunction; +import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction; import com.gemstone.gemfire.internal.cache.GemFireCacheImpl; import com.gemstone.gemfire.internal.cache.extension.Extensible; import com.gemstone.gemfire.internal.cache.extension.Extension; @@ -39,7 +39,7 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> { this.cache = gfc; - FunctionService.registerFunction(new LuceneQueryFunction()); + FunctionService.registerFunction(new LuceneFunction()); // Initialize the Map which maintains indexes this.indexMap = new HashMap<String, LuceneIndex>(); http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java new file mode 100644 index 0000000..25b7454 --- /dev/null +++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java @@ -0,0 +1,90 @@ +package com.gemstone.gemfire.cache.lucene.internal.distributed; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.logging.log4j.Logger; + +import com.gemstone.gemfire.cache.Region; +import com.gemstone.gemfire.cache.execute.FunctionAdapter; +import com.gemstone.gemfire.cache.execute.FunctionContext; +import com.gemstone.gemfire.cache.execute.RegionFunctionContext; +import com.gemstone.gemfire.cache.execute.ResultSender; +import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; +import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager; +import com.gemstone.gemfire.internal.cache.BucketNotFoundException; +import com.gemstone.gemfire.internal.logging.LogService; + +/** + * {@link LuceneFunction} coordinates text search on a member. It receives text search query from the coordinator + * and arguments like region and buckets. It invokes search on the local index and provides a result collector. The + * locally collected results are sent to the search coordinator. + */ +public class LuceneFunction extends FunctionAdapter { + private static final long serialVersionUID = 1L; + public static final String ID = LuceneFunction.class.getName(); + + private static final Logger logger = LogService.getLogger(); + + private RepositoryManager repoManager; + + @Override + public void execute(FunctionContext context) { + RegionFunctionContext ctx = (RegionFunctionContext) context; + ResultSender<TopEntriesCollector> resultSender = ctx.getResultSender(); + + Region region = ctx.getDataSet(); + if (logger.isDebugEnabled()) { + logger.debug("Executing lucene query on region:" + region.getFullPath()); + } + + LuceneFunctionContext searchContext = (LuceneFunctionContext) ctx.getArguments(); + + CollectorManager manager = (searchContext == null) ? null : searchContext.getCollectorManager(); + if (manager == null) { + int resultLimit = (searchContext == null ? LuceneQueryFactory.DEFAULT_LIMIT : searchContext.getLimit()); + manager = new TopEntriesCollectorManager(null, resultLimit); + } + + Collection<IndexResultCollector> results = new ArrayList<>(); + try { + Collection<IndexRepository> repositories = repoManager.getRepositories(region, ctx); + for (IndexRepository repo : repositories) { + IndexResultCollector collector = manager.newCollector(repo.toString()); + logger.debug("Executing search on repo: " + repo.toString()); + repo.query(null, 0, collector); + results.add(collector); + } + } catch (IOException e) { + logger.warn("", e); + resultSender.sendException(e); + return; + } catch (BucketNotFoundException e) { + logger.warn("", e); + resultSender.sendException(e); + return; + } + + TopEntriesCollector mergedResult; + try { + mergedResult = (TopEntriesCollector) manager.reduce(results); + resultSender.lastResult(mergedResult); + } catch (IOException e) { + logger.warn("", e); + resultSender.sendException(e); + return; + } + } + + void setRepositoryManager(RepositoryManager manager) { + this.repoManager = manager; + } + + @Override + public String getId() { + return ID; + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContext.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContext.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContext.java new file mode 100644 index 0000000..170bb65 --- /dev/null +++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContext.java @@ -0,0 +1,58 @@ +package com.gemstone.gemfire.cache.lucene.internal.distributed; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; +import com.gemstone.gemfire.internal.Version; +import com.gemstone.gemfire.internal.VersionedDataSerializable; + +/** + * Contains function arguments for text / lucene search + */ +public class LuceneFunctionContext<C extends IndexResultCollector> implements VersionedDataSerializable { + private static final long serialVersionUID = 1L; + private final CollectorManager<C> manager; + + public LuceneFunctionContext(CollectorManager<C> manager) { + this.manager = manager; + } + + @Override + public void toData(DataOutput out) throws IOException { + // TODO Auto-generated method stub + + } + + @Override + public void fromData(DataInput in) throws IOException, ClassNotFoundException { + // TODO Auto-generated method stub + + } + + @Override + public Version[] getSerializationVersions() { + // TODO Auto-generated method stub + return null; + } + + /** + * @return The maximum count of result objects to be produced by the function + */ + public int getLimit() { + return LuceneQueryFactory.DEFAULT_LIMIT; + } + + /** + * On each member, search query is executed on one or more {@link IndexRepository}s. A {@link CollectorManager} could + * be provided to customize the way results from these repositories is collected and merged. + * + * @return {@link CollectorManager} instance to be used by function + */ + public CollectorManager<C> getCollectorManager() { + return this.manager; + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunction.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunction.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunction.java deleted file mode 100644 index 6e1d217..0000000 --- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunction.java +++ /dev/null @@ -1,90 +0,0 @@ -package com.gemstone.gemfire.cache.lucene.internal.distributed; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; - -import org.apache.logging.log4j.Logger; - -import com.gemstone.gemfire.cache.Region; -import com.gemstone.gemfire.cache.execute.FunctionAdapter; -import com.gemstone.gemfire.cache.execute.FunctionContext; -import com.gemstone.gemfire.cache.execute.RegionFunctionContext; -import com.gemstone.gemfire.cache.execute.ResultSender; -import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; -import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager; -import com.gemstone.gemfire.internal.cache.BucketNotFoundException; -import com.gemstone.gemfire.internal.logging.LogService; - -/** - * {@link LuceneQueryFunction} coordinates text search on a member. It receives text search query from the coordinator - * and arguments like region and buckets. It invokes search on the local index and provides a result collector. The - * locally collected results are sent to the search coordinator. - */ -public class LuceneQueryFunction extends FunctionAdapter { - private static final long serialVersionUID = 1L; - public static final String ID = LuceneQueryFunction.class.getName(); - - private static final Logger logger = LogService.getLogger(); - - private RepositoryManager repoManager; - - @Override - public void execute(FunctionContext context) { - RegionFunctionContext ctx = (RegionFunctionContext) context; - ResultSender<TopEntriesCollector> resultSender = ctx.getResultSender(); - - Region region = ctx.getDataSet(); - if (logger.isDebugEnabled()) { - logger.debug("Executing lucene query on region:" + region.getFullPath()); - } - - LuceneSearchFunctionArgs args = (LuceneSearchFunctionArgs) ctx.getArguments(); - - CollectorManager manager = (args == null) ? null : args.getCollectorManager(); - if (manager == null) { - int resultLimit = (args == null ? LuceneQueryFactory.DEFAULT_LIMIT : args.getLimit()); - manager = new TopEntriesCollectorManager(null, resultLimit); - } - - Collection<IndexResultCollector> results = new ArrayList<>(); - try { - Collection<IndexRepository> repositories = repoManager.getRepositories(region, ctx); - for (IndexRepository repo : repositories) { - IndexResultCollector collector = manager.newCollector(repo.toString()); - logger.debug("Executing search on repo: " + repo.toString()); - repo.query(null, 0, collector); - results.add(collector); - } - } catch (IOException e) { - logger.warn("", e); - resultSender.sendException(e); - return; - } catch (BucketNotFoundException e) { - logger.warn("", e); - resultSender.sendException(e); - return; - } - - TopEntriesCollector mergedResult; - try { - mergedResult = (TopEntriesCollector) manager.reduce(results); - resultSender.lastResult(mergedResult); - } catch (IOException e) { - logger.warn("", e); - resultSender.sendException(e); - return; - } - } - - void setRepositoryManager(RepositoryManager manager) { - this.repoManager = manager; - } - - @Override - public String getId() { - return ID; - } -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneSearchFunctionArgs.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneSearchFunctionArgs.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneSearchFunctionArgs.java deleted file mode 100644 index 52f3ce9..0000000 --- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneSearchFunctionArgs.java +++ /dev/null @@ -1,53 +0,0 @@ -package com.gemstone.gemfire.cache.lucene.internal.distributed; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; -import com.gemstone.gemfire.internal.Version; -import com.gemstone.gemfire.internal.VersionedDataSerializable; - -/** - * Contains function arguments for text / lucene search - */ -public class LuceneSearchFunctionArgs implements VersionedDataSerializable { - private static final long serialVersionUID = 1L; - - @Override - public void toData(DataOutput out) throws IOException { - // TODO Auto-generated method stub - - } - - @Override - public void fromData(DataInput in) throws IOException, ClassNotFoundException { - // TODO Auto-generated method stub - - } - - @Override - public Version[] getSerializationVersions() { - // TODO Auto-generated method stub - return null; - } - - /** - * @return The maximum count of result objects to be produced by the function - */ - public int getLimit() { - return LuceneQueryFactory.DEFAULT_LIMIT; - } - - /** - * On each member, search query is executed on one or more {@link IndexRepository}s. A {@link CollectorManager} could - * be provided to customize the way results from these repositories is collected and merged. - * - * @return {@link CollectorManager} instance to be used by function - */ - public <C extends IndexResultCollector> CollectorManager<C> getCollectorManager() { - return null; - } -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java index 3b39538..0bb36cf 100644 --- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java +++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java @@ -45,14 +45,19 @@ public class TopEntriesFunctionCollector implements ResultCollector<TopEntriesCo this(null, null); } - public TopEntriesFunctionCollector(CollectorManager<TopEntriesCollector> manager) { - this(manager, null); + public TopEntriesFunctionCollector(LuceneFunctionContext context) { + this(context, null); } - public TopEntriesFunctionCollector(CollectorManager<TopEntriesCollector> manager, GemFireCacheImpl cache) { + public TopEntriesFunctionCollector(LuceneFunctionContext context, GemFireCacheImpl cache) { this.cache = cache; id = cache == null ? String.valueOf(this.hashCode()) : cache.getName(); - this.manager = manager == null ? new TopEntriesCollectorManager(id) : manager; + + if (context != null && context.getCollectorManager() != null) { + this.manager = context.getCollectorManager(); + } else { + this.manager = new TopEntriesCollectorManager(id); + } } @Override @@ -86,7 +91,7 @@ public class TopEntriesFunctionCollector implements ResultCollector<TopEntriesCo } throw new FunctionException(e); } - + return aggregateResults(); } @@ -101,7 +106,7 @@ public class TopEntriesFunctionCollector implements ResultCollector<TopEntriesCo } } } - + @Override public void endResults() { synchronized (subResults) { @@ -115,7 +120,7 @@ public class TopEntriesFunctionCollector implements ResultCollector<TopEntriesCo if (waitForResults.getCount() == 0) { throw new IllegalStateException("This collector is closed and cannot accept anymore results"); } - + subResults.clear(); } } http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java index 6ee13a7..14929b5 100644 --- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java +++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java @@ -11,7 +11,7 @@ import com.gemstone.gemfire.cache.Cache; import com.gemstone.gemfire.cache.CacheFactory; import com.gemstone.gemfire.cache.execute.Function; import com.gemstone.gemfire.cache.execute.FunctionService; -import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneQueryFunction; +import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction; import com.gemstone.gemfire.internal.cache.GemFireCacheImpl; import com.gemstone.gemfire.test.junit.categories.UnitTest; @@ -22,13 +22,13 @@ public class LuceneServiceImplJUnitTest { // lucene service will register query execution function on initialization @Test public void shouldRegisterQueryFunction() { - Function function = FunctionService.getFunction(LuceneQueryFunction.ID); + Function function = FunctionService.getFunction(LuceneFunction.ID); assertNull(function); cache = createBasicCache(); new LuceneServiceImpl(cache); - function = FunctionService.getFunction(LuceneQueryFunction.ID); + function = FunctionService.getFunction(LuceneFunction.ID); assertNotNull(function); } http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java new file mode 100644 index 0000000..ec7cd3d --- /dev/null +++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java @@ -0,0 +1,359 @@ +package com.gemstone.gemfire.cache.lucene.internal.distributed; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.lucene.search.Query; +import org.jmock.Expectations; +import org.jmock.Mockery; +import org.jmock.api.Invocation; +import org.jmock.lib.action.CustomAction; +import org.jmock.lib.concurrent.Synchroniser; +import org.jmock.lib.legacy.ClassImposteriser; +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.execute.RegionFunctionContext; +import com.gemstone.gemfire.cache.execute.ResultSender; +import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; +import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; +import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager; +import com.gemstone.gemfire.internal.cache.BucketNotFoundException; +import com.gemstone.gemfire.test.junit.categories.UnitTest; + +@Category(UnitTest.class) +public class LuceneFunctionJUnitTest { + Mockery mocker; + + final EntryScore r1_1 = new EntryScore("key-1-1", .5f); + final EntryScore r1_2 = new EntryScore("key-1-2", .4f); + final EntryScore r1_3 = new EntryScore("key-1-3", .3f); + final EntryScore r2_1 = new EntryScore("key-2-1", .45f); + final EntryScore r2_2 = new EntryScore("key-2-2", .35f); + + @Test + public void testRepoQueryAndMerge() throws Exception { + final AtomicReference<TopEntriesCollector> result = new AtomicReference<>(); + + final QueryMocks m = new QueryMocks(); + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getArguments(); + will(returnValue(null)); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + will(returnValue(m.repos)); + + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + + oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(new CustomAction("streamSearchResults") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); + collector.collect(r1_1.key, r1_1.score); + collector.collect(r1_2.key, r1_2.score); + collector.collect(r1_3.key, r1_3.score); + return null; + } + }); + + oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(new CustomAction("streamSearchResults") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); + collector.collect(r2_1.key, r2_1.score); + collector.collect(r2_2.key, r2_2.score); + return null; + } + }); + + oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); + will(new CustomAction("collectResult") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + result.set((TopEntriesCollector) invocation.getParameter(0)); + return null; + } + }); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + List<EntryScore> hits = result.get().getEntries().getHits(); + assertEquals(5, hits.size()); + TopEntriesJUnitTest.verifyResultOrder(result.get().getEntries().getHits(), r1_1, r2_1, r1_2, r2_2, r1_3); + } + + @Test + public void testResultLimitClause() throws Exception { + final AtomicReference<TopEntriesCollector> result = new AtomicReference<>(); + + final QueryMocks m = new QueryMocks(); + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getArguments(); + will(returnValue(m.mockSearchContext)); + + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + + oneOf(m.mockSearchContext).getLimit(); + will(returnValue(3)); + oneOf(m.mockSearchContext).getCollectorManager(); + will(returnValue(null)); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + will(returnValue(m.repos)); + + oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(new CustomAction("streamSearchResults") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); + collector.collect(r1_1.key, r1_1.score); + collector.collect(r1_2.key, r1_2.score); + collector.collect(r1_3.key, r1_3.score); + return null; + } + }); + + oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(new CustomAction("streamSearchResults") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); + collector.collect(r2_1.key, r2_1.score); + collector.collect(r2_2.key, r2_2.score); + return null; + } + }); + + oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); + will(new CustomAction("collectResult") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + result.set((TopEntriesCollector) invocation.getParameter(0)); + return null; + } + }); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + List<EntryScore> hits = result.get().getEntries().getHits(); + assertEquals(3, hits.size()); + TopEntriesJUnitTest.verifyResultOrder(result.get().getEntries().getHits(), r1_1, r2_1, r1_2); + } + + @Test + public void injectCustomCollectorManager() throws Exception { + final QueryMocks m = new QueryMocks(); + + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getArguments(); + will(returnValue(m.mockSearchContext)); + + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + + oneOf(m.mockSearchContext).getCollectorManager(); + will(returnValue(m.mockManager)); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + m.repos.remove(0); + will(returnValue(m.repos)); + + oneOf(m.mockManager).newCollector("repo2"); + will(returnValue(m.mockCollector)); + oneOf(m.mockManager).reduce(with(any(Collection.class))); + will(new CustomAction("reduce") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + Collection<IndexResultCollector> collectors = (Collection<IndexResultCollector>) invocation.getParameter(0); + assertEquals(1, collectors.size()); + assertEquals(m.mockCollector, collectors.iterator().next()); + return new TopEntriesCollector(null); + } + }); + + oneOf(m.mockCollector).collect("key-2-1", .45f); + + oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(new CustomAction("streamSearchResults") { + @Override + public Object invoke(Invocation invocation) throws Throwable { + IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); + collector.collect(r2_1.key, r2_1.score); + return null; + } + }); + + oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + } + + @Test + public void testIndexRepoQueryFails() throws Exception { + final QueryMocks m = new QueryMocks(); + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getArguments(); + will(returnValue(null)); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + will(returnValue(m.repos)); + + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + oneOf(m.mockResultSender).sendException(with(any(IOException.class))); + + oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); + will(throwException(new IOException())); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + } + + @Test + public void testBucketNotFound() throws Exception { + final QueryMocks m = new QueryMocks(); + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getArguments(); + will(returnValue(null)); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + will(throwException(new BucketNotFoundException(""))); + + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + oneOf(m.mockResultSender).sendException(with(any(BucketNotFoundException.class))); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + } + + @Test + public void testReduceError() throws Exception { + final QueryMocks m = new QueryMocks(); + mocker.checking(new Expectations() { + { + oneOf(m.mockContext).getDataSet(); + will(returnValue(m.mockRegion)); + oneOf(m.mockContext).getResultSender(); + will(returnValue(m.mockResultSender)); + + oneOf(m.mockContext).getArguments(); + will(returnValue(m.mockSearchContext)); + oneOf(m.mockSearchContext).getCollectorManager(); + will(returnValue(m.mockManager)); + oneOf(m.mockManager).newCollector("repo1"); + will(returnValue(m.mockCollector)); + oneOf(m.mockManager).reduce(with(any(Collection.class))); + will(throwException(new IOException())); + + oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); + m.repos.remove(1); + will(returnValue(m.repos)); + + oneOf(m.mockRepository1).query(null, 0, m.mockCollector); + oneOf(m.mockResultSender).sendException(with(any(IOException.class))); + } + }); + + LuceneFunction function = new LuceneFunction(); + function.setRepositoryManager(m.mockRepoManager); + + function.execute(m.mockContext); + } + + @Test + public void testQueryFunctionId() { + String id = new LuceneFunction().getId(); + assertEquals(LuceneFunction.class.getName(), id); + } + + class QueryMocks { + RegionFunctionContext mockContext = mocker.mock(RegionFunctionContext.class); + ResultSender<TopEntriesCollector> mockResultSender = mocker.mock(ResultSender.class); + Region<Object, Object> mockRegion = mocker.mock(Region.class); + + RepositoryManager mockRepoManager = mocker.mock(RepositoryManager.class); + ArrayList<IndexRepository> repos = new ArrayList<IndexRepository>(); + IndexRepository mockRepository1 = mocker.mock(IndexRepository.class, "repo1"); + IndexRepository mockRepository2 = mocker.mock(IndexRepository.class, "repo2"); + LuceneFunctionContext mockSearchContext = mocker.mock(LuceneFunctionContext.class); + CollectorManager mockManager = mocker.mock(CollectorManager.class); + IndexResultCollector mockCollector = mocker.mock(IndexResultCollector.class); + + QueryMocks() { + repos.add(mockRepository1); + repos.add(mockRepository2); + } + } + + @Test + public void testLuceneFunctionArgsDefaults() { + LuceneFunctionContext context = new LuceneFunctionContext(null); + assertEquals(LuceneQueryFactory.DEFAULT_LIMIT, context.getLimit()); + } + + @Before + public void setupMock() { + mocker = new Mockery() { + { + setImposteriser(ClassImposteriser.INSTANCE); + setThreadingPolicy(new Synchroniser()); + } + }; + } + + @After + public void validateMock() { + mocker.assertIsSatisfied(); + mocker = null; + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java deleted file mode 100644 index ee2847e..0000000 --- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java +++ /dev/null @@ -1,359 +0,0 @@ -package com.gemstone.gemfire.cache.lucene.internal.distributed; - -import static org.junit.Assert.assertEquals; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.lucene.search.Query; -import org.jmock.Expectations; -import org.jmock.Mockery; -import org.jmock.api.Invocation; -import org.jmock.lib.action.CustomAction; -import org.jmock.lib.concurrent.Synchroniser; -import org.jmock.lib.legacy.ClassImposteriser; -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.execute.RegionFunctionContext; -import com.gemstone.gemfire.cache.execute.ResultSender; -import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository; -import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector; -import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager; -import com.gemstone.gemfire.internal.cache.BucketNotFoundException; -import com.gemstone.gemfire.test.junit.categories.UnitTest; - -@Category(UnitTest.class) -public class LuceneQueryFunctionJUnitTest { - Mockery mocker; - - final EntryScore r1_1 = new EntryScore("key-1-1", .5f); - final EntryScore r1_2 = new EntryScore("key-1-2", .4f); - final EntryScore r1_3 = new EntryScore("key-1-3", .3f); - final EntryScore r2_1 = new EntryScore("key-2-1", .45f); - final EntryScore r2_2 = new EntryScore("key-2-2", .35f); - - @Test - public void testRepoQueryAndMerge() throws Exception { - final AtomicReference<TopEntriesCollector> result = new AtomicReference<>(); - - final QueryMocks m = new QueryMocks(); - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getArguments(); - will(returnValue(null)); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - will(returnValue(m.repos)); - - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - - oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(new CustomAction("streamSearchResults") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); - collector.collect(r1_1.key, r1_1.score); - collector.collect(r1_2.key, r1_2.score); - collector.collect(r1_3.key, r1_3.score); - return null; - } - }); - - oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(new CustomAction("streamSearchResults") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); - collector.collect(r2_1.key, r2_1.score); - collector.collect(r2_2.key, r2_2.score); - return null; - } - }); - - oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); - will(new CustomAction("collectResult") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - result.set((TopEntriesCollector) invocation.getParameter(0)); - return null; - } - }); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - List<EntryScore> hits = result.get().getEntries().getHits(); - assertEquals(5, hits.size()); - TopEntriesJUnitTest.verifyResultOrder(result.get().getEntries().getHits(), r1_1, r2_1, r1_2, r2_2, r1_3); - } - - @Test - public void testResultLimitClause() throws Exception { - final AtomicReference<TopEntriesCollector> result = new AtomicReference<>(); - - final QueryMocks m = new QueryMocks(); - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getArguments(); - will(returnValue(m.mockFuncArgs)); - - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - - oneOf(m.mockFuncArgs).getLimit(); - will(returnValue(3)); - oneOf(m.mockFuncArgs).getCollectorManager(); - will(returnValue(null)); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - will(returnValue(m.repos)); - - oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(new CustomAction("streamSearchResults") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); - collector.collect(r1_1.key, r1_1.score); - collector.collect(r1_2.key, r1_2.score); - collector.collect(r1_3.key, r1_3.score); - return null; - } - }); - - oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(new CustomAction("streamSearchResults") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); - collector.collect(r2_1.key, r2_1.score); - collector.collect(r2_2.key, r2_2.score); - return null; - } - }); - - oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); - will(new CustomAction("collectResult") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - result.set((TopEntriesCollector) invocation.getParameter(0)); - return null; - } - }); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - List<EntryScore> hits = result.get().getEntries().getHits(); - assertEquals(3, hits.size()); - TopEntriesJUnitTest.verifyResultOrder(result.get().getEntries().getHits(), r1_1, r2_1, r1_2); - } - - @Test - public void injectCustomCollectorManager() throws Exception { - final QueryMocks m = new QueryMocks(); - - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getArguments(); - will(returnValue(m.mockFuncArgs)); - - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - - oneOf(m.mockFuncArgs).getCollectorManager(); - will(returnValue(m.mockManager)); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - m.repos.remove(0); - will(returnValue(m.repos)); - - oneOf(m.mockManager).newCollector("repo2"); - will(returnValue(m.mockCollector)); - oneOf(m.mockManager).reduce(with(any(Collection.class))); - will(new CustomAction("reduce") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - Collection<IndexResultCollector> collectors = (Collection<IndexResultCollector>) invocation.getParameter(0); - assertEquals(1, collectors.size()); - assertEquals(m.mockCollector, collectors.iterator().next()); - return new TopEntriesCollector(null); - } - }); - - oneOf(m.mockCollector).collect("key-2-1", .45f); - - oneOf(m.mockRepository2).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(new CustomAction("streamSearchResults") { - @Override - public Object invoke(Invocation invocation) throws Throwable { - IndexResultCollector collector = (IndexResultCollector) invocation.getParameter(2); - collector.collect(r2_1.key, r2_1.score); - return null; - } - }); - - oneOf(m.mockResultSender).lastResult(with(any(TopEntriesCollector.class))); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - } - - @Test - public void testIndexRepoQueryFails() throws Exception { - final QueryMocks m = new QueryMocks(); - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getArguments(); - will(returnValue(null)); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - will(returnValue(m.repos)); - - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - oneOf(m.mockResultSender).sendException(with(any(IOException.class))); - - oneOf(m.mockRepository1).query(with(aNull(Query.class)), with(equal(0)), with(any(IndexResultCollector.class))); - will(throwException(new IOException())); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - } - - @Test - public void testBucketNotFound() throws Exception { - final QueryMocks m = new QueryMocks(); - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getArguments(); - will(returnValue(null)); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - will(throwException(new BucketNotFoundException(""))); - - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - oneOf(m.mockResultSender).sendException(with(any(BucketNotFoundException.class))); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - } - - @Test - public void testReduceError() throws Exception { - final QueryMocks m = new QueryMocks(); - mocker.checking(new Expectations() { - { - oneOf(m.mockContext).getDataSet(); - will(returnValue(m.mockRegion)); - oneOf(m.mockContext).getResultSender(); - will(returnValue(m.mockResultSender)); - - oneOf(m.mockContext).getArguments(); - will(returnValue(m.mockFuncArgs)); - oneOf(m.mockFuncArgs).getCollectorManager(); - will(returnValue(m.mockManager)); - oneOf(m.mockManager).newCollector("repo1"); - will(returnValue(m.mockCollector)); - oneOf(m.mockManager).reduce(with(any(Collection.class))); - will(throwException(new IOException())); - - oneOf(m.mockRepoManager).getRepositories(m.mockRegion, m.mockContext); - m.repos.remove(1); - will(returnValue(m.repos)); - - oneOf(m.mockRepository1).query(null, 0, m.mockCollector); - oneOf(m.mockResultSender).sendException(with(any(IOException.class))); - } - }); - - LuceneQueryFunction function = new LuceneQueryFunction(); - function.setRepositoryManager(m.mockRepoManager); - - function.execute(m.mockContext); - } - - @Test - public void testQueryFunctionId() { - String id = new LuceneQueryFunction().getId(); - assertEquals(LuceneQueryFunction.class.getName(), id); - } - - class QueryMocks { - RegionFunctionContext mockContext = mocker.mock(RegionFunctionContext.class); - ResultSender<TopEntriesCollector> mockResultSender = mocker.mock(ResultSender.class); - Region<Object, Object> mockRegion = mocker.mock(Region.class); - - RepositoryManager mockRepoManager = mocker.mock(RepositoryManager.class); - ArrayList<IndexRepository> repos = new ArrayList<IndexRepository>(); - IndexRepository mockRepository1 = mocker.mock(IndexRepository.class, "repo1"); - IndexRepository mockRepository2 = mocker.mock(IndexRepository.class, "repo2"); - LuceneSearchFunctionArgs mockFuncArgs = mocker.mock(LuceneSearchFunctionArgs.class); - CollectorManager mockManager = mocker.mock(CollectorManager.class); - IndexResultCollector mockCollector = mocker.mock(IndexResultCollector.class); - - QueryMocks() { - repos.add(mockRepository1); - repos.add(mockRepository2); - } - } - - @Test - public void testLuceneFunctionArgsDefaults() { - LuceneSearchFunctionArgs args = new LuceneSearchFunctionArgs(); - assertEquals(LuceneQueryFactory.DEFAULT_LIMIT, args.getLimit()); - } - - @Before - public void setupMock() { - mocker = new Mockery() { - { - setImposteriser(ClassImposteriser.INSTANCE); - setThreadingPolicy(new Synchroniser()); - } - }; - } - - @After - public void validateMock() { - mocker.assertIsSatisfied(); - mocker = null; - } -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7160e88c/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java index 1c42912..17cee85 100644 --- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java +++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java @@ -184,7 +184,7 @@ public class TopEntriesFunctionCollectorJUnitTest { @Test public void mergeResultsDefaultCollectorManager() throws Exception { - TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(null); + TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(); collector.addResult(null, result1); collector.addResult(null, result2); collector.endResults(); @@ -211,7 +211,8 @@ public class TopEntriesFunctionCollectorJUnitTest { } })); - TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(mockManager); + LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(mockManager); + TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context); collector.addResult(null, result1); collector.addResult(null, result2); collector.endResults(); @@ -222,7 +223,7 @@ public class TopEntriesFunctionCollectorJUnitTest { @Test public void mergeAfterClearResults() throws Exception { - TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(null); + TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(); collector.addResult(null, result1); collector.clearResults(); collector.addResult(null, result2); @@ -239,7 +240,8 @@ public class TopEntriesFunctionCollectorJUnitTest { TopEntriesCollectorManager mockManager = mock(TopEntriesCollectorManager.class); Mockito.doThrow(new IOException()).when(mockManager).reduce(any(Collection.class)); - TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(mockManager); + LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(mockManager); + TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context); collector.endResults(); collector.getResult(); } @@ -266,5 +268,4 @@ public class TopEntriesFunctionCollectorJUnitTest { TopEntriesFunctionCollector function = new TopEntriesFunctionCollector(null, mockCache); assertEquals("server", function.id); } - }
