You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/01/09 16:09:51 UTC

[01/34] lucene-solr:jira/solr-9856: SOLR-9923: Remove solr.http metric group and merge its metrics to solr.node group

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-9856 ca2e2befa -> 8616344ab


SOLR-9923: Remove solr.http metric group and merge its metrics to solr.node group


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

Branch: refs/heads/jira/solr-9856
Commit: e5d28848d7d98d52b378b2be18203f058a48a2cc
Parents: 7ef6a81
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 5 20:25:23 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 5 20:25:23 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                         |  2 +-
 .../src/java/org/apache/solr/core/CoreContainer.java     |  5 ++---
 .../src/java/org/apache/solr/core/SolrInfoMBean.java     |  4 ++--
 .../solr/handler/component/HttpShardHandlerFactory.java  |  8 +++++---
 .../java/org/apache/solr/update/UpdateShardHandler.java  | 11 ++++++-----
 .../solr/util/stats/InstrumentedHttpRequestExecutor.java |  2 +-
 .../apache/solr/handler/admin/MetricsHandlerTest.java    |  5 +----
 7 files changed, 18 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f4dd133..098dfda 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -201,7 +201,7 @@ New Features
 
 * SOLR-9725: Substitute properties into JdbcDataSource configuration ( Jamie Jackson, Yuri Sashevsky via Mikhail Khludnev)
 
-* SOLR-9877: Use instrumented http client and connection pool. (shalin)
+* SOLR-9877: SOLR-9923: Use instrumented http client and connection pool. (shalin)
 
 * SOLR-9880: Add Ganglia, Graphite and SLF4J metrics reporters. (ab)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index de7c34d..8a72617 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -471,11 +471,11 @@ public class CoreContainer {
     shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
     if (shardHandlerFactory instanceof SolrMetricProducer) {
       SolrMetricProducer metricProducer = (SolrMetricProducer) shardHandlerFactory;
-      metricProducer.initializeMetrics(metricManager, SolrInfoMBean.Group.http.toString(), "httpShardHandler");
+      metricProducer.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), "httpShardHandler");
     }
 
     updateShardHandler = new UpdateShardHandler(cfg.getUpdateShardHandlerConfig());
-    updateShardHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.http.toString(), "updateShardHandler");
+    updateShardHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), "updateShardHandler");
 
     solrCores.allocateLazyCores(cfg.getTransientCacheSize(), loader);
 
@@ -507,7 +507,6 @@ public class CoreContainer {
     metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.node);
     metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jvm);
     metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jetty);
-    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.http);
 
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index c64af47..421a4ef 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -32,12 +32,12 @@ public interface SolrInfoMBean {
   /**
    * Category of {@link SolrCore} component.
    */
-  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, OTHER }
+  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, HTTP, OTHER }
 
   /**
    * Top-level group of beans for a subsystem.
    */
-  enum Group { jvm, jetty, http, node, core }
+  enum Group { jvm, jetty, node, core }
 
   /**
    * Simple common usage name, e.g. BasicQueryHandler,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 14af88a..258be97 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.update.UpdateShardHandlerConfig;
@@ -363,10 +364,11 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    clientConnectionManager.initializeMetrics(manager, registry, scope);
-    httpRequestExecutor.initializeMetrics(manager, registry, scope);
+    String expandedScope = SolrMetricManager.mkName(scope, SolrInfoMBean.Category.HTTP.name());
+    clientConnectionManager.initializeMetrics(manager, registry, expandedScope);
+    httpRequestExecutor.initializeMetrics(manager, registry, expandedScope);
     commExecutor = MetricUtils.instrumentedExecutorService(commExecutor,
         manager.registry(registry),
-        SolrMetricManager.mkName("httpShardExecutor", scope, "threadPool"));
+        SolrMetricManager.mkName("httpShardExecutor", expandedScope, "threadPool"));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index cc13f3e..f13cfb5 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -98,14 +98,15 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
 
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    clientConnectionManager.initializeMetrics(manager, registry, scope);
-    httpRequestExecutor.initializeMetrics(manager, registry, scope);
+    String expandedScope = SolrMetricManager.mkName(scope, getCategory().name());
+    clientConnectionManager.initializeMetrics(manager, registry, expandedScope);
+    httpRequestExecutor.initializeMetrics(manager, registry, expandedScope);
     updateExecutor = new InstrumentedExecutorService(updateExecutor,
         manager.registry(registry),
-        SolrMetricManager.mkName("updateExecutor", scope, "threadPool"));
+        SolrMetricManager.mkName("updateExecutor", expandedScope, "threadPool"));
     recoveryExecutor = new InstrumentedExecutorService(recoveryExecutor,
         manager.registry(registry),
-        SolrMetricManager.mkName("recoveryExecutor", scope, "threadPool"));
+        SolrMetricManager.mkName("recoveryExecutor", expandedScope, "threadPool"));
   }
 
   @Override
@@ -115,7 +116,7 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
 
   @Override
   public Category getCategory() {
-    return null;
+    return Category.HTTP;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index 6f3f759..91ddd8a 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -44,7 +44,7 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   protected String scope;
 
   private static String methodNameString(HttpRequest request) {
-    return request.getRequestLine().getMethod().toLowerCase(Locale.ROOT) + "-requests";
+    return request.getRequestLine().getMethod().toLowerCase(Locale.ROOT) + ".requests";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5d28848/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index 67bf0e3..a3e724a 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -42,10 +42,8 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     NamedList values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
-    System.out.println(values);
     assertNotNull(values.get("solr.jetty"));
     assertNotNull(values.get("solr.jvm"));
-    assertNotNull(values.get("solr.http"));
     assertNotNull(values.get("solr.node"));
     NamedList nl = (NamedList) values.get("solr.core.collection1");
     assertNotNull(nl);
@@ -100,9 +98,8 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
-    assertEquals(5, values.size());
+    assertEquals(4, values.size());
     assertEquals(0, ((NamedList)values.get("solr.jvm")).size());
-    assertEquals(0, ((NamedList)values.get("solr.http")).size());
     assertEquals(0, ((NamedList)values.get("solr.jetty")).size());
     assertEquals(0, ((NamedList)values.get("solr.core.collection1")).size());
     assertEquals(3, ((NamedList)values.get("solr.node")).size());


[24/34] lucene-solr:jira/solr-9856: LUCENE-7620: UnifiedHighlighter: new LengthGoalBreakIterator wrapper

Posted by ab...@apache.org.
LUCENE-7620: UnifiedHighlighter: new LengthGoalBreakIterator wrapper


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

Branch: refs/heads/jira/solr-9856
Commit: ea49989524e96563f2b9bdd4256012239907882f
Parents: ac14fc3
Author: David Smiley <ds...@apache.org>
Authored: Sat Jan 7 23:10:48 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Jan 7 23:10:48 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +-
 .../uhighlight/LengthGoalBreakIterator.java     | 185 +++++++++++++++++++
 .../lucene/search/uhighlight/Passage.java       |   1 +
 .../uhighlight/LengthGoalBreakIteratorTest.java | 104 +++++++++++
 4 files changed, 295 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 30c9ab0..4bbf9ee 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -216,7 +216,11 @@ Improvements
   ensure all dimensions are indexed. (Adrien Grand)
 
 * LUCENE-7614: Complex Phrase Query parser ignores double quotes around single token 
-  prefix, wildcard, range queries (Mikhail Khludnev) 
+  prefix, wildcard, range queries (Mikhail Khludnev)
+
+* LUCENE-7620: Added LengthGoalBreakIterator, a wrapper around another B.I. to skip breaks
+  that would create Passages that are too short.  Only for use with the UnifiedHighlighter
+  (and probably PostingsHighlighter).  (David Smiley)
 
 Optimizations
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
new file mode 100644
index 0000000..3134013
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.uhighlight;
+
+import java.text.BreakIterator;
+import java.text.CharacterIterator;
+
+/**
+ * Wraps another {@link BreakIterator} to skip past breaks that would result in passages that are too
+ * short.  It's still possible to get a short passage but only at the very end of the input text.
+ * <p>
+ * Important: This is not a general purpose {@link BreakIterator}; it's only designed to work in a way
+ * compatible with the {@link UnifiedHighlighter}.  Some assumptions are checked with Java assertions.
+ *
+ * @lucene.experimental
+ */
+public class LengthGoalBreakIterator extends BreakIterator {
+
+  private final BreakIterator baseIter;
+  private final int lengthGoal;
+  private final boolean isMinimumLength; // if false then is "closest to" length
+
+  /** Breaks will be at least {@code minLength} apart (to the extent possible). */
+  public static LengthGoalBreakIterator createMinLength(BreakIterator baseIter, int minLength) {
+    return new LengthGoalBreakIterator(baseIter, minLength, true);
+  }
+
+  /** Breaks will be on average {@code targetLength} apart; the closest break to this target (before or after)
+   * is chosen. */
+  public static LengthGoalBreakIterator createClosestToLength(BreakIterator baseIter, int targetLength) {
+    return new LengthGoalBreakIterator(baseIter, targetLength, false);
+  }
+
+  private LengthGoalBreakIterator(BreakIterator baseIter, int lengthGoal, boolean isMinimumLength) {
+    this.baseIter = baseIter;
+    this.lengthGoal = lengthGoal;
+    this.isMinimumLength = isMinimumLength;
+  }
+
+  // note: the only methods that will get called are setText(txt), getText(),
+  // getSummaryPassagesNoHighlight: current(), first(), next()
+  // highlightOffsetsEnums: preceding(int), and following(int)
+  //   Nonetheless we make some attempt to implement the rest; mostly delegating.
+
+  @Override
+  public String toString() {
+    String goalDesc = isMinimumLength ? "minLen" : "targetLen";
+    return getClass().getSimpleName() + "{" + goalDesc + "=" + lengthGoal + ", baseIter=" + baseIter + "}";
+  }
+
+  @Override
+  public Object clone() {
+    return new LengthGoalBreakIterator((BreakIterator) baseIter.clone(), lengthGoal, isMinimumLength);
+  }
+
+  @Override
+  public CharacterIterator getText() {
+    return baseIter.getText();
+  }
+
+  @Override
+  public void setText(String newText) {
+    baseIter.setText(newText);
+  }
+
+  @Override
+  public void setText(CharacterIterator newText) {
+    baseIter.setText(newText);
+  }
+
+  @Override
+  public int current() {
+    return baseIter.current();
+  }
+
+  @Override
+  public int first() {
+    return baseIter.first();
+  }
+
+  @Override
+  public int last() {
+    return baseIter.last();
+  }
+
+  @Override
+  public int next(int n) {
+    assert false : "Not supported";
+    return baseIter.next(n); // probably wrong
+  }
+
+  // called by getSummaryPassagesNoHighlight to generate default summary.
+  @Override
+  public int next() {
+    return following(current());
+  }
+
+  @Override
+  public int previous() {
+    assert false : "Not supported";
+    return baseIter.previous();
+  }
+
+  // called while the current position is the start of a new passage; find end of passage
+  @Override
+  public int following(int followingIdx) {
+    final int startIdx = current();
+    if (followingIdx < startIdx) {
+      assert false : "Not supported";
+      return baseIter.following(followingIdx);
+    }
+    final int targetIdx = startIdx + lengthGoal;
+    // When followingIdx >= targetIdx, we can simply delegate since it will be >= the target
+    if (followingIdx >= targetIdx - 1) {
+      return baseIter.following(followingIdx);
+    }
+    // If target exceeds the text length, return the last index.
+    if (targetIdx >= getText().getEndIndex()) {
+      return baseIter.last();
+    }
+
+    // Find closest break >= the target
+    final int afterIdx = baseIter.following(targetIdx - 1);
+    if (afterIdx == DONE) { // we're at the end; can this happen?
+      return current();
+    }
+    if (afterIdx == targetIdx) { // right on the money
+      return afterIdx;
+    }
+    if (isMinimumLength) { // thus never undershoot
+      return afterIdx;
+    }
+
+    // note: it is a shame that we invoke preceding() *in addition to* following(); BI's are sometimes expensive.
+
+    // Find closest break < target
+    final int beforeIdx = baseIter.preceding(targetIdx); // or could do baseIter.previous() but we hope the BI implements preceding()
+    if (beforeIdx <= followingIdx) { // too far back
+      return moveToBreak(afterIdx);
+    }
+
+    if (targetIdx - beforeIdx <= afterIdx - targetIdx) {
+      return beforeIdx;
+    }
+    return moveToBreak(afterIdx);
+  }
+
+  private int moveToBreak(int idx) { // precondition: idx is a known break
+    // bi.isBoundary(idx) has side-effect of moving the position.  Not obvious!
+    //boolean moved = baseIter.isBoundary(idx); // probably not particularly expensive
+    //assert moved && current() == idx;
+
+    // TODO fix: Would prefer to do "- 1" instead of "- 2" but CustomSeparatorBreakIterator has a bug.
+    int current = baseIter.following(idx - 2);
+    assert current == idx : "following() didn't move us to the expected index.";
+    return idx;
+  }
+
+  // called at start of new Passage given first word start offset
+  @Override
+  public int preceding(int offset) {
+    return baseIter.preceding(offset); // no change needed
+  }
+
+  @Override
+  public boolean isBoundary(int offset) {
+    assert false : "Not supported";
+    return baseIter.isBoundary(offset);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
index d64b96e..3efb694 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
@@ -171,6 +171,7 @@ public class Passage {
 
   /** @lucene.internal */
   public void setEndOffset(int endOffset) {
+    assert startOffset <= endOffset;
     this.endOffset = endOffset;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
new file mode 100644
index 0000000..42d2bf6
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.uhighlight;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.postingshighlight.CustomSeparatorBreakIterator;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.QueryBuilder;
+
+public class LengthGoalBreakIteratorTest extends LuceneTestCase {
+  private static final String FIELD = "body";
+
+  // We test LengthGoalBreakIterator as it is used by the UnifiedHighlighter instead of directly, because it is
+  //  not a general purpose BreakIterator.  A unit test of it directly wouldn't give as much confidence.
+
+  private final Analyzer analyzer =
+      new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+
+  // We do a '.' BreakIterator and test varying the length goal.
+  //                      0         1
+  //                      01234567890123456789
+  final String content = "Aa bb. Cc dd. Ee ff";
+
+  public void testTargetLen() throws IOException {
+    // "goal" means target length goal to find closest break
+
+    // at first word:
+    Query query = query("aa");
+    assertEquals("almost two sent",
+        "<b>Aa</b> bb.", highlightClosestToLen(content, query, 9));
+    assertEquals( "barely two sent",
+        "<b>Aa</b> bb. Cc dd.", highlightClosestToLen(content, query, 10));
+    assertEquals("long goal",
+        "<b>Aa</b> bb. Cc dd. Ee ff", highlightClosestToLen(content, query, 17 + random().nextInt(20)));
+
+    // at some word not at start of passage
+    query = query("dd");
+    assertEquals("short goal",
+        " Cc <b>dd</b>.", highlightClosestToLen(content, query, random().nextInt(5)));
+    assertEquals("almost two sent",
+        " Cc <b>dd</b>.", highlightClosestToLen(content, query, 10));
+    assertEquals("barely two sent",
+        " Cc <b>dd</b>. Ee ff", highlightClosestToLen(content, query, 11));
+    assertEquals("long goal",
+        " Cc <b>dd</b>. Ee ff", highlightClosestToLen(content, query, 12 + random().nextInt(20)));
+  }
+
+  public void testMinLen() throws IOException {
+    // minLen mode is simpler than targetLen... just test a few cases
+
+    Query query = query("dd");
+    assertEquals("almost two sent",
+        " Cc <b>dd</b>.", highlightMinLen(content, query, 6));
+    assertEquals("barely two sent",
+        " Cc <b>dd</b>. Ee ff", highlightMinLen(content, query, 7));
+  }
+
+  public void testDefaultSummaryTargetLen() throws IOException {
+    Query query = query("zz");
+    assertEquals("Aa bb.",
+        highlightClosestToLen(content, query, random().nextInt(10))); // < 10
+    assertEquals("Aa bb. Cc dd.",
+        highlightClosestToLen(content, query, 10 + 6)); // cusp of adding 3rd sentence
+    assertEquals("Aa bb. Cc dd. Ee ff",
+        highlightClosestToLen(content, query, 17 + random().nextInt(20))); // >= 14
+  }
+
+  private Query query(String qStr) {
+    return new QueryBuilder(analyzer).createBooleanQuery(FIELD, qStr);
+  }
+
+  private String highlightClosestToLen(String content, Query query, int lengthGoal) throws IOException {
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
+    highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createClosestToLength(new CustomSeparatorBreakIterator('.'), lengthGoal));
+    return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
+  }
+
+  private String highlightMinLen(String content, Query query, int lengthGoal) throws IOException {
+    // differs from above only by "createMinLength"
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
+    highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createMinLength(new CustomSeparatorBreakIterator('.'), lengthGoal));
+    return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
+  }
+}
\ No newline at end of file


[13/34] lucene-solr:jira/solr-9856: LUCENE-7617: Grouping collector API cleanup

Posted by ab...@apache.org.
LUCENE-7617: Grouping collector API cleanup


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

Branch: refs/heads/jira/solr-9856
Commit: da30f21f5d2c90a4e3d4fae87a297adfd4bbb3cb
Parents: 52f2a77
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Jan 3 11:00:47 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 09:58:47 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +
 .../AbstractAllGroupHeadsCollector.java         | 176 ---------
 .../grouping/AbstractAllGroupsCollector.java    |  67 ----
 .../AbstractDistinctValuesCollector.java        |  59 ---
 .../AbstractFirstPassGroupingCollector.java     | 354 ------------------
 .../grouping/AbstractGroupFacetCollector.java   | 319 ----------------
 .../AbstractSecondPassGroupingCollector.java    | 162 ---------
 .../search/grouping/AllGroupHeadsCollector.java | 176 +++++++++
 .../search/grouping/AllGroupsCollector.java     |  67 ++++
 .../search/grouping/CollectedSearchGroup.java   |   2 +-
 .../grouping/DistinctValuesCollector.java       |  59 +++
 .../grouping/FirstPassGroupingCollector.java    | 363 +++++++++++++++++++
 .../lucene/search/grouping/GroupDocs.java       |   8 +-
 .../search/grouping/GroupFacetCollector.java    | 324 +++++++++++++++++
 .../apache/lucene/search/grouping/Grouper.java  |  56 +++
 .../lucene/search/grouping/GroupingSearch.java  | 130 ++-----
 .../lucene/search/grouping/SearchGroup.java     |  23 +-
 .../grouping/SecondPassGroupingCollector.java   | 169 +++++++++
 .../lucene/search/grouping/TopGroups.java       |  12 +-
 .../FunctionAllGroupHeadsCollector.java         |  32 +-
 .../function/FunctionAllGroupsCollector.java    |   4 +-
 .../FunctionDistinctValuesCollector.java        |  35 +-
 .../FunctionFirstPassGroupingCollector.java     |   6 +-
 .../grouping/function/FunctionGrouper.java      |  69 ++++
 .../FunctionSecondPassGroupingCollector.java    |   6 +-
 .../term/TermAllGroupHeadsCollector.java        |  26 +-
 .../grouping/term/TermAllGroupsCollector.java   |   8 +-
 .../term/TermDistinctValuesCollector.java       |  26 +-
 .../term/TermFirstPassGroupingCollector.java    |   6 +-
 .../grouping/term/TermGroupFacetCollector.java  |  10 +-
 .../search/grouping/term/TermGrouper.java       |  81 +++++
 .../term/TermSecondPassGroupingCollector.java   |   6 +-
 .../grouping/AllGroupHeadsCollectorTest.java    |  14 +-
 .../search/grouping/AllGroupsCollectorTest.java |   6 +-
 .../grouping/DistinctValuesCollectorTest.java   |  46 +--
 .../grouping/GroupFacetCollectorTest.java       |  10 +-
 .../lucene/search/grouping/TestGrouping.java    |  70 ++--
 .../org/apache/solr/request/SimpleFacets.java   |   4 +-
 .../java/org/apache/solr/search/Grouping.java   |  14 +-
 .../solr/search/grouping/CommandHandler.java    |   4 +-
 .../command/SearchGroupsFieldCommand.java       |   8 +-
 .../command/TopGroupsFieldCommand.java          |   4 +-
 42 files changed, 1597 insertions(+), 1430 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b74056f..fa5cc1c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -68,6 +68,12 @@ API Changes
 * LUCENE-7607: LeafFieldComparator.setScorer and SimpleFieldComparator.setScorer
   are declared as throwing IOException (Alan Woodward)
 
+* LUCENE-7617: Collector construction for two-pass grouping queries is
+  abstracted into a new Grouper class, which can be passed as a constructor
+  parameter to GroupingSearch.  The abstract base classes for the different
+  grouping Collectors are renamed to remove the Abstract* prefix.
+  (Alan Woodward, Martijn van Groningen)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
deleted file mode 100644
index 7108762..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.FixedBitSet;
-
-/**
- * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
- *
- * @lucene.experimental
- */
-@SuppressWarnings({"unchecked","rawtypes"})
-public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead> extends SimpleCollector {
-
-  protected final int[] reversed;
-  protected final int compIDXEnd;
-  protected final TemporalResult temporalResult;
-
-  protected AbstractAllGroupHeadsCollector(int numberOfSorts) {
-    this.reversed = new int[numberOfSorts];
-    this.compIDXEnd = numberOfSorts - 1;
-    temporalResult = new TemporalResult();
-  }
-
-  /**
-   * @param maxDoc The maxDoc of the top level {@link IndexReader}.
-   * @return a {@link FixedBitSet} containing all group heads.
-   */
-  public FixedBitSet retrieveGroupHeads(int maxDoc) {
-    FixedBitSet bitSet = new FixedBitSet(maxDoc);
-
-    Collection<GH> groupHeads = getCollectedGroupHeads();
-    for (GroupHead groupHead : groupHeads) {
-      bitSet.set(groupHead.doc);
-    }
-
-    return bitSet;
-  }
-
-  /**
-   * @return an int array containing all group heads. The size of the array is equal to number of collected unique groups.
-   */
-  public int[] retrieveGroupHeads() {
-    Collection<GH> groupHeads = getCollectedGroupHeads();
-    int[] docHeads = new int[groupHeads.size()];
-
-    int i = 0;
-    for (GroupHead groupHead : groupHeads) {
-      docHeads[i++] = groupHead.doc;
-    }
-
-    return docHeads;
-  }
-
-  /**
-   * @return the number of group heads found for a query.
-   */
-  public int groupHeadsSize() {
-    return getCollectedGroupHeads().size();
-  }
-
-  /**
-   * Returns the group head and puts it into {@link #temporalResult}.
-   * If the group head wasn't encountered before then it will be added to the collected group heads.
-   * <p>
-   * The {@link TemporalResult#stop} property will be <code>true</code> if the group head wasn't encountered before
-   * otherwise <code>false</code>.
-   *
-   * @param doc The document to retrieve the group head for.
-   * @throws IOException If I/O related errors occur
-   */
-  protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
-
-  /**
-   * Returns the collected group heads.
-   * Subsequent calls should return the same group heads.
-   *
-   * @return the collected group heads
-   */
-  protected abstract Collection<GH> getCollectedGroupHeads();
-
-  @Override
-  public void collect(int doc) throws IOException {
-    retrieveGroupHeadAndAddIfNotExist(doc);
-    if (temporalResult.stop) {
-      return;
-    }
-    GH groupHead = temporalResult.groupHead;
-
-    // Ok now we need to check if the current doc is more relevant then current doc for this group
-    for (int compIDX = 0; ; compIDX++) {
-      final int c = reversed[compIDX] * groupHead.compare(compIDX, doc);
-      if (c < 0) {
-        // Definitely not competitive. So don't even bother to continue
-        return;
-      } else if (c > 0) {
-        // Definitely competitive.
-        break;
-      } else if (compIDX == compIDXEnd) {
-        // Here c=0. If we're at the last comparator, this doc is not
-        // competitive, since docs are visited in doc Id order, which means
-        // this doc cannot compete with any other document in the queue.
-        return;
-      }
-    }
-    groupHead.updateDocHead(doc);
-  }
-
-  /**
-   * Contains the result of group head retrieval.
-   * To prevent new object creations of this class for every collect.
-   */
-  protected class TemporalResult {
-
-    public GH groupHead;
-    public boolean stop;
-
-  }
-
-  /**
-   * Represents a group head. A group head is the most relevant document for a particular group.
-   * The relevancy is based is usually based on the sort.
-   *
-   * The group head contains a group value with its associated most relevant document id.
-   */
-  public static abstract class GroupHead<GROUP_VALUE_TYPE> {
-
-    public final GROUP_VALUE_TYPE groupValue;
-    public int doc;
-
-    protected GroupHead(GROUP_VALUE_TYPE groupValue, int doc) {
-      this.groupValue = groupValue;
-      this.doc = doc;
-    }
-
-    /**
-     * Compares the specified document for a specified comparator against the current most relevant document.
-     *
-     * @param compIDX The comparator index of the specified comparator.
-     * @param doc The specified document.
-     * @return -1 if the specified document wasn't competitive against the current most relevant document, 1 if the
-     *         specified document was competitive against the current most relevant document. Otherwise 0.
-     * @throws IOException If I/O related errors occur
-     */
-    protected abstract int compare(int compIDX, int doc) throws IOException;
-
-    /**
-     * Updates the current most relevant document with the specified document.
-     *
-     * @param doc The specified document
-     * @throws IOException If I/O related errors occur
-     */
-    protected abstract void updateDocHead(int doc) throws IOException;
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
deleted file mode 100644
index 954f9e0..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * A collector that collects all groups that match the
- * query. Only the group value is collected, and the order
- * is undefined.  This collector does not determine
- * the most relevant document of a group.
- * <p>
- * This is an abstract version. Concrete implementations define
- * what a group actually is and how it is internally collected.
- *
- * @lucene.experimental
- */
-public abstract class AbstractAllGroupsCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
-
-  /**
-   * Returns the total number of groups for the executed search.
-   * This is a convenience method. The following code snippet has the same effect: <pre>getGroups().size()</pre>
-   *
-   * @return The total number of groups for the executed search
-   */
-  public int getGroupCount() {
-    return getGroups().size();
-  }
-
-  /**
-   * Returns the group values
-   * <p>
-   * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef}
-   * representing a group value.
-   *
-   * @return the group values
-   */
-  public abstract Collection<GROUP_VALUE_TYPE> getGroups();
-
-  // Empty not necessary
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {}
-
-  @Override
-  public boolean needsScores() {
-    return false; // the result is unaffected by relevancy
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
deleted file mode 100644
index b2181e4..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.lucene.search.SimpleCollector;
-
-/**
- * A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
- *
- * @lucene.experimental
- */
-public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends SimpleCollector {
-
-  /**
-   * Returns all unique values for each top N group.
-   *
-   * @return all unique values for each top N group
-   */
-  public abstract List<GC> getGroups();
-
-  /**
-   * Returned by {@link AbstractDistinctValuesCollector#getGroups()},
-   * representing the value and set of distinct values for the group.
-   */
-  public abstract static class GroupCount<GROUP_VALUE_TYPE> {
-
-    public final GROUP_VALUE_TYPE groupValue;
-    public final Set<GROUP_VALUE_TYPE> uniqueValues;
-
-    public GroupCount(GROUP_VALUE_TYPE groupValue) {
-      this.groupValue = groupValue;
-      this.uniqueValues = new HashSet<>();
-    }
-  }
-
-  @Override
-  public boolean needsScores() {
-    return false; // not needed to fetch all values
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
deleted file mode 100644
index 4de04f0..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
-
-import java.io.IOException;
-import java.util.*;
-
-/** FirstPassGroupingCollector is the first of two passes necessary
- *  to collect grouped hits.  This pass gathers the top N sorted
- *  groups. Concrete subclasses define what a group is and how it
- *  is internally collected.
- *
- *  <p>See {@link org.apache.lucene.search.grouping} for more
- *  details including a full code example.</p>
- *
- * @lucene.experimental
- */
-abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
-
-  private final FieldComparator<?>[] comparators;
-  private final LeafFieldComparator[] leafComparators;
-  private final int[] reversed;
-  private final int topNGroups;
-  private final boolean needsScores;
-  private final HashMap<GROUP_VALUE_TYPE, CollectedSearchGroup<GROUP_VALUE_TYPE>> groupMap;
-  private final int compIDXEnd;
-
-  // Set once we reach topNGroups unique groups:
-  /** @lucene.internal */
-  protected TreeSet<CollectedSearchGroup<GROUP_VALUE_TYPE>> orderedGroups;
-  private int docBase;
-  private int spareSlot;
-
-  /**
-   * Create the first pass collector.
-   *
-   *  @param groupSort The {@link Sort} used to sort the
-   *    groups.  The top sorted document within each group
-   *    according to groupSort, determines how that group
-   *    sorts against other groups.  This must be non-null,
-   *    ie, if you want to groupSort by relevance use
-   *    Sort.RELEVANCE.
-   *  @param topNGroups How many top groups to keep.
-   *  @throws IOException If I/O related errors occur
-   */
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public AbstractFirstPassGroupingCollector(Sort groupSort, int topNGroups) throws IOException {
-    if (topNGroups < 1) {
-      throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
-    }
-
-    // TODO: allow null groupSort to mean "by relevance",
-    // and specialize it?
-
-    this.topNGroups = topNGroups;
-    this.needsScores = groupSort.needsScores();
-    final SortField[] sortFields = groupSort.getSort();
-    comparators = new FieldComparator[sortFields.length];
-    leafComparators = new LeafFieldComparator[sortFields.length];
-    compIDXEnd = comparators.length - 1;
-    reversed = new int[sortFields.length];
-    for (int i = 0; i < sortFields.length; i++) {
-      final SortField sortField = sortFields[i];
-
-      // use topNGroups + 1 so we have a spare slot to use for comparing (tracked by this.spareSlot):
-      comparators[i] = sortField.getComparator(topNGroups + 1, i);
-      reversed[i] = sortField.getReverse() ? -1 : 1;
-    }
-
-    spareSlot = topNGroups;
-    groupMap = new HashMap<>(topNGroups);
-  }
-
-  @Override
-  public boolean needsScores() {
-    return needsScores;
-  }
-
-  /**
-   * Returns top groups, starting from offset.  This may
-   * return null, if no groups were collected, or if the
-   * number of unique groups collected is &lt;= offset.
-   *
-   * @param groupOffset The offset in the collected groups
-   * @param fillFields Whether to fill to {@link SearchGroup#sortValues}
-   * @return top groups, starting from offset
-   */
-  public Collection<SearchGroup<GROUP_VALUE_TYPE>> getTopGroups(int groupOffset, boolean fillFields) throws IOException {
-
-    //System.out.println("FP.getTopGroups groupOffset=" + groupOffset + " fillFields=" + fillFields + " groupMap.size()=" + groupMap.size());
-
-    if (groupOffset < 0) {
-      throw new IllegalArgumentException("groupOffset must be >= 0 (got " + groupOffset + ")");
-    }
-
-    if (groupMap.size() <= groupOffset) {
-      return null;
-    }
-
-    if (orderedGroups == null) {
-      buildSortedSet();
-    }
-
-    final Collection<SearchGroup<GROUP_VALUE_TYPE>> result = new ArrayList<>();
-    int upto = 0;
-    final int sortFieldCount = comparators.length;
-    for(CollectedSearchGroup<GROUP_VALUE_TYPE> group : orderedGroups) {
-      if (upto++ < groupOffset) {
-        continue;
-      }
-      //System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
-      SearchGroup<GROUP_VALUE_TYPE> searchGroup = new SearchGroup<>();
-      searchGroup.groupValue = group.groupValue;
-      if (fillFields) {
-        searchGroup.sortValues = new Object[sortFieldCount];
-        for(int sortFieldIDX=0;sortFieldIDX<sortFieldCount;sortFieldIDX++) {
-          searchGroup.sortValues[sortFieldIDX] = comparators[sortFieldIDX].value(group.comparatorSlot);
-        }
-      }
-      result.add(searchGroup);
-    }
-    //System.out.println("  return " + result.size() + " groups");
-    return result;
-  }
-
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    for (LeafFieldComparator comparator : leafComparators) {
-      comparator.setScorer(scorer);
-    }
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    //System.out.println("FP.collect doc=" + doc);
-
-    // If orderedGroups != null we already have collected N groups and
-    // can short circuit by comparing this document to the bottom group,
-    // without having to find what group this document belongs to.
-    
-    // Even if this document belongs to a group in the top N, we'll know that
-    // we don't have to update that group.
-
-    // Downside: if the number of unique groups is very low, this is
-    // wasted effort as we will most likely be updating an existing group.
-    if (orderedGroups != null) {
-      for (int compIDX = 0;; compIDX++) {
-        final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
-        if (c < 0) {
-          // Definitely not competitive. So don't even bother to continue
-          return;
-        } else if (c > 0) {
-          // Definitely competitive.
-          break;
-        } else if (compIDX == compIDXEnd) {
-          // Here c=0. If we're at the last comparator, this doc is not
-          // competitive, since docs are visited in doc Id order, which means
-          // this doc cannot compete with any other document in the queue.
-          return;
-        }
-      }
-    }
-
-    // TODO: should we add option to mean "ignore docs that
-    // don't have the group field" (instead of stuffing them
-    // under null group)?
-    final GROUP_VALUE_TYPE groupValue = getDocGroupValue(doc);
-
-    final CollectedSearchGroup<GROUP_VALUE_TYPE> group = groupMap.get(groupValue);
-
-    if (group == null) {
-
-      // First time we are seeing this group, or, we've seen
-      // it before but it fell out of the top N and is now
-      // coming back
-
-      if (groupMap.size() < topNGroups) {
-
-        // Still in startup transient: we have not
-        // seen enough unique groups to start pruning them;
-        // just keep collecting them
-
-        // Add a new CollectedSearchGroup:
-        CollectedSearchGroup<GROUP_VALUE_TYPE> sg = new CollectedSearchGroup<>();
-        sg.groupValue = copyDocGroupValue(groupValue, null);
-        sg.comparatorSlot = groupMap.size();
-        sg.topDoc = docBase + doc;
-        for (LeafFieldComparator fc : leafComparators) {
-          fc.copy(sg.comparatorSlot, doc);
-        }
-        groupMap.put(sg.groupValue, sg);
-
-        if (groupMap.size() == topNGroups) {
-          // End of startup transient: we now have max
-          // number of groups; from here on we will drop
-          // bottom group when we insert new one:
-          buildSortedSet();
-        }
-
-        return;
-      }
-
-      // We already tested that the document is competitive, so replace
-      // the bottom group with this new group.
-      final CollectedSearchGroup<GROUP_VALUE_TYPE> bottomGroup = orderedGroups.pollLast();
-      assert orderedGroups.size() == topNGroups -1;
-
-      groupMap.remove(bottomGroup.groupValue);
-
-      // reuse the removed CollectedSearchGroup
-      bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
-      bottomGroup.topDoc = docBase + doc;
-
-      for (LeafFieldComparator fc : leafComparators) {
-        fc.copy(bottomGroup.comparatorSlot, doc);
-      }
-
-      groupMap.put(bottomGroup.groupValue, bottomGroup);
-      orderedGroups.add(bottomGroup);
-      assert orderedGroups.size() == topNGroups;
-
-      final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
-      for (LeafFieldComparator fc : leafComparators) {
-        fc.setBottom(lastComparatorSlot);
-      }
-
-      return;
-    }
-
-    // Update existing group:
-    for (int compIDX = 0;; compIDX++) {
-      leafComparators[compIDX].copy(spareSlot, doc);
-
-      final int c = reversed[compIDX] * comparators[compIDX].compare(group.comparatorSlot, spareSlot);
-      if (c < 0) {
-        // Definitely not competitive.
-        return;
-      } else if (c > 0) {
-        // Definitely competitive; set remaining comparators:
-        for (int compIDX2=compIDX+1; compIDX2<comparators.length; compIDX2++) {
-          leafComparators[compIDX2].copy(spareSlot, doc);
-        }
-        break;
-      } else if (compIDX == compIDXEnd) {
-        // Here c=0. If we're at the last comparator, this doc is not
-        // competitive, since docs are visited in doc Id order, which means
-        // this doc cannot compete with any other document in the queue.
-        return;
-      }
-    }
-
-    // Remove before updating the group since lookup is done via comparators
-    // TODO: optimize this
-
-    final CollectedSearchGroup<GROUP_VALUE_TYPE> prevLast;
-    if (orderedGroups != null) {
-      prevLast = orderedGroups.last();
-      orderedGroups.remove(group);
-      assert orderedGroups.size() == topNGroups-1;
-    } else {
-      prevLast = null;
-    }
-
-    group.topDoc = docBase + doc;
-
-    // Swap slots
-    final int tmp = spareSlot;
-    spareSlot = group.comparatorSlot;
-    group.comparatorSlot = tmp;
-
-    // Re-add the changed group
-    if (orderedGroups != null) {
-      orderedGroups.add(group);
-      assert orderedGroups.size() == topNGroups;
-      final CollectedSearchGroup<?> newLast = orderedGroups.last();
-      // If we changed the value of the last group, or changed which group was last, then update bottom:
-      if (group == newLast || prevLast != newLast) {
-        for (LeafFieldComparator fc : leafComparators) {
-          fc.setBottom(newLast.comparatorSlot);
-        }
-      }
-    }
-  }
-
-  private void buildSortedSet() throws IOException {
-    final Comparator<CollectedSearchGroup<?>> comparator = new Comparator<CollectedSearchGroup<?>>() {
-      @Override
-      public int compare(CollectedSearchGroup<?> o1, CollectedSearchGroup<?> o2) {
-        for (int compIDX = 0;; compIDX++) {
-          FieldComparator<?> fc = comparators[compIDX];
-          final int c = reversed[compIDX] * fc.compare(o1.comparatorSlot, o2.comparatorSlot);
-          if (c != 0) {
-            return c;
-          } else if (compIDX == compIDXEnd) {
-            return o1.topDoc - o2.topDoc;
-          }
-        }
-      }
-    };
-
-    orderedGroups = new TreeSet<>(comparator);
-    orderedGroups.addAll(groupMap.values());
-    assert orderedGroups.size() > 0;
-
-    for (LeafFieldComparator fc : leafComparators) {
-      fc.setBottom(orderedGroups.last().comparatorSlot);
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    docBase = readerContext.docBase;
-    for (int i=0; i<comparators.length; i++) {
-      leafComparators[i] = comparators[i].getLeafComparator(readerContext);
-    }
-  }
-
-  /**
-   * Returns the group value for the specified doc.
-   *
-   * @param doc The specified doc
-   * @return the group value for the specified doc
-   */
-  protected abstract GROUP_VALUE_TYPE getDocGroupValue(int doc) throws IOException;
-
-  /**
-   * Returns a copy of the specified group value by creating a new instance and copying the value from the specified
-   * groupValue in the new instance. Or optionally the reuse argument can be used to copy the group value in.
-   *
-   * @param groupValue The group value to copy
-   * @param reuse Optionally a reuse instance to prevent a new instance creation
-   * @return a copy of the specified group value
-   */
-  protected abstract GROUP_VALUE_TYPE copyDocGroupValue(GROUP_VALUE_TYPE groupValue, GROUP_VALUE_TYPE reuse);
-
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
deleted file mode 100644
index 6824684..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.PriorityQueue;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * Base class for computing grouped facets.
- *
- * @lucene.experimental
- */
-public abstract class AbstractGroupFacetCollector extends SimpleCollector {
-
-  protected final String groupField;
-  protected final String facetField;
-  protected final BytesRef facetPrefix;
-  protected final List<SegmentResult> segmentResults;
-
-  protected int[] segmentFacetCounts;
-  protected int segmentTotalCount;
-  protected int startFacetOrd;
-  protected int endFacetOrd;
-
-  protected AbstractGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix) {
-    this.groupField = groupField;
-    this.facetField = facetField;
-    this.facetPrefix = facetPrefix;
-    segmentResults = new ArrayList<>();
-  }
-
-  /**
-   * Returns grouped facet results that were computed over zero or more segments.
-   * Grouped facet counts are merged from zero or more segment results.
-   *
-   * @param size The total number of facets to include. This is typically offset + limit
-   * @param minCount The minimum count a facet entry should have to be included in the grouped facet result
-   * @param orderByCount Whether to sort the facet entries by facet entry count. If <code>false</code> then the facets
-   *                     are sorted lexicographically in ascending order.
-   * @return grouped facet results
-   * @throws IOException If I/O related errors occur during merging segment grouped facet counts.
-   */
-  public GroupedFacetResult mergeSegmentResults(int size, int minCount, boolean orderByCount) throws IOException {
-    if (segmentFacetCounts != null) {
-      segmentResults.add(createSegmentResult());
-      segmentFacetCounts = null; // reset
-    }
-
-    int totalCount = 0;
-    int missingCount = 0;
-    SegmentResultPriorityQueue segments = new SegmentResultPriorityQueue(segmentResults.size());
-    for (SegmentResult segmentResult : segmentResults) {
-      missingCount += segmentResult.missing;
-      if (segmentResult.mergePos >= segmentResult.maxTermPos) {
-        continue;
-      }
-      totalCount += segmentResult.total;
-      segments.add(segmentResult);
-    }
-
-    GroupedFacetResult facetResult = new GroupedFacetResult(size, minCount, orderByCount, totalCount, missingCount);
-    while (segments.size() > 0) {
-      SegmentResult segmentResult = segments.top();
-      BytesRef currentFacetValue = BytesRef.deepCopyOf(segmentResult.mergeTerm);
-      int count = 0;
-
-      do {
-        count += segmentResult.counts[segmentResult.mergePos++];
-        if (segmentResult.mergePos < segmentResult.maxTermPos) {
-          segmentResult.nextTerm();
-          segmentResult = segments.updateTop();
-        } else {
-          segments.pop();
-          segmentResult = segments.top();
-          if (segmentResult == null) {
-            break;
-          }
-        }
-      } while (currentFacetValue.equals(segmentResult.mergeTerm));
-      facetResult.addFacetCount(currentFacetValue, count);
-    }
-    return facetResult;
-  }
-
-  protected abstract SegmentResult createSegmentResult() throws IOException;
-
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {
-  }
-
-  @Override
-  public boolean needsScores() {
-    return false;
-  }
-
-  /**
-   * The grouped facet result. Containing grouped facet entries, total count and total missing count.
-   */
-  public static class GroupedFacetResult {
-
-    private final static Comparator<FacetEntry> orderByCountAndValue = new Comparator<FacetEntry>() {
-
-      @Override
-      public int compare(FacetEntry a, FacetEntry b) {
-        int cmp = b.count - a.count; // Highest count first!
-        if (cmp != 0) {
-          return cmp;
-        }
-        return a.value.compareTo(b.value);
-      }
-
-    };
-
-    private final static Comparator<FacetEntry> orderByValue = new Comparator<FacetEntry>() {
-
-      @Override
-      public int compare(FacetEntry a, FacetEntry b) {
-        return a.value.compareTo(b.value);
-      }
-
-    };
-
-    private final int maxSize;
-    private final NavigableSet<FacetEntry> facetEntries;
-    private final int totalMissingCount;
-    private final int totalCount;
-
-    private int currentMin;
-
-    public GroupedFacetResult(int size, int minCount, boolean orderByCount, int totalCount, int totalMissingCount) {
-      this.facetEntries = new TreeSet<>(orderByCount ? orderByCountAndValue : orderByValue);
-      this.totalMissingCount = totalMissingCount;
-      this.totalCount = totalCount;
-      maxSize = size;
-      currentMin = minCount;
-    }
-
-    public void addFacetCount(BytesRef facetValue, int count) {
-      if (count < currentMin) {
-        return;
-      }
-
-      FacetEntry facetEntry = new FacetEntry(facetValue, count);
-      if (facetEntries.size() == maxSize) {
-        if (facetEntries.higher(facetEntry) == null) {
-          return;
-        }
-        facetEntries.pollLast();
-      }
-      facetEntries.add(facetEntry);
-
-      if (facetEntries.size() == maxSize) {
-        currentMin = facetEntries.last().count;
-      }
-    }
-
-    /**
-     * Returns a list of facet entries to be rendered based on the specified offset and limit.
-     * The facet entries are retrieved from the facet entries collected during merging.
-     *
-     * @param offset The offset in the collected facet entries during merging
-     * @param limit The number of facets to return starting from the offset.
-     * @return a list of facet entries to be rendered based on the specified offset and limit
-     */
-    public List<FacetEntry> getFacetEntries(int offset, int limit) {
-      List<FacetEntry> entries = new LinkedList<>();
-
-      int skipped = 0;
-      int included = 0;
-      for (FacetEntry facetEntry : facetEntries) {
-        if (skipped < offset) {
-          skipped++;
-          continue;
-        }
-        if (included++ >= limit) {
-          break;
-        }
-        entries.add(facetEntry);
-      }
-      return entries;
-    }
-
-    /**
-     * Returns the sum of all facet entries counts.
-     *
-     * @return the sum of all facet entries counts
-     */
-    public int getTotalCount() {
-      return totalCount;
-    }
-
-    /**
-     * Returns the number of groups that didn't have a facet value.
-     *
-     * @return the number of groups that didn't have a facet value
-     */
-    public int getTotalMissingCount() {
-      return totalMissingCount;
-    }
-  }
-
-  /**
-   * Represents a facet entry with a value and a count.
-   */
-  public static class FacetEntry {
-
-    private final BytesRef value;
-    private final int count;
-
-    public FacetEntry(BytesRef value, int count) {
-      this.value = value;
-      this.count = count;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (o == null || getClass() != o.getClass()) return false;
-
-      FacetEntry that = (FacetEntry) o;
-
-      if (count != that.count) return false;
-      if (!value.equals(that.value)) return false;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int result = value.hashCode();
-      result = 31 * result + count;
-      return result;
-    }
-
-    @Override
-    public String toString() {
-      return "FacetEntry{" +
-          "value=" + value.utf8ToString() +
-          ", count=" + count +
-          '}';
-    }
-
-    /**
-     * @return The value of this facet entry
-     */
-    public BytesRef getValue() {
-      return value;
-    }
-
-    /**
-     * @return The count (number of groups) of this facet entry.
-     */
-    public int getCount() {
-      return count;
-    }
-  }
-
-  /**
-   * Contains the local grouped segment counts for a particular segment.
-   * Each <code>SegmentResult</code> must be added together.
-   */
-  protected abstract static class SegmentResult {
-
-    protected final int[] counts;
-    protected final int total;
-    protected final int missing;
-    protected final int maxTermPos;
-
-    protected BytesRef mergeTerm;
-    protected int mergePos;
-
-    protected SegmentResult(int[] counts, int total, int missing, int maxTermPos) {
-      this.counts = counts;
-      this.total = total;
-      this.missing = missing;
-      this.maxTermPos = maxTermPos;
-    }
-
-    /**
-     * Go to next term in this <code>SegmentResult</code> in order to retrieve the grouped facet counts.
-     *
-     * @throws IOException If I/O related errors occur
-     */
-    protected abstract void nextTerm() throws IOException;
-
-  }
-
-  private static class SegmentResultPriorityQueue extends PriorityQueue<SegmentResult> {
-
-    SegmentResultPriorityQueue(int maxSize) {
-      super(maxSize);
-    }
-
-    @Override
-    protected boolean lessThan(SegmentResult a, SegmentResult b) {
-      return a.mergeTerm.compareTo(b.mergeTerm) < 0;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
deleted file mode 100644
index 13b6189..0000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * SecondPassGroupingCollector is the second of two passes
- * necessary to collect grouped docs.  This pass gathers the
- * top N documents per top group computed from the
- * first pass. Concrete subclasses define what a group is and how it
- * is internally collected.
- *
- * <p>See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.</p>
- *
- * @lucene.experimental
- */
-public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
-
-  private final Collection<SearchGroup<GROUP_VALUE_TYPE>> groups;
-  private final Sort groupSort;
-  private final Sort withinGroupSort;
-  private final int maxDocsPerGroup;
-  private final boolean needsScores;
-  protected final Map<GROUP_VALUE_TYPE, SearchGroupDocs<GROUP_VALUE_TYPE>> groupMap;
-
-  protected SearchGroupDocs<GROUP_VALUE_TYPE>[] groupDocs;
-
-  private int totalHitCount;
-  private int totalGroupedHitCount;
-
-  public AbstractSecondPassGroupingCollector(Collection<SearchGroup<GROUP_VALUE_TYPE>> groups, Sort groupSort, Sort withinGroupSort,
-                                             int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
-    throws IOException {
-
-    //System.out.println("SP init");
-    if (groups.isEmpty()) {
-      throw new IllegalArgumentException("no groups to collect (groups is empty)");
-    }
-
-    this.groups = Objects.requireNonNull(groups);
-    this.groupSort = Objects.requireNonNull(groupSort);
-    this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
-    this.maxDocsPerGroup = maxDocsPerGroup;
-    this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
-
-    this.groupMap = new HashMap<>(groups.size());
-    for (SearchGroup<GROUP_VALUE_TYPE> group : groups) {
-      //System.out.println("  prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
-      final TopDocsCollector<?> collector;
-      if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector
-        // Sort by score
-        collector = TopScoreDocCollector.create(maxDocsPerGroup);
-      } else {
-        // Sort by fields
-        collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
-      }
-      groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector));
-    }
-  }
-
-  @Override
-  public boolean needsScores() {
-    return needsScores;
-  }
-
-  @Override
-  public void setScorer(Scorer scorer) throws IOException {
-    for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
-      group.leafCollector.setScorer(scorer);
-    }
-  }
-
-  @Override
-  public void collect(int doc) throws IOException {
-    totalHitCount++;
-    SearchGroupDocs<GROUP_VALUE_TYPE> group = retrieveGroup(doc);
-    if (group != null) {
-      totalGroupedHitCount++;
-      group.leafCollector.collect(doc);
-    }
-  }
-
-  /**
-   * Returns the group the specified doc belongs to or <code>null</code> if no group could be retrieved.
-   *
-   * @param doc The specified doc
-   * @return the group the specified doc belongs to or <code>null</code> if no group could be retrieved
-   * @throws IOException If an I/O related error occurred
-   */
-  protected abstract SearchGroupDocs<GROUP_VALUE_TYPE> retrieveGroup(int doc) throws IOException;
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
-    //System.out.println("SP.setNextReader");
-    for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
-      group.leafCollector = group.collector.getLeafCollector(readerContext);
-    }
-  }
-
-  public TopGroups<GROUP_VALUE_TYPE> getTopGroups(int withinGroupOffset) {
-    @SuppressWarnings({"unchecked","rawtypes"})
-    final GroupDocs<GROUP_VALUE_TYPE>[] groupDocsResult = (GroupDocs<GROUP_VALUE_TYPE>[]) new GroupDocs[groups.size()];
-
-    int groupIDX = 0;
-    float maxScore = Float.MIN_VALUE;
-    for(SearchGroup<?> group : groups) {
-      final SearchGroupDocs<GROUP_VALUE_TYPE> groupDocs = groupMap.get(group.groupValue);
-      final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
-      groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
-                                                                    topDocs.getMaxScore(),
-                                                                    topDocs.totalHits,
-                                                                    topDocs.scoreDocs,
-                                                                    groupDocs.groupValue,
-                                                                    group.sortValues);
-      maxScore = Math.max(maxScore, topDocs.getMaxScore());
-    }
-
-    return new TopGroups<>(groupSort.getSort(),
-                                           withinGroupSort.getSort(),
-                                           totalHitCount, totalGroupedHitCount, groupDocsResult,
-                                           maxScore);
-  }
-
-
-  // TODO: merge with SearchGroup or not?
-  // ad: don't need to build a new hashmap
-  // disad: blows up the size of SearchGroup if we need many of them, and couples implementations
-  public class SearchGroupDocs<GROUP_VALUE_TYPE> {
-
-    public final GROUP_VALUE_TYPE groupValue;
-    public final TopDocsCollector<?> collector;
-    public LeafCollector leafCollector;
-
-    public SearchGroupDocs(GROUP_VALUE_TYPE groupValue, TopDocsCollector<?> collector) {
-      this.groupValue = groupValue;
-      this.collector = collector;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
new file mode 100644
index 0000000..b5fbdc3
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ *
+ * @lucene.experimental
+ */
+@SuppressWarnings({"unchecked","rawtypes"})
+public abstract class AllGroupHeadsCollector<T> extends SimpleCollector {
+
+  protected final int[] reversed;
+  protected final int compIDXEnd;
+  protected final TemporalResult temporalResult;
+
+  protected AllGroupHeadsCollector(int numberOfSorts) {
+    this.reversed = new int[numberOfSorts];
+    this.compIDXEnd = numberOfSorts - 1;
+    temporalResult = new TemporalResult();
+  }
+
+  /**
+   * @param maxDoc The maxDoc of the top level {@link IndexReader}.
+   * @return a {@link FixedBitSet} containing all group heads.
+   */
+  public FixedBitSet retrieveGroupHeads(int maxDoc) {
+    FixedBitSet bitSet = new FixedBitSet(maxDoc);
+
+    Collection<? extends GroupHead<T>> groupHeads = getCollectedGroupHeads();
+    for (GroupHead groupHead : groupHeads) {
+      bitSet.set(groupHead.doc);
+    }
+
+    return bitSet;
+  }
+
+  /**
+   * @return an int array containing all group heads. The size of the array is equal to number of collected unique groups.
+   */
+  public int[] retrieveGroupHeads() {
+    Collection<? extends GroupHead<T>> groupHeads = getCollectedGroupHeads();
+    int[] docHeads = new int[groupHeads.size()];
+
+    int i = 0;
+    for (GroupHead groupHead : groupHeads) {
+      docHeads[i++] = groupHead.doc;
+    }
+
+    return docHeads;
+  }
+
+  /**
+   * @return the number of group heads found for a query.
+   */
+  public int groupHeadsSize() {
+    return getCollectedGroupHeads().size();
+  }
+
+  /**
+   * Returns the group head and puts it into {@link #temporalResult}.
+   * If the group head wasn't encountered before then it will be added to the collected group heads.
+   * <p>
+   * The {@link TemporalResult#stop} property will be <code>true</code> if the group head wasn't encountered before
+   * otherwise <code>false</code>.
+   *
+   * @param doc The document to retrieve the group head for.
+   * @throws IOException If I/O related errors occur
+   */
+  protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
+
+  /**
+   * Returns the collected group heads.
+   * Subsequent calls should return the same group heads.
+   *
+   * @return the collected group heads
+   */
+  protected abstract Collection<? extends GroupHead<T>> getCollectedGroupHeads();
+
+  @Override
+  public void collect(int doc) throws IOException {
+    retrieveGroupHeadAndAddIfNotExist(doc);
+    if (temporalResult.stop) {
+      return;
+    }
+    GroupHead<T> groupHead = temporalResult.groupHead;
+
+    // Ok now we need to check if the current doc is more relevant then current doc for this group
+    for (int compIDX = 0; ; compIDX++) {
+      final int c = reversed[compIDX] * groupHead.compare(compIDX, doc);
+      if (c < 0) {
+        // Definitely not competitive. So don't even bother to continue
+        return;
+      } else if (c > 0) {
+        // Definitely competitive.
+        break;
+      } else if (compIDX == compIDXEnd) {
+        // Here c=0. If we're at the last comparator, this doc is not
+        // competitive, since docs are visited in doc Id order, which means
+        // this doc cannot compete with any other document in the queue.
+        return;
+      }
+    }
+    groupHead.updateDocHead(doc);
+  }
+
+  /**
+   * Contains the result of group head retrieval.
+   * To prevent new object creations of this class for every collect.
+   */
+  protected class TemporalResult {
+
+    public GroupHead<T> groupHead;
+    public boolean stop;
+
+  }
+
+  /**
+   * Represents a group head. A group head is the most relevant document for a particular group.
+   * The relevancy is based is usually based on the sort.
+   *
+   * The group head contains a group value with its associated most relevant document id.
+   */
+  public static abstract class GroupHead<T> {
+
+    public final T groupValue;
+    public int doc;
+
+    protected GroupHead(T groupValue, int doc) {
+      this.groupValue = groupValue;
+      this.doc = doc;
+    }
+
+    /**
+     * Compares the specified document for a specified comparator against the current most relevant document.
+     *
+     * @param compIDX The comparator index of the specified comparator.
+     * @param doc The specified document.
+     * @return -1 if the specified document wasn't competitive against the current most relevant document, 1 if the
+     *         specified document was competitive against the current most relevant document. Otherwise 0.
+     * @throws IOException If I/O related errors occur
+     */
+    protected abstract int compare(int compIDX, int doc) throws IOException;
+
+    /**
+     * Updates the current most relevant document with the specified document.
+     *
+     * @param doc The specified document
+     * @throws IOException If I/O related errors occur
+     */
+    protected abstract void updateDocHead(int doc) throws IOException;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
new file mode 100644
index 0000000..af697af
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A collector that collects all groups that match the
+ * query. Only the group value is collected, and the order
+ * is undefined.  This collector does not determine
+ * the most relevant document of a group.
+ * <p>
+ * This is an abstract version. Concrete implementations define
+ * what a group actually is and how it is internally collected.
+ *
+ * @lucene.experimental
+ */
+public abstract class AllGroupsCollector<T> extends SimpleCollector {
+
+  /**
+   * Returns the total number of groups for the executed search.
+   * This is a convenience method. The following code snippet has the same effect: <pre>getGroups().size()</pre>
+   *
+   * @return The total number of groups for the executed search
+   */
+  public int getGroupCount() {
+    return getGroups().size();
+  }
+
+  /**
+   * Returns the group values
+   * <p>
+   * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef}
+   * representing a group value.
+   *
+   * @return the group values
+   */
+  public abstract Collection<T> getGroups();
+
+  // Empty not necessary
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {}
+
+  @Override
+  public boolean needsScores() {
+    return false; // the result is unaffected by relevancy
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
index af6fd04..5e4bf14 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/CollectedSearchGroup.java
@@ -19,7 +19,7 @@ package org.apache.lucene.search.grouping;
 import org.apache.lucene.search.FieldComparator; // javadocs
 
 /** 
- * Expert: representation of a group in {@link AbstractFirstPassGroupingCollector},
+ * Expert: representation of a group in {@link FirstPassGroupingCollector},
  * tracking the top doc and {@link FieldComparator} slot.
  * @lucene.internal */
 public class CollectedSearchGroup<T> extends SearchGroup<T> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
new file mode 100644
index 0000000..54d752c
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.search.SimpleCollector;
+
+/**
+ * A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
+ *
+ * @lucene.experimental
+ */
+public abstract class DistinctValuesCollector<T> extends SimpleCollector {
+
+  /**
+   * Returns all unique values for each top N group.
+   *
+   * @return all unique values for each top N group
+   */
+  public abstract List<GroupCount<T>> getGroups();
+
+  /**
+   * Returned by {@link DistinctValuesCollector#getGroups()},
+   * representing the value and set of distinct values for the group.
+   */
+  public static class GroupCount<T> {
+
+    public final T groupValue;
+    public final Set<T> uniqueValues;
+
+    public GroupCount(T groupValue) {
+      this.groupValue = groupValue;
+      this.uniqueValues = new HashSet<>();
+    }
+  }
+
+  @Override
+  public boolean needsScores() {
+    return false; // not needed to fetch all values
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
new file mode 100644
index 0000000..ef47f96
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -0,0 +1,363 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.TreeSet;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+
+/** FirstPassGroupingCollector is the first of two passes necessary
+ *  to collect grouped hits.  This pass gathers the top N sorted
+ *  groups. Concrete subclasses define what a group is and how it
+ *  is internally collected.
+ *
+ *  <p>See {@link org.apache.lucene.search.grouping} for more
+ *  details including a full code example.</p>
+ *
+ * @lucene.experimental
+ */
+abstract public class FirstPassGroupingCollector<T> extends SimpleCollector {
+
+  private final FieldComparator<?>[] comparators;
+  private final LeafFieldComparator[] leafComparators;
+  private final int[] reversed;
+  private final int topNGroups;
+  private final boolean needsScores;
+  private final HashMap<T, CollectedSearchGroup<T>> groupMap;
+  private final int compIDXEnd;
+
+  // Set once we reach topNGroups unique groups:
+  /** @lucene.internal */
+  protected TreeSet<CollectedSearchGroup<T>> orderedGroups;
+  private int docBase;
+  private int spareSlot;
+
+  /**
+   * Create the first pass collector.
+   *
+   *  @param groupSort The {@link Sort} used to sort the
+   *    groups.  The top sorted document within each group
+   *    according to groupSort, determines how that group
+   *    sorts against other groups.  This must be non-null,
+   *    ie, if you want to groupSort by relevance use
+   *    Sort.RELEVANCE.
+   *  @param topNGroups How many top groups to keep.
+   *  @throws IOException If I/O related errors occur
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public FirstPassGroupingCollector(Sort groupSort, int topNGroups) throws IOException {
+    if (topNGroups < 1) {
+      throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
+    }
+
+    // TODO: allow null groupSort to mean "by relevance",
+    // and specialize it?
+
+    this.topNGroups = topNGroups;
+    this.needsScores = groupSort.needsScores();
+    final SortField[] sortFields = groupSort.getSort();
+    comparators = new FieldComparator[sortFields.length];
+    leafComparators = new LeafFieldComparator[sortFields.length];
+    compIDXEnd = comparators.length - 1;
+    reversed = new int[sortFields.length];
+    for (int i = 0; i < sortFields.length; i++) {
+      final SortField sortField = sortFields[i];
+
+      // use topNGroups + 1 so we have a spare slot to use for comparing (tracked by this.spareSlot):
+      comparators[i] = sortField.getComparator(topNGroups + 1, i);
+      reversed[i] = sortField.getReverse() ? -1 : 1;
+    }
+
+    spareSlot = topNGroups;
+    groupMap = new HashMap<>(topNGroups);
+  }
+
+  @Override
+  public boolean needsScores() {
+    return needsScores;
+  }
+
+  /**
+   * Returns top groups, starting from offset.  This may
+   * return null, if no groups were collected, or if the
+   * number of unique groups collected is &lt;= offset.
+   *
+   * @param groupOffset The offset in the collected groups
+   * @param fillFields Whether to fill to {@link SearchGroup#sortValues}
+   * @return top groups, starting from offset
+   */
+  public Collection<SearchGroup<T>> getTopGroups(int groupOffset, boolean fillFields) throws IOException {
+
+    //System.out.println("FP.getTopGroups groupOffset=" + groupOffset + " fillFields=" + fillFields + " groupMap.size()=" + groupMap.size());
+
+    if (groupOffset < 0) {
+      throw new IllegalArgumentException("groupOffset must be >= 0 (got " + groupOffset + ")");
+    }
+
+    if (groupMap.size() <= groupOffset) {
+      return null;
+    }
+
+    if (orderedGroups == null) {
+      buildSortedSet();
+    }
+
+    final Collection<SearchGroup<T>> result = new ArrayList<>();
+    int upto = 0;
+    final int sortFieldCount = comparators.length;
+    for(CollectedSearchGroup<T> group : orderedGroups) {
+      if (upto++ < groupOffset) {
+        continue;
+      }
+      //System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+      SearchGroup<T> searchGroup = new SearchGroup<>();
+      searchGroup.groupValue = group.groupValue;
+      if (fillFields) {
+        searchGroup.sortValues = new Object[sortFieldCount];
+        for(int sortFieldIDX=0;sortFieldIDX<sortFieldCount;sortFieldIDX++) {
+          searchGroup.sortValues[sortFieldIDX] = comparators[sortFieldIDX].value(group.comparatorSlot);
+        }
+      }
+      result.add(searchGroup);
+    }
+    //System.out.println("  return " + result.size() + " groups");
+    return result;
+  }
+
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {
+    for (LeafFieldComparator comparator : leafComparators) {
+      comparator.setScorer(scorer);
+    }
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    //System.out.println("FP.collect doc=" + doc);
+
+    // If orderedGroups != null we already have collected N groups and
+    // can short circuit by comparing this document to the bottom group,
+    // without having to find what group this document belongs to.
+    
+    // Even if this document belongs to a group in the top N, we'll know that
+    // we don't have to update that group.
+
+    // Downside: if the number of unique groups is very low, this is
+    // wasted effort as we will most likely be updating an existing group.
+    if (orderedGroups != null) {
+      for (int compIDX = 0;; compIDX++) {
+        final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
+        if (c < 0) {
+          // Definitely not competitive. So don't even bother to continue
+          return;
+        } else if (c > 0) {
+          // Definitely competitive.
+          break;
+        } else if (compIDX == compIDXEnd) {
+          // Here c=0. If we're at the last comparator, this doc is not
+          // competitive, since docs are visited in doc Id order, which means
+          // this doc cannot compete with any other document in the queue.
+          return;
+        }
+      }
+    }
+
+    // TODO: should we add option to mean "ignore docs that
+    // don't have the group field" (instead of stuffing them
+    // under null group)?
+    final T groupValue = getDocGroupValue(doc);
+
+    final CollectedSearchGroup<T> group = groupMap.get(groupValue);
+
+    if (group == null) {
+
+      // First time we are seeing this group, or, we've seen
+      // it before but it fell out of the top N and is now
+      // coming back
+
+      if (groupMap.size() < topNGroups) {
+
+        // Still in startup transient: we have not
+        // seen enough unique groups to start pruning them;
+        // just keep collecting them
+
+        // Add a new CollectedSearchGroup:
+        CollectedSearchGroup<T> sg = new CollectedSearchGroup<>();
+        sg.groupValue = copyDocGroupValue(groupValue, null);
+        sg.comparatorSlot = groupMap.size();
+        sg.topDoc = docBase + doc;
+        for (LeafFieldComparator fc : leafComparators) {
+          fc.copy(sg.comparatorSlot, doc);
+        }
+        groupMap.put(sg.groupValue, sg);
+
+        if (groupMap.size() == topNGroups) {
+          // End of startup transient: we now have max
+          // number of groups; from here on we will drop
+          // bottom group when we insert new one:
+          buildSortedSet();
+        }
+
+        return;
+      }
+
+      // We already tested that the document is competitive, so replace
+      // the bottom group with this new group.
+      final CollectedSearchGroup<T> bottomGroup = orderedGroups.pollLast();
+      assert orderedGroups.size() == topNGroups -1;
+
+      groupMap.remove(bottomGroup.groupValue);
+
+      // reuse the removed CollectedSearchGroup
+      bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+      bottomGroup.topDoc = docBase + doc;
+
+      for (LeafFieldComparator fc : leafComparators) {
+        fc.copy(bottomGroup.comparatorSlot, doc);
+      }
+
+      groupMap.put(bottomGroup.groupValue, bottomGroup);
+      orderedGroups.add(bottomGroup);
+      assert orderedGroups.size() == topNGroups;
+
+      final int lastComparatorSlot = orderedGroups.last().comparatorSlot;
+      for (LeafFieldComparator fc : leafComparators) {
+        fc.setBottom(lastComparatorSlot);
+      }
+
+      return;
+    }
+
+    // Update existing group:
+    for (int compIDX = 0;; compIDX++) {
+      leafComparators[compIDX].copy(spareSlot, doc);
+
+      final int c = reversed[compIDX] * comparators[compIDX].compare(group.comparatorSlot, spareSlot);
+      if (c < 0) {
+        // Definitely not competitive.
+        return;
+      } else if (c > 0) {
+        // Definitely competitive; set remaining comparators:
+        for (int compIDX2=compIDX+1; compIDX2<comparators.length; compIDX2++) {
+          leafComparators[compIDX2].copy(spareSlot, doc);
+        }
+        break;
+      } else if (compIDX == compIDXEnd) {
+        // Here c=0. If we're at the last comparator, this doc is not
+        // competitive, since docs are visited in doc Id order, which means
+        // this doc cannot compete with any other document in the queue.
+        return;
+      }
+    }
+
+    // Remove before updating the group since lookup is done via comparators
+    // TODO: optimize this
+
+    final CollectedSearchGroup<T> prevLast;
+    if (orderedGroups != null) {
+      prevLast = orderedGroups.last();
+      orderedGroups.remove(group);
+      assert orderedGroups.size() == topNGroups-1;
+    } else {
+      prevLast = null;
+    }
+
+    group.topDoc = docBase + doc;
+
+    // Swap slots
+    final int tmp = spareSlot;
+    spareSlot = group.comparatorSlot;
+    group.comparatorSlot = tmp;
+
+    // Re-add the changed group
+    if (orderedGroups != null) {
+      orderedGroups.add(group);
+      assert orderedGroups.size() == topNGroups;
+      final CollectedSearchGroup<?> newLast = orderedGroups.last();
+      // If we changed the value of the last group, or changed which group was last, then update bottom:
+      if (group == newLast || prevLast != newLast) {
+        for (LeafFieldComparator fc : leafComparators) {
+          fc.setBottom(newLast.comparatorSlot);
+        }
+      }
+    }
+  }
+
+  private void buildSortedSet() throws IOException {
+    final Comparator<CollectedSearchGroup<?>> comparator = new Comparator<CollectedSearchGroup<?>>() {
+      @Override
+      public int compare(CollectedSearchGroup<?> o1, CollectedSearchGroup<?> o2) {
+        for (int compIDX = 0;; compIDX++) {
+          FieldComparator<?> fc = comparators[compIDX];
+          final int c = reversed[compIDX] * fc.compare(o1.comparatorSlot, o2.comparatorSlot);
+          if (c != 0) {
+            return c;
+          } else if (compIDX == compIDXEnd) {
+            return o1.topDoc - o2.topDoc;
+          }
+        }
+      }
+    };
+
+    orderedGroups = new TreeSet<>(comparator);
+    orderedGroups.addAll(groupMap.values());
+    assert orderedGroups.size() > 0;
+
+    for (LeafFieldComparator fc : leafComparators) {
+      fc.setBottom(orderedGroups.last().comparatorSlot);
+    }
+  }
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
+    docBase = readerContext.docBase;
+    for (int i=0; i<comparators.length; i++) {
+      leafComparators[i] = comparators[i].getLeafComparator(readerContext);
+    }
+  }
+
+  /**
+   * Returns the group value for the specified doc.
+   *
+   * @param doc The specified doc
+   * @return the group value for the specified doc
+   */
+  protected abstract T getDocGroupValue(int doc) throws IOException;
+
+  /**
+   * Returns a copy of the specified group value by creating a new instance and copying the value from the specified
+   * groupValue in the new instance. Or optionally the reuse argument can be used to copy the group value in.
+   *
+   * @param groupValue The group value to copy
+   * @param reuse Optionally a reuse instance to prevent a new instance creation
+   * @return a copy of the specified group value
+   */
+  protected abstract T copyDocGroupValue(T groupValue, T reuse);
+
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
index a310703..48f12aa 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupDocs.java
@@ -21,10 +21,10 @@ import org.apache.lucene.search.ScoreDoc;
 /** Represents one group in the results.
  * 
  * @lucene.experimental */
-public class GroupDocs<GROUP_VALUE_TYPE> {
+public class GroupDocs<T> {
   /** The groupField value for all docs in this group; this
    *  may be null if hits did not have the groupField. */
-  public final GROUP_VALUE_TYPE groupValue;
+  public final T groupValue;
 
   /** Max score in this group */
   public final float maxScore;
@@ -42,14 +42,14 @@ public class GroupDocs<GROUP_VALUE_TYPE> {
   public final int totalHits;
 
   /** Matches the groupSort passed to {@link
-   *  AbstractFirstPassGroupingCollector}. */
+   *  FirstPassGroupingCollector}. */
   public final Object[] groupSortValues;
 
   public GroupDocs(float score,
                    float maxScore,
                    int totalHits,
                    ScoreDoc[] scoreDocs,
-                   GROUP_VALUE_TYPE groupValue,
+                   T groupValue,
                    Object[] groupSortValues) {
     this.score = score;
     this.maxScore = maxScore;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
new file mode 100644
index 0000000..fc6ef96
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupFacetCollector.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * Base class for computing grouped facets.
+ *
+ * @lucene.experimental
+ */
+public abstract class GroupFacetCollector extends SimpleCollector {
+
+  protected final String groupField;
+  protected final String facetField;
+  protected final BytesRef facetPrefix;
+  protected final List<SegmentResult> segmentResults;
+
+  protected int[] segmentFacetCounts;
+  protected int segmentTotalCount;
+  protected int startFacetOrd;
+  protected int endFacetOrd;
+
+  protected GroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix) {
+    this.groupField = groupField;
+    this.facetField = facetField;
+    this.facetPrefix = facetPrefix;
+    segmentResults = new ArrayList<>();
+  }
+
+  /**
+   * Returns grouped facet results that were computed over zero or more segments.
+   * Grouped facet counts are merged from zero or more segment results.
+   *
+   * @param size The total number of facets to include. This is typically offset + limit
+   * @param minCount The minimum count a facet entry should have to be included in the grouped facet result
+   * @param orderByCount Whether to sort the facet entries by facet entry count. If <code>false</code> then the facets
+   *                     are sorted lexicographically in ascending order.
+   * @return grouped facet results
+   * @throws IOException If I/O related errors occur during merging segment grouped facet counts.
+   */
+  public GroupedFacetResult mergeSegmentResults(int size, int minCount, boolean orderByCount) throws IOException {
+    if (segmentFacetCounts != null) {
+      segmentResults.add(createSegmentResult());
+      segmentFacetCounts = null; // reset
+    }
+
+    int totalCount = 0;
+    int missingCount = 0;
+    SegmentResultPriorityQueue segments = new SegmentResultPriorityQueue(segmentResults.size());
+    for (SegmentResult segmentResult : segmentResults) {
+      missingCount += segmentResult.missing;
+      if (segmentResult.mergePos >= segmentResult.maxTermPos) {
+        continue;
+      }
+      totalCount += segmentResult.total;
+      segments.add(segmentResult);
+    }
+
+    GroupedFacetResult facetResult = new GroupedFacetResult(size, minCount, orderByCount, totalCount, missingCount);
+    while (segments.size() > 0) {
+      SegmentResult segmentResult = segments.top();
+      BytesRef currentFacetValue = BytesRef.deepCopyOf(segmentResult.mergeTerm);
+      int count = 0;
+
+      do {
+        count += segmentResult.counts[segmentResult.mergePos++];
+        if (segmentResult.mergePos < segmentResult.maxTermPos) {
+          segmentResult.nextTerm();
+          segmentResult = segments.updateTop();
+        } else {
+          segments.pop();
+          segmentResult = segments.top();
+          if (segmentResult == null) {
+            break;
+          }
+        }
+      } while (currentFacetValue.equals(segmentResult.mergeTerm));
+      facetResult.addFacetCount(currentFacetValue, count);
+    }
+    return facetResult;
+  }
+
+  protected abstract SegmentResult createSegmentResult() throws IOException;
+
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {
+  }
+
+  @Override
+  public boolean needsScores() {
+    return false;
+  }
+
+  /**
+   * The grouped facet result. Containing grouped facet entries, total count and total missing count.
+   */
+  public static class GroupedFacetResult {
+
+    private final static Comparator<FacetEntry> orderByCountAndValue = new Comparator<FacetEntry>() {
+
+      @Override
+      public int compare(FacetEntry a, FacetEntry b) {
+        int cmp = b.count - a.count; // Highest count first!
+        if (cmp != 0) {
+          return cmp;
+        }
+        return a.value.compareTo(b.value);
+      }
+
+    };
+
+    private final static Comparator<FacetEntry> orderByValue = new Comparator<FacetEntry>() {
+
+      @Override
+      public int compare(FacetEntry a, FacetEntry b) {
+        return a.value.compareTo(b.value);
+      }
+
+    };
+
+    private final int maxSize;
+    private final NavigableSet<FacetEntry> facetEntries;
+    private final int totalMissingCount;
+    private final int totalCount;
+
+    private int currentMin;
+
+    public GroupedFacetResult(int size, int minCount, boolean orderByCount, int totalCount, int totalMissingCount) {
+      this.facetEntries = new TreeSet<>(orderByCount ? orderByCountAndValue : orderByValue);
+      this.totalMissingCount = totalMissingCount;
+      this.totalCount = totalCount;
+      maxSize = size;
+      currentMin = minCount;
+    }
+
+    public void addFacetCount(BytesRef facetValue, int count) {
+      if (count < currentMin) {
+        return;
+      }
+
+      FacetEntry facetEntry = new FacetEntry(facetValue, count);
+      if (facetEntries.size() == maxSize) {
+        if (facetEntries.higher(facetEntry) == null) {
+          return;
+        }
+        facetEntries.pollLast();
+      }
+      facetEntries.add(facetEntry);
+
+      if (facetEntries.size() == maxSize) {
+        currentMin = facetEntries.last().count;
+      }
+    }
+
+    /**
+     * Returns a list of facet entries to be rendered based on the specified offset and limit.
+     * The facet entries are retrieved from the facet entries collected during merging.
+     *
+     * @param offset The offset in the collected facet entries during merging
+     * @param limit The number of facets to return starting from the offset.
+     * @return a list of facet entries to be rendered based on the specified offset and limit
+     */
+    public List<FacetEntry> getFacetEntries(int offset, int limit) {
+      List<FacetEntry> entries = new LinkedList<>();
+
+      int skipped = 0;
+      int included = 0;
+      for (FacetEntry facetEntry : facetEntries) {
+        if (skipped < offset) {
+          skipped++;
+          continue;
+        }
+        if (included++ >= limit) {
+          break;
+        }
+        entries.add(facetEntry);
+      }
+      return entries;
+    }
+
+    /**
+     * Returns the sum of all facet entries counts.
+     *
+     * @return the sum of all facet entries counts
+     */
+    public int getTotalCount() {
+      return totalCount;
+    }
+
+    /**
+     * Returns the number of groups that didn't have a facet value.
+     *
+     * @return the number of groups that didn't have a facet value
+     */
+    public int getTotalMissingCount() {
+      return totalMissingCount;
+    }
+  }
+
+  /**
+   * Represents a facet entry with a value and a count.
+   */
+  public static class FacetEntry {
+
+    private final BytesRef value;
+    private final int count;
+
+    public FacetEntry(BytesRef value, int count) {
+      this.value = value;
+      this.count = count;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      FacetEntry that = (FacetEntry) o;
+
+      if (count != that.count) return false;
+      if (!value.equals(that.value)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = value.hashCode();
+      result = 31 * result + count;
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return "FacetEntry{" +
+          "value=" + value.utf8ToString() +
+          ", count=" + count +
+          '}';
+    }
+
+    /**
+     * @return The value of this facet entry
+     */
+    public BytesRef getValue() {
+      return value;
+    }
+
+    /**
+     * @return The count (number of groups) of this facet entry.
+     */
+    public int getCount() {
+      return count;
+    }
+  }
+
+  /**
+   * Contains the local grouped segment counts for a particular segment.
+   * Each <code>SegmentResult</code> must be added together.
+   */
+  protected abstract static class SegmentResult {
+
+    protected final int[] counts;
+    protected final int total;
+    protected final int missing;
+    protected final int maxTermPos;
+
+    protected BytesRef mergeTerm;
+    protected int mergePos;
+
+    protected SegmentResult(int[] counts, int total, int missing, int maxTermPos) {
+      this.counts = counts;
+      this.total = total;
+      this.missing = missing;
+      this.maxTermPos = maxTermPos;
+    }
+
+    /**
+     * Go to next term in this <code>SegmentResult</code> in order to retrieve the grouped facet counts.
+     *
+     * @throws IOException If I/O related errors occur
+     */
+    protected abstract void nextTerm() throws IOException;
+
+  }
+
+  private static class SegmentResultPriorityQueue extends PriorityQueue<SegmentResult> {
+
+    SegmentResultPriorityQueue(int maxSize) {
+      super(maxSize);
+    }
+
+    @Override
+    protected boolean lessThan(SegmentResult a, SegmentResult b) {
+      return a.mergeTerm.compareTo(b.mergeTerm) < 0;
+    }
+  }
+
+}


[05/34] lucene-solr:jira/solr-9856: SOLR-9931: return 0 for hll on field with no values in bucket

Posted by ab...@apache.org.
SOLR-9931: return 0 for hll on field with no values in bucket


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

Branch: refs/heads/jira/solr-9856
Commit: a810fb3234ec461e23c76533fbfcc523d4c46faa
Parents: 2cffa2e
Author: yonik <yo...@apache.org>
Authored: Thu Jan 5 17:02:24 2017 -0500
Committer: yonik <yo...@apache.org>
Committed: Thu Jan 5 17:02:24 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../org/apache/solr/search/facet/HLLAgg.java    | 13 +++++++---
 .../apache/solr/search/facet/UniqueSlotAcc.java |  2 +-
 .../solr/search/facet/TestJsonFacets.java       | 27 ++++++++++++++++++++
 4 files changed, 41 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a810fb32/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9de2d76..b747876 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -332,6 +332,9 @@ Bug Fixes
 * SOLR-9917: JSON Facet API percentile function caused a NullPointerException in distributed mode when
   there were no values in a bucket from a shard. (yonik)
 
+* SOLR-9931: JSON Facet API hll (hyper-log-log) function returned 0 for non-empty buckets with no field values
+  in local mode, but nothing for distributed mode.  Both modes now return 0.  (yonik)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a810fb32/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
index 005e9fc..72cdd27 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
@@ -27,6 +27,8 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
 
 public class HLLAgg extends StrAggValueSource {
+  public static Integer NO_VALUES = 0;
+
   protected HLLFactory factory;
 
   public HLLAgg(String field) {
@@ -71,10 +73,15 @@ public class HLLAgg extends StrAggValueSource {
 
   private static class Merger extends FacetSortableMerger {
     HLL aggregate = null;
-    long answer = -1;
+    long answer = -1; // -1 means unset
 
     @Override
     public void merge(Object facetResult, Context mcontext) {
+      if (facetResult instanceof Number) {
+        assert NO_VALUES.equals(facetResult);
+        return;
+      }
+
       SimpleOrderedMap map = (SimpleOrderedMap)facetResult;
       byte[] serialized = ((byte[])map.get("hll"));
       HLL subHLL = HLL.fromBytes(serialized);
@@ -87,7 +94,7 @@ public class HLLAgg extends StrAggValueSource {
 
     private long getLong() {
       if (answer < 0) {
-        answer = aggregate.cardinality();
+        answer = aggregate == null ? 0 : aggregate.cardinality();
       }
       return answer;
     }
@@ -170,7 +177,7 @@ public class HLLAgg extends StrAggValueSource {
 
     public Object getShardValue(int slot) throws IOException {
       HLL hll = sets[slot];
-      if (hll == null) return null;
+      if (hll == null) return NO_VALUES;
       SimpleOrderedMap map = new SimpleOrderedMap();
       map.add("hll", hll.toBytes());
       // optionally use explicit values

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a810fb32/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
index 9f9e9b1..ae542ac 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
@@ -73,7 +73,7 @@ abstract class UniqueSlotAcc extends SlotAcc {
 
   private Object getShardHLL(int slot) throws IOException {
     FixedBitSet ords = arr[slot];
-    if (ords == null) return null;  // TODO: when we get to refinements, may be useful to return something???
+    if (ords == null) return HLLAgg.NO_VALUES;
 
     HLL hll = factory.getHLL();
     long maxOrd = ords.length();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a810fb32/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index d7e1cc0..d8f3ae5 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -870,6 +870,33 @@ public class TestJsonFacets extends SolrTestCaseHS {
             " }"
     );
 
+
+    // stats at top level, matching documents, but no values in the field
+    // NOTE: this represents the current state of what is returned, not the ultimate desired state.
+    client.testJQ(params(p, "q", "id:3"
+        , "json.facet", "{ sum1:'sum(${num_d})', sumsq1:'sumsq(${num_d})', avg1:'avg(${num_d})', min1:'min(${num_d})', max1:'max(${num_d})'" +
+            ", numwhere:'unique(${where_s})', unique_num_i:'unique(${num_i})', unique_num_d:'unique(${num_d})', unique_date:'unique(${date})'" +
+            ", where_hll:'hll(${where_s})', hll_num_i:'hll(${num_i})', hll_num_d:'hll(${num_d})', hll_date:'hll(${date})'" +
+            ", med:'percentile(${num_d},50)', perc:'percentile(${num_d},0,50.0,100)' }"
+        )
+        , "facets=={count:1 " +
+            ",sum1:0.0," +
+            " sumsq1:0.0," +
+            " avg1:0.0," +   // TODO: undesirable. omit?
+            " min1:'NaN'," + // TODO: undesirable. omit?
+            " max1:'NaN'," +
+            " numwhere:0," +
+            " unique_num_i:0," +
+            " unique_num_d:0," +
+            " unique_date:0," +
+            " where_hll:0," +
+            " hll_num_i:0," +
+            " hll_num_d:0," +
+            " hll_date:0" +
+            " }"
+    );
+
+
     //
     // tests on a multi-valued field with actual multiple values, just to ensure that we are
     // using a multi-valued method for the rest of the tests when appropriate.


[14/34] lucene-solr:jira/solr-9856: LUCENE-7609: Refactor expressions module to use DoubleValuesSource

Posted by ab...@apache.org.
LUCENE-7609: Refactor expressions module to use DoubleValuesSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8b055382
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8b055382
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8b055382

Branch: refs/heads/jira/solr-9856
Commit: 8b055382d6c88acaed9fe472a038c7ee6b35c016
Parents: da30f21
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Jan 5 13:15:19 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:05:06 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../ExpressionAggregationFacetsExample.java     |   2 +-
 lucene/expressions/build.xml                    |  11 --
 .../org/apache/lucene/expressions/Bindings.java |  14 +--
 .../apache/lucene/expressions/Expression.java   |  21 ++--
 .../expressions/ExpressionComparator.java       | 100 -----------------
 .../expressions/ExpressionFunctionValues.java   |  35 +++---
 .../lucene/expressions/ExpressionRescorer.java  |  33 ++++--
 .../lucene/expressions/ExpressionSortField.java |  77 -------------
 .../expressions/ExpressionValueSource.java      |  70 ++++++------
 .../lucene/expressions/ScoreFunctionValues.java |  46 --------
 .../lucene/expressions/ScoreValueSource.java    |  61 ----------
 .../lucene/expressions/SimpleBindings.java      |  30 +++--
 .../expressions/js/JavascriptCompiler.java      |  15 ++-
 .../apache/lucene/expressions/package-info.java |   5 +-
 .../lucene/expressions/TestDemoExpressions.java |  44 ++++++--
 .../expressions/TestExpressionSortField.java    |   2 +-
 .../expressions/TestExpressionValueSource.java  | 111 ++++++-------------
 .../expressions/js/TestCustomFunctions.java     |  20 ++--
 .../expressions/js/TestJavascriptFunction.java  |   2 +-
 .../js/TestJavascriptOperations.java            |   2 +-
 21 files changed, 197 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index fa5cc1c..c667040 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -74,6 +74,11 @@ API Changes
   grouping Collectors are renamed to remove the Abstract* prefix.
   (Alan Woodward, Martijn van Groningen)
 
+* LUCENE-7609: The expressions module now uses the DoubleValuesSource API, and 
+  no longer depends on the queries module.  Expression#getValueSource() is
+  replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
+  Grand)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
index fca7b6c..7f29b38 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/ExpressionAggregationFacetsExample.java
@@ -103,7 +103,7 @@ public class ExpressionAggregationFacetsExample {
     FacetsCollector.search(searcher, new MatchAllDocsQuery(), 10, fc);
 
     // Retrieve results
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getValueSource(bindings));
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, expr.getDoubleValuesSource(bindings));
     FacetResult result = facets.getTopChildren(10, "A");
     
     indexReader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/build.xml
----------------------------------------------------------------------
diff --git a/lucene/expressions/build.xml b/lucene/expressions/build.xml
index 1dddc6d..61ae64f 100644
--- a/lucene/expressions/build.xml
+++ b/lucene/expressions/build.xml
@@ -26,7 +26,6 @@
   <path id="classpath">
     <path refid="base.classpath"/>
     <fileset dir="lib"/>
-    <pathelement path="${queries.jar}"/>
   </path>
 
   <path id="test.classpath">
@@ -35,16 +34,6 @@
     <pathelement path="src/test-files"/>
   </path>
 
-  <target name="compile-core" depends="jar-queries,common.compile-core" />
-
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate" unless="javadocs-uptodate-${name}">
-    <invoke-module-javadoc>
-      <links>
-        <link href="../queries"/>
-      </links>
-    </invoke-module-javadoc>
-  </target>
-
   <target name="regenerate" depends="run-antlr"/>
 
   <target name="resolve-antlr" xmlns:ivy="antlib:org.apache.ivy.ant">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
index 5ec2edb..91a22f5 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Bindings.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
  * Binds variable names in expressions to actual data.
@@ -31,14 +31,10 @@ public abstract class Bindings {
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected Bindings() {}
-  
+
   /**
-   * Returns a ValueSource bound to the variable name.
+   * Returns a DoubleValuesSource bound to the variable name
    */
-  public abstract ValueSource getValueSource(String name);
-  
-  /** Returns a {@code ValueSource} over relevance scores */
-  protected final ValueSource getScoreValueSource() {
-    return new ScoreValueSource();
-  }
+  public abstract DoubleValuesSource getDoubleValuesSource(String name);
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
index 02be23b..c92e21b 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/Expression.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.expressions.js.JavascriptCompiler; // javadocs
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Rescorer;
 import org.apache.lucene.search.SortField;
 
@@ -63,26 +63,25 @@ public abstract class Expression {
   }
 
   /**
-   * Evaluates the expression for the given document.
+   * Evaluates the expression for the current document.
    *
-   * @param document <code>docId</code> of the document to compute a value for
-   * @param functionValues {@link FunctionValues} for each element of {@link #variables}.
+   * @param functionValues {@link DoubleValues} for each element of {@link #variables}.
    * @return The computed value of the expression for the given document.
    */
-  public abstract double evaluate(int document, FunctionValues[] functionValues);
+  public abstract double evaluate(DoubleValues[] functionValues);
 
   /**
-   * Get a value source which can compute the value of this expression in the context of the given bindings.
+   * Get a DoubleValuesSource which can compute the value of this expression in the context of the given bindings.
    * @param bindings Bindings to use for external values in this expression
-   * @return A value source which will evaluate this expression when used
+   * @return A DoubleValuesSource which will evaluate this expression when used
    */
-  public ValueSource getValueSource(Bindings bindings) {
+  public DoubleValuesSource getDoubleValuesSource(Bindings bindings) {
     return new ExpressionValueSource(bindings, this);
   }
   
   /** Get a sort field which can be used to rank documents by this expression. */
   public SortField getSortField(Bindings bindings, boolean reverse) {
-    return getValueSource(bindings).getSortField(reverse);
+    return getDoubleValuesSource(bindings).getSortField(reverse);
   }
 
   /** Get a {@link Rescorer}, to rescore first-pass hits

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
deleted file mode 100644
index eabf6dd..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionComparator.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.expressions;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-
-/** A custom comparator for sorting documents by an expression */
-class ExpressionComparator extends FieldComparator<Double> implements LeafFieldComparator {
-  private final double[] values;
-  private double bottom;
-  private double topValue;
-  
-  private ValueSource source;
-  private FunctionValues scores;
-  private LeafReaderContext readerContext;
-  
-  public ExpressionComparator(ValueSource source, int numHits) {
-    values = new double[numHits];
-    this.source = source;
-  }
-  
-  // TODO: change FieldComparator.setScorer to throw IOException and remove this try-catch
-  @Override
-  public void setScorer(Scorer scorer) {
-    // TODO: might be cleaner to lazy-init 'source' and set scorer after?
-    assert readerContext != null;
-    try {
-      Map<String,Object> context = new HashMap<>();
-      assert scorer != null;
-      context.put("scorer", scorer);
-      scores = source.getValues(context, readerContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-  @Override
-  public int compare(int slot1, int slot2) {
-    return Double.compare(values[slot1], values[slot2]);
-  }
-  
-  @Override
-  public void setBottom(int slot) {
-    bottom = values[slot];
-  }
-  
-  @Override
-  public void setTopValue(Double value) {
-    topValue = value.doubleValue();
-  }
-  
-  @Override
-  public int compareBottom(int doc) throws IOException {
-    return Double.compare(bottom, scores.doubleVal(doc));
-  }
-  
-  @Override
-  public void copy(int slot, int doc) throws IOException {
-    values[slot] = scores.doubleVal(doc);
-  }
-  
-  @Override
-  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-    this.readerContext = context;
-    return this;
-  }
-  
-  @Override
-  public Double value(int slot) {
-    return Double.valueOf(values[slot]);
-  }
-  
-  @Override
-  public int compareTop(int doc) throws IOException {
-    return Double.compare(topValue, scores.doubleVal(doc));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
index ad195cd..2e6f7c4 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionFunctionValues.java
@@ -16,20 +16,16 @@
  */
 package org.apache.lucene.expressions;
 
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import java.io.IOException;
 
-/** A {@link FunctionValues} which evaluates an expression */
-class ExpressionFunctionValues extends DoubleDocValues {
+import org.apache.lucene.search.DoubleValues;
+
+/** A {@link DoubleValues} which evaluates an expression */
+class ExpressionFunctionValues extends DoubleValues {
   final Expression expression;
-  final FunctionValues[] functionValues;
-  
-  int currentDocument = -1;
-  double currentValue;
+  final DoubleValues[] functionValues;
   
-  ExpressionFunctionValues(ValueSource parent, Expression expression, FunctionValues[] functionValues) {
-    super(parent);
+  ExpressionFunctionValues(Expression expression, DoubleValues[] functionValues) {
     if (expression == null) {
       throw new NullPointerException();
     }
@@ -39,14 +35,17 @@ class ExpressionFunctionValues extends DoubleDocValues {
     this.expression = expression;
     this.functionValues = functionValues;
   }
-  
+
   @Override
-  public double doubleVal(int document) {
-    if (currentDocument != document) {
-      currentDocument = document;
-      currentValue = expression.evaluate(document, functionValues);
+  public boolean advanceExact(int doc) throws IOException {
+    for (DoubleValues v : functionValues) {
+      v.advanceExact(doc);
     }
-    
-    return currentValue;
+    return true;
+  }
+  
+  @Override
+  public double doubleValue() {
+    return expression.evaluate(functionValues);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
index 33e8428..e3e7a4e 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionRescorer.java
@@ -20,13 +20,11 @@ package org.apache.lucene.expressions;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Rescorer;
@@ -49,7 +47,7 @@ class ExpressionRescorer extends SortRescorer {
   private final Expression expression;
   private final Bindings bindings;
 
-  /** Uses the provided {@link ValueSource} to assign second
+  /** Uses the provided {@link Expression} to assign second
    *  pass scores. */
   public ExpressionRescorer(Expression expression, Bindings bindings) {
     super(new Sort(expression.getSortField(bindings, true)));
@@ -57,6 +55,21 @@ class ExpressionRescorer extends SortRescorer {
     this.bindings = bindings;
   }
 
+  private static DoubleValues scores(int doc, float score) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return score;
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        assert doc == target;
+        return true;
+      }
+    };
+  }
+
   @Override
   public Explanation explain(IndexSearcher searcher, Explanation firstPassExplanation, int docID) throws IOException {
     Explanation superExpl = super.explain(searcher, firstPassExplanation, docID);
@@ -65,18 +78,14 @@ class ExpressionRescorer extends SortRescorer {
     int subReader = ReaderUtil.subIndex(docID, leaves);
     LeafReaderContext readerContext = leaves.get(subReader);
     int docIDInSegment = docID - readerContext.docBase;
-    Map<String,Object> context = new HashMap<>();
-
-    FakeScorer fakeScorer = new FakeScorer();
-    fakeScorer.score = firstPassExplanation.getValue();
-    fakeScorer.doc = docIDInSegment;
 
-    context.put("scorer", fakeScorer);
+    DoubleValues scores = scores(docIDInSegment, firstPassExplanation.getValue());
 
     List<Explanation> subs = new ArrayList<>(Arrays.asList(superExpl.getDetails()));
     for(String variable : expression.variables) {
-      subs.add(Explanation.match((float) bindings.getValueSource(variable).getValues(context, readerContext).doubleVal(docIDInSegment),
-                                       "variable \"" + variable + "\""));
+      DoubleValues dv = bindings.getDoubleValuesSource(variable).getValues(readerContext, scores);
+      if (dv.advanceExact(docIDInSegment))
+        subs.add(Explanation.match((float) dv.doubleValue(), "variable \"" + variable + "\""));
     }
 
     return Explanation.match(superExpl.getValue(), superExpl.getDescription(), subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
deleted file mode 100644
index 2b39834..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionSortField.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.SortField;
-
-/** A {@link SortField} which sorts documents by the evaluated value of an expression for each document */
-class ExpressionSortField extends SortField {
-  private final ExpressionValueSource source;
-
-  ExpressionSortField(String name, ExpressionValueSource source, boolean reverse) {
-    super(name, Type.CUSTOM, reverse);
-    this.source = source;
-  }
-  
-  @Override
-  public FieldComparator<?> getComparator(final int numHits, final int sortPos) throws IOException {
-    return new ExpressionComparator(source, numHits);
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + ((source == null) ? 0 : source.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (!super.equals(obj)) return false;
-    if (getClass() != obj.getClass()) return false;
-    ExpressionSortField other = (ExpressionSortField) obj;
-    if (source == null) {
-      if (other.source != null) return false;
-    } else if (!source.equals(other.source)) return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    
-    buffer.append("<expr \"");
-    buffer.append(getField());
-    buffer.append("\">");
-    
-    if (getReverse()) {
-      buffer.append('!');
-    }
-
-    return buffer.toString();
-  }
-
-  @Override
-  public boolean needsScores() {
-    return source.needsScores();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
index fcba455..7842de9 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
@@ -20,76 +20,77 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 
 /**
- * A {@link ValueSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
+ * A {@link DoubleValuesSource} which evaluates a {@link Expression} given the context of an {@link Bindings}.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
-final class ExpressionValueSource extends ValueSource {
-  final ValueSource variables[];
+final class ExpressionValueSource extends DoubleValuesSource {
+  final DoubleValuesSource variables[];
   final Expression expression;
   final boolean needsScores;
 
   ExpressionValueSource(Bindings bindings, Expression expression) {
     if (bindings == null) throw new NullPointerException();
-    if (expression == null) throw new NullPointerException();
-    this.expression = expression;
-    variables = new ValueSource[expression.variables.length];
+    this.expression = Objects.requireNonNull(expression);
+    variables = new DoubleValuesSource[expression.variables.length];
     boolean needsScores = false;
     for (int i = 0; i < variables.length; i++) {
-      ValueSource source = bindings.getValueSource(expression.variables[i]);
-      if (source instanceof ScoreValueSource) {
-        needsScores = true;
-      } else if (source instanceof ExpressionValueSource) {
-        if (((ExpressionValueSource)source).needsScores()) {
-          needsScores = true;
-        }
-      } else if (source == null) {
+      DoubleValuesSource source = bindings.getDoubleValuesSource(expression.variables[i]);
+      if (source == null) {
         throw new RuntimeException("Internal error. Variable (" + expression.variables[i] + ") does not exist.");
       }
+      needsScores |= source.needsScores();
       variables[i] = source;
     }
     this.needsScores = needsScores;
   }
 
   @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Map<String, FunctionValues> valuesCache = (Map<String, FunctionValues>)context.get("valuesCache");
-    if (valuesCache == null) {
-      valuesCache = new HashMap<>();
-      context = new HashMap(context);
-      context.put("valuesCache", valuesCache);
-    }
-    FunctionValues[] externalValues = new FunctionValues[expression.variables.length];
+  public DoubleValues getValues(LeafReaderContext readerContext, DoubleValues scores) throws IOException {
+    Map<String, DoubleValues> valuesCache = new HashMap<>();
+    DoubleValues[] externalValues = new DoubleValues[expression.variables.length];
 
     for (int i = 0; i < variables.length; ++i) {
       String externalName = expression.variables[i];
-      FunctionValues values = valuesCache.get(externalName);
+      DoubleValues values = valuesCache.get(externalName);
       if (values == null) {
-        values = variables[i].getValues(context, readerContext);
+        values = variables[i].getValues(readerContext, scores);
         if (values == null) {
           throw new RuntimeException("Internal error. External (" + externalName + ") does not exist.");
         }
         valuesCache.put(externalName, values);
       }
-      externalValues[i] = values;
+      externalValues[i] = zeroWhenUnpositioned(values);
     }
 
-    return new ExpressionFunctionValues(this, expression, externalValues);
+    return new ExpressionFunctionValues(expression, externalValues);
   }
 
-  @Override
-  public SortField getSortField(boolean reverse) {
-    return new ExpressionSortField(expression.sourceText, this, reverse);
+  private static DoubleValues zeroWhenUnpositioned(DoubleValues in) {
+    return new DoubleValues() {
+
+      boolean positioned = false;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return positioned ? in.doubleValue() : 0;
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        return positioned = in.advanceExact(doc);
+      }
+    };
   }
 
   @Override
-  public String description() {
+  public String toString() {
     return "expr(" + expression.sourceText + ")";
   }
   
@@ -132,7 +133,8 @@ final class ExpressionValueSource extends ValueSource {
     return true;
   }
 
-  boolean needsScores() {
+  @Override
+  public boolean needsScores() {
     return needsScores;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
deleted file mode 100644
index e310c06..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreFunctionValues.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.expressions;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.search.Scorer;
-
-/**
- * A utility class to allow expressions to access the score as a {@link FunctionValues}.
- */
-class ScoreFunctionValues extends DoubleDocValues {
-  final Scorer scorer;
-
-  ScoreFunctionValues(ValueSource parent, Scorer scorer) {
-    super(parent);
-    this.scorer = scorer;
-  }
-  
-  @Override
-  public double doubleVal(int document) {
-    try {
-      assert document == scorer.docID();
-      return scorer.score();
-    } catch (IOException exception) {
-      throw new RuntimeException(exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
deleted file mode 100644
index ea1669c..0000000
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ScoreValueSource.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.expressions;
-
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Scorer;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * A {@link ValueSource} which uses the {@link Scorer} passed through
- * the context map by {@link ExpressionComparator}.
- */
-@SuppressWarnings({"rawtypes"})
-class ScoreValueSource extends ValueSource {
-
-  /**
-   * <code>context</code> must contain a key "scorer" which is a {@link Scorer}.
-   */
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    Scorer v = (Scorer) context.get("scorer");
-    if (v == null) {
-      throw new IllegalStateException("Expressions referencing the score can only be used for sorting");
-    }
-    return new ScoreFunctionValues(this, v);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return o == this;
-  }
-
-  @Override
-  public int hashCode() {
-    return System.identityHashCode(this);
-  }
-
-  @Override
-  public String description() {
-    return "score()";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
index 1c11cb2..6276055 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/SimpleBindings.java
@@ -20,11 +20,7 @@ package org.apache.lucene.expressions;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 
 /**
@@ -64,9 +60,9 @@ public final class SimpleBindings extends Bindings {
   }
 
   /**
-   * Bind a {@link ValueSource} directly to the given name.
+   * Bind a {@link DoubleValuesSource} directly to the given name.
    */
-  public void add(String name, ValueSource source) { map.put(name, source); }
+  public void add(String name, DoubleValuesSource source) { map.put(name, source); }
   
   /** 
    * Adds an Expression to the bindings.
@@ -78,27 +74,27 @@ public final class SimpleBindings extends Bindings {
   }
   
   @Override
-  public ValueSource getValueSource(String name) {
+  public DoubleValuesSource getDoubleValuesSource(String name) {
     Object o = map.get(name);
     if (o == null) {
       throw new IllegalArgumentException("Invalid reference '" + name + "'");
     } else if (o instanceof Expression) {
-      return ((Expression)o).getValueSource(this);
-    } else if (o instanceof ValueSource) {
-      return ((ValueSource)o);
+      return ((Expression)o).getDoubleValuesSource(this);
+    } else if (o instanceof DoubleValuesSource) {
+      return ((DoubleValuesSource) o);
     }
     SortField field = (SortField) o;
     switch(field.getType()) {
       case INT:
-        return new IntFieldSource(field.getField());
+        return DoubleValuesSource.fromIntField(field.getField());
       case LONG:
-        return new LongFieldSource(field.getField());
+        return DoubleValuesSource.fromLongField(field.getField());
       case FLOAT:
-        return new FloatFieldSource(field.getField());
+        return DoubleValuesSource.fromFloatField(field.getField());
       case DOUBLE:
-        return new DoubleFieldSource(field.getField());
+        return DoubleValuesSource.fromDoubleField(field.getField());
       case SCORE:
-        return getScoreValueSource();
+        return DoubleValuesSource.SCORES;
       default:
         throw new UnsupportedOperationException(); 
     }
@@ -113,7 +109,7 @@ public final class SimpleBindings extends Bindings {
       if (o instanceof Expression) {
         Expression expr = (Expression) o;
         try {
-          expr.getValueSource(this);
+          expr.getDoubleValuesSource(this);
         } catch (StackOverflowError e) {
           throw new IllegalArgumentException("Recursion Error: Cycle detected originating in (" + expr.sourceText + ")");
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
index 13174bc..87e41c0 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
@@ -39,7 +39,7 @@ import org.antlr.v4.runtime.CommonTokenStream;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.js.JavascriptParser.ExpressionContext;
-import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.util.IOUtils;
 import org.objectweb.asm.ClassWriter;
 import org.objectweb.asm.Label;
@@ -93,13 +93,13 @@ public final class JavascriptCompiler {
   private static final String COMPILED_EXPRESSION_INTERNAL = COMPILED_EXPRESSION_CLASS.replace('.', '/');
   
   static final Type EXPRESSION_TYPE = Type.getType(Expression.class);
-  static final Type FUNCTION_VALUES_TYPE = Type.getType(FunctionValues.class);
+  static final Type FUNCTION_VALUES_TYPE = Type.getType(DoubleValues.class);
 
   private static final org.objectweb.asm.commons.Method
     EXPRESSION_CTOR = getAsmMethod(void.class, "<init>", String.class, String[].class),
-    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", int.class, FunctionValues[].class);
+    EVALUATE_METHOD = getAsmMethod(double.class, "evaluate", DoubleValues[].class);
 
-  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleVal", int.class);
+  static final org.objectweb.asm.commons.Method DOUBLE_VAL_METHOD = getAsmMethod(double.class, "doubleValue");
   
   /** create an ASM Method object from return type, method name, and parameters. */
   private static org.objectweb.asm.commons.Method getAsmMethod(Class<?> rtype, String name, Class<?>... ptypes) {
@@ -155,8 +155,8 @@ public final class JavascriptCompiler {
    */
   @SuppressWarnings({"unused", "null"})
   private static void unusedTestCompile() throws IOException {
-    FunctionValues f = null;
-    double ret = f.doubleVal(2);
+    DoubleValues f = null;
+    double ret = f.doubleValue();
   }
   
   /**
@@ -325,10 +325,9 @@ public final class JavascriptCompiler {
             externalsMap.put(text, index);
           }
 
-          gen.loadArg(1);
+          gen.loadArg(0);
           gen.push(index);
           gen.arrayLoad(FUNCTION_VALUES_TYPE);
-          gen.loadArg(0);
           gen.invokeVirtual(FUNCTION_VALUES_TYPE, DOUBLE_VAL_METHOD);
           gen.cast(Type.DOUBLE_TYPE, typeStack.peek());
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
index 62a519b..4a1eadf 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/package-info.java
@@ -25,11 +25,12 @@
  * 
  * <p>
  * {@link org.apache.lucene.expressions.Bindings} - abstraction for binding external variables
- * to a way to get a value for those variables for a particular document (ValueSource).
+ * to a way to get a value for those variables for a particular document (DoubleValuesSource).
  * </p>
  * 
  * <p>
- * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which provide easy ways to bind sort fields and other expressions to external variables
+ * {@link org.apache.lucene.expressions.SimpleBindings} - default implementation of bindings which
+ * provide easy ways to bind sort fields and other expressions to external variables
  * </p>
  */
 package org.apache.lucene.expressions;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
index 01b3394..d76ef1c 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestDemoExpressions.java
@@ -16,18 +16,20 @@
  */
 package org.apache.lucene.expressions;
 
+import java.io.IOException;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.expressions.js.VariableContext;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -39,9 +41,9 @@ import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 import static org.apache.lucene.expressions.js.VariableContext.Type.MEMBER;
 import static org.apache.lucene.expressions.js.VariableContext.Type.STR_INDEX;
-import static org.apache.lucene.expressions.js.VariableContext.Type.INT_INDEX;
 
 
 /** simple demo of using expressions */
@@ -236,7 +238,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
   public void testStaticExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc[\"popularity\"].value");
     SimpleBindings bindings = new SimpleBindings();
-    bindings.add("doc['popularity'].value", new IntFieldSource("popularity"));
+    bindings.add("doc['popularity'].value", DoubleValuesSource.fromIntField("popularity"));
     Sort sort = new Sort(popularity.getSortField(bindings, true));
     TopFieldDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
 
@@ -250,6 +252,30 @@ public class  TestDemoExpressions extends LuceneTestCase {
     assertEquals(2D, (Double)d.fields[0], 1E-4);
   }
 
+  private static DoubleValuesSource constant(double value) {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          @Override
+          public double doubleValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   public void testDynamicExtendedVariableExample() throws Exception {
     Expression popularity = JavascriptCompiler.compile("doc['popularity'].value + magicarray[0] + fourtytwo");
 
@@ -258,7 +284,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
     // filled in with proper error messages for a real use case.
     Bindings bindings = new Bindings() {
       @Override
-      public ValueSource getValueSource(String name) {
+      public DoubleValuesSource getDoubleValuesSource(String name) {
         VariableContext[] var = VariableContext.parse(name);
         assert var[0].type == MEMBER;
         String base = var[0].text;
@@ -266,7 +292,7 @@ public class  TestDemoExpressions extends LuceneTestCase {
           if (var.length > 1 && var[1].type == STR_INDEX) {
             String field = var[1].text;
             if (var.length > 2 && var[2].type == MEMBER && var[2].text.equals("value")) {
-              return new IntFieldSource(field);
+              return DoubleValuesSource.fromIntField(field);
             } else {
               fail("member: " + var[2].text);// error case, non/missing "value" member access
             }
@@ -275,12 +301,12 @@ public class  TestDemoExpressions extends LuceneTestCase {
           }
         } else if (base.equals("magicarray")) {
           if (var.length > 1 && var[1].type == INT_INDEX) {
-            return new DoubleConstValueSource(2048);
+            return constant(2048);
           } else {
             fail();// error case, magic array isn't an array
           }
         } else if (base.equals("fourtytwo")) {
-          return new DoubleConstValueSource(42);
+          return constant(42);
         } else {
           fail();// error case (variable doesn't exist)
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
index ec6ea11..73e7b8b 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionSortField.java
@@ -31,7 +31,7 @@ public class TestExpressionSortField extends LuceneTestCase {
     bindings.add(new SortField("popularity", SortField.Type.INT));
     
     SortField sf = expr.getSortField(bindings, true);
-    assertEquals("<expr \"sqrt(_score) + ln(popularity)\">!", sf.toString());
+    assertEquals("<expr(sqrt(_score) + ln(popularity))>!", sf.toString());
   }
   
   public void testEquals() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
index eeb3c9c..39217d8 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/TestExpressionValueSource.java
@@ -17,21 +17,17 @@
 package org.apache.lucene.expressions;
 
 
-import java.util.HashMap;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.ValueSourceScorer;
-import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -47,7 +43,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
-    
+
     Document doc = new Document();
     doc.add(newStringField("id", "1", Field.Store.YES));
     doc.add(newTextField("body", "some contents and more contents", Field.Store.NO));
@@ -58,6 +54,7 @@ public class TestExpressionValueSource extends LuceneTestCase {
     doc.add(newStringField("id", "2", Field.Store.YES));
     doc.add(newTextField("body", "another document with different contents", Field.Store.NO));
     doc.add(new NumericDocValuesField("popularity", 20));
+    doc.add(new NumericDocValuesField("count", 1));
     iw.addDocument(doc);
     
     doc = new Document();
@@ -77,81 +74,34 @@ public class TestExpressionValueSource extends LuceneTestCase {
     dir.close();
     super.tearDown();
   }
-  
-  public void testTypes() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
-    SimpleBindings bindings = new SimpleBindings();
-    bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
-    assertEquals(1, reader.leaves().size());
-    LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    assertEquals(10, values.doubleVal(0), 0);
-    assertEquals(10, values.floatVal(0), 0);
-    assertEquals(10, values.longVal(0));
-    assertEquals(10, values.intVal(0));
-    assertEquals(10, values.shortVal(0));
-    assertEquals(10, values.byteVal(0));
-    assertEquals("10.0", values.strVal(0));
-    assertEquals(new Double(10), values.objectVal(0));
-    
-    assertEquals(40, values.doubleVal(1), 0);
-    assertEquals(40, values.floatVal(1), 0);
-    assertEquals(40, values.longVal(1));
-    assertEquals(40, values.intVal(1));
-    assertEquals(40, values.shortVal(1));
-    assertEquals(40, values.byteVal(1));
-    assertEquals("40.0", values.strVal(1));
-    assertEquals(new Double(40), values.objectVal(1));
-    
-    assertEquals(4, values.doubleVal(2), 0);
-    assertEquals(4, values.floatVal(2), 0);
-    assertEquals(4, values.longVal(2));
-    assertEquals(4, values.intVal(2));
-    assertEquals(4, values.shortVal(2));
-    assertEquals(4, values.byteVal(2));
-    assertEquals("4.0", values.strVal(2));
-    assertEquals(new Double(4), values.objectVal(2));    
-  }
-  
-  public void testRangeScorer() throws Exception {
-    Expression expr = JavascriptCompiler.compile("2*popularity");
+
+  public void testDoubleValuesSourceTypes() throws Exception {
+    Expression expr = JavascriptCompiler.compile("2*popularity + count");
     SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("popularity", SortField.Type.LONG));
-    ValueSource vs = expr.getValueSource(bindings);
-    
+    bindings.add(new SortField("count", SortField.Type.LONG));
+    DoubleValuesSource vs = expr.getDoubleValuesSource(bindings);
+
     assertEquals(1, reader.leaves().size());
     LeafReaderContext leaf = reader.leaves().get(0);
-    FunctionValues values = vs.getValues(new HashMap<String,Object>(), leaf);
-    
-    // everything
-    ValueSourceScorer scorer = values.getRangeScorer(leaf, "4", "40", true, true);
-    DocIdSetIterator iter = scorer.iterator();
-    assertEquals(-1, iter.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(1, iter.nextDoc());
-    assertEquals(2, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    DoubleValues values = vs.getValues(leaf, null);
 
-    // just the first doc
-    values = vs.getValues(new HashMap<String,Object>(), leaf);
-    scorer = values.getRangeScorer(leaf, "4", "40", false, false);
-    iter = scorer.iterator();
-    assertEquals(-1, scorer.docID());
-    assertEquals(0, iter.nextDoc());
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, iter.nextDoc());
+    assertTrue(values.advanceExact(0));
+    assertEquals(10, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(1));
+    assertEquals(41, values.doubleValue(), 0);
+    assertTrue(values.advanceExact(2));
+    assertEquals(4, values.doubleValue(), 0);
   }
-  
-  public void testEquals() throws Exception {
+
+  public void testDoubleValuesSourceEquals() throws Exception {
     Expression expr = JavascriptCompiler.compile("sqrt(a) + ln(b)");
-    
-    SimpleBindings bindings = new SimpleBindings();    
+
+    SimpleBindings bindings = new SimpleBindings();
     bindings.add(new SortField("a", SortField.Type.INT));
     bindings.add(new SortField("b", SortField.Type.INT));
-    
-    ValueSource vs1 = expr.getValueSource(bindings);
+
+    DoubleValuesSource vs1 = expr.getDoubleValuesSource(bindings);
     // same instance
     assertEquals(vs1, vs1);
     // null
@@ -159,20 +109,21 @@ public class TestExpressionValueSource extends LuceneTestCase {
     // other object
     assertFalse(vs1.equals("foobar"));
     // same bindings and expression instances
-    ValueSource vs2 = expr.getValueSource(bindings);
+    DoubleValuesSource vs2 = expr.getDoubleValuesSource(bindings);
     assertEquals(vs1.hashCode(), vs2.hashCode());
     assertEquals(vs1, vs2);
     // equiv bindings (different instance)
-    SimpleBindings bindings2 = new SimpleBindings();    
+    SimpleBindings bindings2 = new SimpleBindings();
     bindings2.add(new SortField("a", SortField.Type.INT));
     bindings2.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs3 = expr.getValueSource(bindings2);
+    DoubleValuesSource vs3 = expr.getDoubleValuesSource(bindings2);
     assertEquals(vs1, vs3);
     // different bindings (same names, different types)
-    SimpleBindings bindings3 = new SimpleBindings();    
+    SimpleBindings bindings3 = new SimpleBindings();
     bindings3.add(new SortField("a", SortField.Type.LONG));
-    bindings3.add(new SortField("b", SortField.Type.INT));
-    ValueSource vs4 = expr.getValueSource(bindings3);
+    bindings3.add(new SortField("b", SortField.Type.FLOAT));
+    DoubleValuesSource vs4 = expr.getDoubleValuesSource(bindings3);
     assertFalse(vs1.equals(vs4));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
index 7c46b05..9df5d74 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestCustomFunctions.java
@@ -50,7 +50,7 @@ public class TestCustomFunctions extends LuceneTestCase {
   public void testDefaultList() throws Exception {
     Map<String,Method> functions = JavascriptCompiler.DEFAULT_FUNCTIONS;
     Expression expr = JavascriptCompiler.compile("sqrt(20)", functions, getClass().getClassLoader());
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
   }
   
   public static double zeroArgMethod() { return 5; }
@@ -60,7 +60,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     Expression expr = JavascriptCompiler.compile("foo()", functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 
   public static double oneArgMethod(double arg1) { return 3 + arg1; }
@@ -70,7 +70,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo(3)", functions, getClass().getClassLoader());
-    assertEquals(6, expr.evaluate(0, null), DELTA);
+    assertEquals(6, expr.evaluate(null), DELTA);
   }
   
   public static double threeArgMethod(double arg1, double arg2, double arg3) { return arg1 + arg2 + arg3; }
@@ -80,7 +80,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> functions = new HashMap<>();
     functions.put("foo", getClass().getMethod("threeArgMethod", double.class, double.class, double.class));
     Expression expr = JavascriptCompiler.compile("foo(3, 4, 5)", functions, getClass().getClassLoader());
-    assertEquals(12, expr.evaluate(0, null), DELTA);
+    assertEquals(12, expr.evaluate(null), DELTA);
   }
   
   /** tests a map with 2 functions */
@@ -89,7 +89,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo", getClass().getMethod("zeroArgMethod"));
     functions.put("bar", getClass().getMethod("oneArgMethod", double.class));
     Expression expr = JavascriptCompiler.compile("foo() + bar(3)", functions, getClass().getClassLoader());
-    assertEquals(11, expr.evaluate(0, null), DELTA);
+    assertEquals(11, expr.evaluate(null), DELTA);
   }
 
   /** tests invalid methods that are not allowed to become variables to be mapped */
@@ -220,7 +220,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     
     // this should pass:
     Expression expr = JavascriptCompiler.compile("bar()", functions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -232,9 +232,9 @@ public class TestCustomFunctions extends LuceneTestCase {
     Map<String,Method> mixedFunctions = new HashMap<>(JavascriptCompiler.DEFAULT_FUNCTIONS);
     mixedFunctions.putAll(functions);
     expr = JavascriptCompiler.compile("bar()", mixedFunctions, childLoader);
-    assertEquals(2.0, expr.evaluate(0, null), DELTA);
+    assertEquals(2.0, expr.evaluate(null), DELTA);
     expr = JavascriptCompiler.compile("sqrt(20)", mixedFunctions, childLoader);
-    assertEquals(Math.sqrt(20), expr.evaluate(0, null), DELTA);
+    assertEquals(Math.sqrt(20), expr.evaluate(null), DELTA);
     
     // use our classloader, not the foreign one, which should fail!
     expected = expectThrows(IllegalArgumentException.class, () -> {
@@ -256,7 +256,7 @@ public class TestCustomFunctions extends LuceneTestCase {
     String source = "3 * foo() / 5";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
     ArithmeticException expected = expectThrows(ArithmeticException.class, () -> {
-      expr.evaluate(0, null);
+      expr.evaluate(null);
     });
     assertEquals(MESSAGE, expected.getMessage());
     StringWriter sw = new StringWriter();
@@ -272,6 +272,6 @@ public class TestCustomFunctions extends LuceneTestCase {
     functions.put("foo.bar", getClass().getMethod("zeroArgMethod"));
     String source = "foo.bar()";
     Expression expr = JavascriptCompiler.compile(source, functions, getClass().getClassLoader());
-    assertEquals(5, expr.evaluate(0, null), DELTA);
+    assertEquals(5, expr.evaluate(null), DELTA);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
index 81362a6..ed68a5f 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptFunction.java
@@ -24,7 +24,7 @@ public class TestJavascriptFunction extends LuceneTestCase {
   
   private void assertEvaluatesTo(String expression, double expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    double actual = evaluator.evaluate(0, null);
+    double actual = evaluator.evaluate(null);
     assertEquals(expected, actual, DELTA);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8b055382/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
index 82d5056..fd098c5 100644
--- a/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
+++ b/lucene/expressions/src/test/org/apache/lucene/expressions/js/TestJavascriptOperations.java
@@ -22,7 +22,7 @@ import org.apache.lucene.util.LuceneTestCase;
 public class TestJavascriptOperations extends LuceneTestCase {
   private void assertEvaluatesTo(String expression, long expected) throws Exception {
     Expression evaluator = JavascriptCompiler.compile(expression);
-    long actual = (long)evaluator.evaluate(0, null);
+    long actual = (long)evaluator.evaluate(null);
     assertEquals(expected, actual);
   }
   


[22/34] lucene-solr:jira/solr-9856: SOLR-9944: Map the nodes function name to the GatherNodesStream

Posted by ab...@apache.org.
SOLR-9944: Map the nodes function name to the GatherNodesStream


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

Branch: refs/heads/jira/solr-9856
Commit: aae4217abc09163837597bf761f21d8019091216
Parents: d817fd4
Author: Joel Bernstein <jb...@apache.org>
Authored: Sat Jan 7 22:17:17 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sat Jan 7 22:17:44 2017 -0500

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/handler/GraphHandler.java      | 1 +
 solr/core/src/java/org/apache/solr/handler/StreamHandler.java     | 1 +
 .../apache/solr/client/solrj/io/graph/GraphExpressionTest.java    | 3 ++-
 3 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index 5e5934f..3b52154 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -117,6 +117,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
         .withFunctionName("topic", TopicStream.class)
         .withFunctionName("shortestPath", ShortestPathStream.class)
         .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
         .withFunctionName("sort", SortStream.class)
         .withFunctionName("scoreNodes", ScoreNodesStream.class)
         .withFunctionName("random", RandomStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 4b319f9..98486b8 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -142,6 +142,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("daemon", DaemonStream.class)
       .withFunctionName("shortestPath", ShortestPathStream.class)
       .withFunctionName("gatherNodes", GatherNodesStream.class)
+      .withFunctionName("nodes", GatherNodesStream.class)
       .withFunctionName("select", SelectStream.class)
       .withFunctionName("scoreNodes", ScoreNodesStream.class)
       .withFunctionName("model", ModelStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
index d6fc514..cf07058 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
@@ -270,6 +270,7 @@ public class GraphExpressionTest extends SolrCloudTestCase {
     StreamFactory factory = new StreamFactory()
         .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
         .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
         .withFunctionName("search", CloudSolrStream.class)
         .withFunctionName("count", CountMetric.class)
         .withFunctionName("avg", MeanMetric.class)
@@ -277,7 +278,7 @@ public class GraphExpressionTest extends SolrCloudTestCase {
         .withFunctionName("min", MinMetric.class)
         .withFunctionName("max", MaxMetric.class);
 
-    String expr = "gatherNodes(collection1, " +
+    String expr = "nodes(collection1, " +
         "walk=\"product1->product_s\"," +
         "gather=\"basket_s\")";
 


[08/34] lucene-solr:jira/solr-9856: LUCENE-7576: AutomatonTermsEnum ctor should also insist on a NORMAL CompiledAutomaton in

Posted by ab...@apache.org.
LUCENE-7576: AutomatonTermsEnum ctor should also insist on a NORMAL CompiledAutomaton in


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

Branch: refs/heads/jira/solr-9856
Commit: ebb5c7e6768c03c83be4aa3abdab22e16cb67c2c
Parents: cd4f908
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Jan 6 14:50:01 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Jan 6 14:50:01 2017 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/index/AutomatonTermsEnum.java | 3 +++
 .../core/src/test/org/apache/lucene/index/TestTermsEnum.java | 8 ++++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ebb5c7e6/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
index 8aa10ec..411a810 100644
--- a/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
@@ -76,6 +76,9 @@ public class AutomatonTermsEnum extends FilteredTermsEnum {
    */
   public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) {
     super(tenum);
+    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+    }
     this.finite = compiled.finite;
     this.runAutomaton = compiled.runAutomaton;
     assert this.runAutomaton != null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ebb5c7e6/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
index a388d42..d2df59f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
@@ -1016,4 +1016,12 @@ public class TestTermsEnum extends LuceneTestCase {
     w.close();
     d.close();
   }
+
+  // LUCENE-7576
+  public void testInvalidAutomatonTermsEnum() throws Exception {
+    expectThrows(IllegalArgumentException.class,
+                 () -> {
+                   new AutomatonTermsEnum(TermsEnum.EMPTY, new CompiledAutomaton(Automata.makeString("foo")));
+                 });
+  }
 }


[25/34] lucene-solr:jira/solr-9856: SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter

Posted by ab...@apache.org.
SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/570880d3
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/570880d3
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/570880d3

Branch: refs/heads/jira/solr-9856
Commit: 570880d3acb45c925e8dc77172e0725ab8ba07b8
Parents: ea49989
Author: David Smiley <ds...@apache.org>
Authored: Sat Jan 7 23:32:37 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Jan 7 23:32:37 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  5 +++-
 .../solr/highlight/UnifiedSolrHighlighter.java  | 12 +++++++++-
 .../highlight/TestUnifiedSolrHighlighter.java   | 24 ++++++++++++++++----
 .../solr/common/params/HighlightParams.java     |  2 +-
 4 files changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 899dcd3..c18381e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -109,7 +109,7 @@ Upgrade Notes
 
 * SOLR-9708: You are encouraged to try out the UnifiedHighlighter by setting hl.method=unified and report feedback. It
   might become the default in 7.0. It's more efficient/faster than the other highlighters, especially compared to the
-  original Highlighter. That said, some options aren't supported yet, notably hl.fragsize.
+  original Highlighter. That said, some options aren't supported yet.
   It will get more features in time, especially with your input. See HighlightParams.java
   for a listing of highlight parameters annotated with which highlighters use them.
   hl.useFastVectorHighlighter is now considered deprecated in lieu of hl.method=fastVector.
@@ -225,6 +225,9 @@ New Features
 * SOLR-7466: Enable leading wildcard in complexphrase query parser, optimize it with  ReversedWildcardFilterFactory 
   when it's provided (Mikhail Khludnev)
 
+* SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter to avoid snippets/Passages that are too small.
+  Defaults to 70. (David Smiley)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
index 910fa2b..5b59b85 100644
--- a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
@@ -30,6 +30,7 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
 import org.apache.lucene.search.uhighlight.DefaultPassageFormatter;
+import org.apache.lucene.search.uhighlight.LengthGoalBreakIterator;
 import org.apache.lucene.search.uhighlight.PassageFormatter;
 import org.apache.lucene.search.uhighlight.PassageScorer;
 import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
@@ -299,7 +300,16 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
       String variant = params.getFieldParam(field, HighlightParams.BS_VARIANT);
       Locale locale = parseLocale(language, country, variant);
       String type = params.getFieldParam(field, HighlightParams.BS_TYPE);
-      return parseBreakIterator(type, locale);
+      BreakIterator baseBI = parseBreakIterator(type, locale);
+
+      // Use a default fragsize the same as the regex Fragmenter (original Highlighter) since we're
+      //  both likely shooting for sentence-like patterns.
+      int fragsize = params.getFieldInt(field, HighlightParams.FRAGSIZE, LuceneRegexFragmenter.DEFAULT_FRAGMENT_SIZE);
+      if (fragsize <= 1 || baseBI instanceof WholeBreakIterator) { // no real minimum size
+        return baseBI;
+      }
+      return LengthGoalBreakIterator.createMinLength(baseBI, fragsize);
+      // TODO option for using createClosestToLength()
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index e2511be..2eb4ba3 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -78,7 +78,8 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "text2", "document one", "text3", "crappy document", "id", "101"));
     assertU(commit());
     assertQ("multiple snippets test",
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.snippets", "2", "hl.bs.type", "SENTENCE"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.snippets", "2", "hl.bs.type", "SENTENCE",
+            "hl.fragsize", "0"),
         "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/*)=2",
         "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>Document</em> snippet one. '",
         "//lst[@name='highlighting']/lst[@name='101']/arr/str[2]='<em>Document</em> snippet two.'");
@@ -202,21 +203,34 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier [document]'");
   }
   
-  public void testBreakIterator() {
+  public void testBreakIteratorWord() {
     assertQ("different breakiterator", 
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WORD"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WORD", "hl.fragsize", "-1"),
         "count(//lst[@name='highlighting']/*)=2",
         "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/str='<em>document</em>'",
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/str='<em>document</em>'");
   }
   
-  public void testBreakIterator2() {
+  public void testBreakIteratorWhole() {
     assertU(adoc("text", "Document one has a first sentence. Document two has a second sentence.", "id", "103"));
     assertU(commit());
     assertQ("different breakiterator", 
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WHOLE"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WHOLE", "hl.fragsize", "-1"),
         "//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em> one has a first sentence. <em>Document</em> two has a second sentence.'");
   }
+
+  public void testFragsize() {
+    // test default is 70... so make a sentence that is a little less (closer to 70 than end of text)
+    clearIndex();
+    assertU(adoc("id", "10", "text", "This is a sentence just under seventy chars in length blah blah. Next sentence is here."));
+    assertU(commit());
+    assertQ("default fragsize",
+        req("q", "text:seventy", "hl", "true"),
+        "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. Next sentence is here.'");
+    assertQ("smaller fragsize",
+        req("q", "text:seventy", "hl", "true", "hl.fragsize", "60"), // a bit smaller
+        "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. '");
+  }
   
   public void testEncoder() {
     assertU(adoc("text", "Document one has a first <i>sentence</i>.", "id", "103"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
index 917e9f5..997fc7e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
@@ -49,7 +49,7 @@ public interface HighlightParams {
   public static final String HIGHLIGHT_ALTERNATE = HIGHLIGHT+".highlightAlternate"; // OH, FVH
 
   // sizing
-  public static final String FRAGSIZE    = HIGHLIGHT+".fragsize"; // OH, FVH
+  public static final String FRAGSIZE    = HIGHLIGHT+".fragsize"; // OH, FVH, UH
   public static final String FRAGMENTER  = HIGHLIGHT+".fragmenter"; // OH
   public static final String INCREMENT   = HIGHLIGHT+".increment"; // OH
   public static final String REGEX       = "regex"; // OH


[20/34] lucene-solr:jira/solr-9856: LUCENE-7611: Remove unnecessary Exception wrapping from DocumentValueSourceDictionary

Posted by ab...@apache.org.
LUCENE-7611: Remove unnecessary Exception wrapping from DocumentValueSourceDictionary


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/67261d2f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/67261d2f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/67261d2f

Branch: refs/heads/jira/solr-9856
Commit: 67261d2fb515f255e05c138281ab6c6b71d66716
Parents: 8f4fee3
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 16:06:29 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 16:06:29 2017 +0000

----------------------------------------------------------------------
 .../suggest/DocumentValueSourceDictionary.java  | 21 +++++++-------------
 1 file changed, 7 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67261d2f/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 2291ac9..9356975 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -132,27 +132,20 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
      * by the <code>weightsValueSource</code>
      * */
     @Override
-    protected long getWeight(Document doc, int docId) {
+    protected long getWeight(Document doc, int docId) throws IOException {
       if (currentWeightValues == null) {
         return 0;
       }
       int subIndex = ReaderUtil.subIndex(docId, starts);
       if (subIndex != currentLeafIndex) {
         currentLeafIndex = subIndex;
-        try {
-          currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      try {
-        if (currentWeightValues.advanceExact(docId - starts[subIndex]))
-          return currentWeightValues.longValue();
-        else
-          return 0;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
+        currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
       }
+      if (currentWeightValues.advanceExact(docId - starts[subIndex]))
+        return currentWeightValues.longValue();
+      else
+        return 0;
+
     }
 
   }


[02/34] lucene-solr:jira/solr-9856: SOLR-9928: MetricsDirectoryFactory::renameWithOverwrite incorrectly calls super (Mike Drob via ab)

Posted by ab...@apache.org.
SOLR-9928: MetricsDirectoryFactory::renameWithOverwrite incorrectly calls super (Mike Drob via ab)


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

Branch: refs/heads/jira/solr-9856
Commit: e5264c48955165ac5c5b1aabba4748378d3f6fa9
Parents: e5d2884
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Jan 5 17:46:14 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Jan 5 17:46:14 2017 +0100

----------------------------------------------------------------------
 .../src/java/org/apache/solr/core/MetricsDirectoryFactory.java | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5264c48/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
index 62e82ac..8706c61 100644
--- a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
@@ -198,7 +198,11 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
-    super.renameWithOverwrite(dir, fileName, toName);
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory) dir).getDelegate();
+    }
+
+    in.renameWithOverwrite(dir, fileName, toName);
   }
 
   @Override


[04/34] lucene-solr:jira/solr-9856: SOLR-9911: Remove http group from example in change log

Posted by ab...@apache.org.
SOLR-9911: Remove http group from example in change log


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2cffa2e3
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2cffa2e3
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2cffa2e3

Branch: refs/heads/jira/solr-9856
Commit: 2cffa2e3e716e3ca3e9e3099f6c12ad157005e4c
Parents: 3eab1b4
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Jan 6 01:20:14 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Jan 6 01:20:14 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2cffa2e3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 098dfda..9de2d76 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -190,7 +190,7 @@ New Features
 
 * SOLR-9812: SOLR-9911: Added a new /admin/metrics API to return all metrics collected by Solr via API.
   API supports three optional parameters:
-  * 'group' (all,jvm,jetty,http,node,core),
+  * 'group' (all,jvm,jetty,node,core),
   * 'type' (all,counter,timer,gauge,histogram) both of which are multi-valued
   * 'prefix' that filters the returned metrics
   Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter


[30/34] lucene-solr:jira/solr-9856: SOLR-9902: Fix move impl.

Posted by ab...@apache.org.
SOLR-9902: Fix move impl.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8bc151d1
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8bc151d1
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8bc151d1

Branch: refs/heads/jira/solr-9856
Commit: 8bc151d1c61932dda26c682cf2281535f0c36058
Parents: 25290ab
Author: markrmiller <ma...@apache.org>
Authored: Sun Jan 8 10:22:42 2017 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Sun Jan 8 10:22:55 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                                  | 2 +-
 .../src/java/org/apache/solr/core/StandardDirectoryFactory.java   | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bc151d1/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c18381e..11151d7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -248,7 +248,7 @@ Optimizations
   resulting in less produced garbage and 5-7% better performance.
   (yonik)
 
-* SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation. (Mark Miller)
+* SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation. (Mark Miller, Mike Drob)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bc151d1/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index 37c15ed..1bc4914 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -131,13 +131,14 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     if (baseFromDir instanceof FSDirectory && baseToDir instanceof FSDirectory) {
   
       Path path1 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
-      Path path2 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
+      Path path2 = ((FSDirectory) baseToDir).getDirectory().toAbsolutePath();
       
       try {
         Files.move(path1.resolve(fileName), path2.resolve(fileName), StandardCopyOption.ATOMIC_MOVE);
       } catch (AtomicMoveNotSupportedException e) {
         Files.move(path1.resolve(fileName), path2.resolve(fileName));
       }
+      return;
     }
 
     super.move(fromDir, toDir, fileName, ioContext);


[09/34] lucene-solr:jira/solr-9856: SOLR-8542: expand 'Assemble training data' content in solr/contrib/ltr/README

Posted by ab...@apache.org.
SOLR-8542: expand 'Assemble training data' content in solr/contrib/ltr/README

(Diego Ceccarelli via Christine Poerschke in response to SOLR-9929 enquiry from Jeffery Yuan.)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/024c4031
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/024c4031
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/024c4031

Branch: refs/heads/jira/solr-9856
Commit: 024c4031e55a998b73288fd276e30ffd626f0b91
Parents: ebb5c7e
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Jan 6 20:52:21 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Jan 6 20:52:21 2017 +0000

----------------------------------------------------------------------
 solr/contrib/ltr/example/README.md        | 118 ++++++++++++++++++++-----
 solr/contrib/ltr/example/user_queries.txt |  12 +--
 2 files changed, 101 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/024c4031/solr/contrib/ltr/example/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/README.md b/solr/contrib/ltr/example/README.md
index 1363d5d..7842494 100644
--- a/solr/contrib/ltr/example/README.md
+++ b/solr/contrib/ltr/example/README.md
@@ -28,33 +28,105 @@ Please refer to the Solr Reference Guide's section on [Result Reranking](https:/
 
 4. Search and rerank the results using the trained model
 
-   http://localhost:8983/solr/techproducts/query?indent=on&q=test&wt=json&rq={!ltr%20model=exampleModel%20reRankDocs=25%20efi.user_query=%27test%27}&fl=price,score,name
+```
+http://localhost:8983/solr/techproducts/query?indent=on&q=test&wt=json&rq={!ltr%20model=exampleModel%20reRankDocs=25%20efi.user_query=%27test%27}&fl=price,score,name
+```
 
 # Assemble training data
 In order to train a learning to rank model you need training data. Training data is
-what "teaches" the model what the appropriate weight for each feature is. In general
+what *teaches* the model what the appropriate weight for each feature is. In general
 training data is a collection of queries with associated documents and what their ranking/score
 should be. As an example:
 ```
-hard drive|SP2514N|0.6666666|CLICK_LOGS
-hard drive|6H500F0|0.330082034|CLICK_LOGS
+hard drive|SP2514N        |0.6|CLICK_LOGS
+hard drive|6H500F0        |0.3|CLICK_LOGS
 hard drive|F8V7067-APL-KIT|0.0|CLICK_LOGS
-hard drive|IW-02|0.0|CLICK_LOGS
-
-ipod|MA147LL/A|1.0|EXPLICIT
-ipod|F8V7067-APL-KIT|0.25|EXPLICIT
-ipod|IW-02|0.25|EXPLICIT
-ipod|6H500F0|0.0|EXPLICIT
-```
-In this example the first column indicates the query, the second column indicates a unique id for that doc,
-the third column indicates the relative importance or relevance of that doc, and the fourth column indicates the source.
-There are 2 primary ways you might collect data for use with your machine learning algorithim. The first
-is to collect the clicks of your users given a specific query. There are many ways of preparing this data
-to train a model (http://www.cs.cornell.edu/people/tj/publications/joachims_etal_05a.pdf). The general idea
-is that if a user sees multiple documents and clicks the one lower down, that document should be scored higher
-than the one above it. The second way is explicitly through a crowdsourcing platform like Mechanical Turk or
-CrowdFlower. These platforms allow you to show human workers documents associated with a query and have them
-tell you what the correct ranking should be.
-
-At this point you'll need to collect feature vectors for each query document pair. You can use the information
-from the Extract features section above to do this. An example script has been included in example/train_and_upload_demo_model.py.
+hard drive|IW-02          |0.0|CLICK_LOGS
+
+ipod      |MA147LL/A      |1.0|HUMAN_JUDGEMENT
+ipod      |F8V7067-APL-KIT|0.5|HUMAN_JUDGEMENT
+ipod      |IW-02          |0.5|HUMAN_JUDGEMENT
+ipod      |6H500F0        |0.0|HUMAN_JUDGEMENT
+```
+The columns in the example represent:
+
+  1. the user query;
+
+  2. a unique id for a document in the response;
+
+  3. the a score representing the relevance of that document (not necessarily between zero and one);
+
+  4. the source, i.e., if the training record was produced by using interaction data (`CLICK_LOGS`) or by human judgements (`HUMAN_JUDGEMENT`).
+
+## How to produce training data
+
+You might collect data for use with your machine learning algorithm relying on:
+
+  * **Users Interactions**: given a specific query, it is possible to log all the users interactions (e.g., clicks, shares on social networks, send by email etc), and then use them as proxies for relevance;
+  * **Human Judgements**: A training dataset is produced by explicitly asking some judges to evaluate the relevance of a document given the query.
+
+### How to prepare training data from interaction data?
+
+There are many ways of preparing interaction data for training a model, and it is outside the scope of this readme to provide a complete review of all the techniques.  In the following we illustrate a simple way for obtaining training data from simple interaction data.
+
+Simple interaction data will be a log file generated by your application after it
+has talked to Solr. The log will contain two different types of record:
+
+  * **query**: when a user performs a query we have a record with `user-id, query, responses`,
+  where `responses` is a list of unique document ids returned for a query.
+
+**Example:**
+
+```
+diego, hard drive, [SP2514N,6H500F0,F8V7067-APL-KIT,IW-02]
+```
+
+  * **click**: when a user performs a click we have a record with `user-id, query, document-id, click`
+
+**Example:**
+```
+christine, hard drive, SP2154N
+diego    , hard drive, SP2154N
+michael  , hard drive, SP2154N
+joshua   , hard drive, IW-02
+```
+
+Given a log composed by records like these, a simple way to produce a training dataset is to group on the query field
+and then assign to each query a relevance score equal to the number of clicks:
+
+```
+hard drive|SP2514N        |3|CLICK_LOGS
+hard drive|IW-02          |1|CLICK_LOGS
+hard drive|6H500F0        |0|CLICK_LOGS
+hard drive|F8V7067-APL-KIT|0|CLICK_LOGS
+```
+
+This is a really trival way to generate a training dataset, and in many settings 
+it might not produce great results. Indeed, it is a well known fact that 
+clicks are *biased*: users tend to click  on the first
+result proposed for a query, also if it is not relevant. A click on a document in position
+five could be considered more important than a click on a document in position one, because
+the user took the effort to browse the results list until position five.
+
+Some approaches take into account the time spent on the clicked document (if the user
+spent only two seconds on the document and then clicked on other documents in the list,
+probably she did not intend to click that document).
+
+There are many papers proposing techniques for removing the bias, or for taking into account the click positions,
+a good survey is  [Click Models for Web Search](http://clickmodels.weebly.com/uploads/5/2/2/5/52257029/mc2015-clickmodels.pdf),
+by Chuklin, Markov and Rijke.
+
+### Prepare training data from human judgements
+
+Another way to get training data is asking human judges to label them.
+Producing human judgements is in general more expensive, but the quality of the
+dataset produced can be better than the one produced from interaction data.
+It is worth to note that human judgements can be produced also relying on a
+crowdsourcing platform, that allows a user to show human workers documents associated with a
+query and to get back relevance labels.
+Usually a human worker visualizes a query together with a list of results and the task
+consists in assigning a relevance label to each document (e.g., Perfect, Excellent, Good, Fair, Not relevant).
+Training data can then be obtained by translating the labels into numeric scores
+(e.g., Perfect = 4, Excellent = 3, Good = 2, Fair = 1, Not relevant = 0).
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/024c4031/solr/contrib/ltr/example/user_queries.txt
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/user_queries.txt b/solr/contrib/ltr/example/user_queries.txt
index a3a3455..5e820c7 100644
--- a/solr/contrib/ltr/example/user_queries.txt
+++ b/solr/contrib/ltr/example/user_queries.txt
@@ -1,8 +1,8 @@
-hard drive|SP2514N|0.6666666|CLICK_LOGS
-hard drive|6H500F0|0.330082034|CLICK_LOGS
+hard drive|SP2514N|0.6|CLICK_LOGS
+hard drive|6H500F0|0.3|CLICK_LOGS
 hard drive|F8V7067-APL-KIT|0.0|CLICK_LOGS
 hard drive|IW-02|0.0|CLICK_LOGS
-ipod|MA147LL/A|1.0|EXPLICIT
-ipod|F8V7067-APL-KIT|0.25|EXPLICIT
-ipod|IW-02|0.25|EXPLICIT
-ipod|6H500F0|0.0|EXPLICIT
+ipod|MA147LL/A|1.0|HUMAN_JUDGEMENT
+ipod|F8V7067-APL-KIT|0.5|HUMAN_JUDGEMENT
+ipod|IW-02|0.5|HUMAN_JUDGEMENT
+ipod|6H500F0|0.0|HUMAN_JUDGEMENT


[15/34] lucene-solr:jira/solr-9856: LUCENE-7610: Deprecate ValueSource methods in facets module

Posted by ab...@apache.org.
LUCENE-7610: Deprecate ValueSource methods in facets module


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/713b65d1
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/713b65d1
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/713b65d1

Branch: refs/heads/jira/solr-9856
Commit: 713b65d1dcc80c1fe147a5bf999e1a88b63b9dce
Parents: 8b05538
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 10:41:12 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:52:24 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../demo/facet/DistanceFacetsExample.java       |  18 +--
 .../org/apache/lucene/facet/package-info.java   |   2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |  33 ++++--
 .../facet/range/DoubleRangeFacetCounts.java     |  80 ++++++++-----
 .../apache/lucene/facet/range/LongRange.java    |  34 ++++--
 .../facet/range/LongRangeFacetCounts.java       |  47 +++++---
 .../org/apache/lucene/facet/range/Range.java    |  31 ------
 .../lucene/facet/taxonomy/FakeScorer.java       |  53 ---------
 .../taxonomy/TaxonomyFacetSumValueSource.java   |  99 ++++++++++++-----
 .../facet/range/TestRangeFacetCounts.java       |  53 +++++----
 .../TestTaxonomyFacetSumValueSource.java        |   5 +-
 .../lucene/queries/function/ValueSource.java    | 111 +++++++++++++++++++
 13 files changed, 361 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c667040..d0bedb7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -79,6 +79,9 @@ API Changes
   replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
   Grand)
 
+* LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
+  methods that take ValueSource parameters are deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
index 7d029ee..fa7ce83 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
@@ -16,9 +16,13 @@
  */
 package org.apache.lucene.demo.facet;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.text.ParseException;
+
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
-import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
@@ -36,9 +40,9 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -48,10 +52,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.SloppyMath;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.text.ParseException;
-
 /** Shows simple usage of dynamic range faceting, using the
  *  expressions module to calculate distance. */
 public class DistanceFacetsExample implements Closeable {
@@ -117,7 +117,7 @@ public class DistanceFacetsExample implements Closeable {
     writer.close();
   }
 
-  private ValueSource getDistanceValueSource() {
+  private DoubleValuesSource getDistanceValueSource() {
     Expression distance;
     try {
       distance = JavascriptCompiler.compile(
@@ -130,7 +130,7 @@ public class DistanceFacetsExample implements Closeable {
     bindings.add(new SortField("latitude", SortField.Type.DOUBLE));
     bindings.add(new SortField("longitude", SortField.Type.DOUBLE));
 
-    return distance.getValueSource(bindings);
+    return distance.getDoubleValuesSource(bindings);
   }
 
   /** Given a latitude and longitude (in degrees) and the
@@ -224,7 +224,7 @@ public class DistanceFacetsExample implements Closeable {
     // Passing no baseQuery means we drill down on all
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(null);
-    final ValueSource vs = getDistanceValueSource();
+    final DoubleValuesSource vs = getDistanceValueSource();
     q.add("field", range.getQuery(getBoundingBoxQuery(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, range.max), vs));
     DrillSideways ds = new DrillSideways(searcher, config, (TaxonomyReader) null) {
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
index 0501d6a..acbdd44 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
@@ -40,7 +40,7 @@
  * 
  *  <li> Range faceting {@link org.apache.lucene.facet.range.LongRangeFacetCounts}, {@link
  *       org.apache.lucene.facet.range.DoubleRangeFacetCounts} compute counts for a dynamic numeric
- *       range from a provided {@link org.apache.lucene.queries.function.ValueSource} (previously indexed
+ *       range from a provided {@link org.apache.lucene.search.LongValuesSource} (previously indexed
  *       numeric field, or a dynamic expression such as distance).
  * </ul>
  * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index 2203be3..ce377f5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -17,16 +17,16 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -95,9 +95,9 @@ public final class DoubleRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final DoubleRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final DoubleValuesSource valueSource;
 
-    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, DoubleValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -158,11 +158,11 @@ public final class DoubleRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final DoubleValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.doubleVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.doubleValue());
             }
 
             @Override
@@ -177,8 +177,27 @@ public final class DoubleRange extends Range {
 
   }
 
-  @Override
+  /**
+   * @deprecated Use {@link #getQuery(Query, DoubleValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asDoubleValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, DoubleValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 2d0ba5c..63fc935 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -17,22 +17,18 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.lucene.document.DoubleDocValuesField; // javadocs
-import org.apache.lucene.document.FloatDocValuesField; // javadocs
+import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource; // javadocs
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -41,46 +37,70 @@ import org.apache.lucene.util.NumericUtils;
 
 /** {@link Facets} implementation that computes counts for
  *  dynamic double ranges from a provided {@link
- *  ValueSource}, using {@link FunctionValues#doubleVal}.  Use
- *  this for dimensions that change in real-time (e.g. a
+ *  DoubleValuesSource}.  Use this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g.
  *  distance from the user's location, "&lt; 1 km", "&lt; 2 km",
  *  etc.).
  *
- *  <p> If you had indexed your field using {@link
- *  FloatDocValuesField} then pass {@link FloatFieldSource}
- *  as the {@link ValueSource}; if you used {@link
- *  DoubleDocValuesField} then pass {@link
- *  DoubleFieldSource} (this is the default used when you
- *  pass just a the field name).
+ *  If you have indexed your field using {@link
+ *  FloatDocValuesField}, then you should use a DoubleValuesSource
+ *  generated from {@link DoubleValuesSource#fromFloatField(String)}.
  *
  *  @lucene.experimental */
 public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
-  /** Create {@code RangeFacetCounts}, using {@link
-   *  DoubleFieldSource} from the specified field. */
+  /**
+   * Create {@code RangeFacetCounts}, using {@link DoubleValues} from the specified field.
+   *
+   * N.B This assumes that the field was indexed with {@link org.apache.lucene.document.DoubleDocValuesField}.
+   * For float-valued fields, use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   */
   public DoubleRangeFacetCounts(String field, FacetsCollector hits, DoubleRange... ranges) throws IOException {
-    this(field, new DoubleFieldSource(field), hits, ranges);
+    this(field, DoubleValuesSource.fromDoubleField(field), hits, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}. */
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
+   * */
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
-  /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}, and using the provided Query as
-   *  a fastmatch: only documents passing the filter are
-   *  checked for the matching ranges.  The filter must be
-   *  random access (implement {@link DocIdSet#bits}). */
+  /**
+   * Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource}
+   */
+  public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
+    this(field, valueSource, hits, null, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link ValueSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   *
+   * @deprecated Use ({@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, Query, DoubleRange...)}
+   */
+  @Deprecated
   public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
+    this(field, valueSource.asDoubleValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided
+   * {@link DoubleValuesSource}, and using the provided Query as
+   * a fastmatch: only documents matching the query are
+   * checked for the matching ranges.
+   */
+ public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     DoubleRange[] ranges = (DoubleRange[]) this.ranges;
 
@@ -96,7 +116,7 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      DoubleValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -129,8 +149,8 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(NumericUtils.doubleToSortableLong(fv.doubleVal(doc)));
+        if (fv.advanceExact(doc)) {
+          counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 9c12ecd..20c408d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -17,17 +17,17 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
@@ -87,9 +87,9 @@ public final class LongRange extends Range {
   private static class ValueSourceQuery extends Query {
     private final LongRange range;
     private final Query fastMatchQuery;
-    private final ValueSource valueSource;
+    private final LongValuesSource valueSource;
 
-    ValueSourceQuery(LongRange range, Query fastMatchQuery, ValueSource valueSource) {
+    ValueSourceQuery(LongRange range, Query fastMatchQuery, LongValuesSource valueSource) {
       this.range = range;
       this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
@@ -150,11 +150,11 @@ public final class LongRange extends Range {
             approximation = s.iterator();
           }
 
-          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final LongValues values = valueSource.getValues(context, null);
           final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
             @Override
             public boolean matches() throws IOException {
-              return range.accept(values.longVal(approximation.docID()));
+              return values.advanceExact(approximation.docID()) && range.accept(values.longValue());
             }
 
             @Override
@@ -169,8 +169,28 @@ public final class LongRange extends Range {
 
   }
 
-  @Override
+
+  /**
+   * @deprecated Use {@link #getQuery(Query, LongValuesSource)}
+   */
+  @Deprecated
   public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asLongValuesSource());
+  }
+
+  /**
+   * Create a Query that matches documents in this range
+   *
+   * The query will check all documents that match the provided match query,
+   * or every document in the index if the match query is null.
+   *
+   * If the value source is static, eg an indexed numeric field, it may be
+   * faster to use {@link org.apache.lucene.search.PointRangeQuery}
+   *
+   * @param fastMatchQuery a query to use as a filter
+   * @param valueSource    the source of values for the range check
+   */
+  public Query getQuery(Query fastMatchQuery, LongValuesSource valueSource) {
     return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index 0512ab3..a3cfc71 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -17,27 +17,25 @@
 package org.apache.lucene.facet.range;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.facet.Facets;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 
 /** {@link Facets} implementation that computes counts for
- *  dynamic long ranges from a provided {@link ValueSource},
- *  using {@link FunctionValues#longVal}.  Use
+ *  dynamic long ranges from a provided {@link LongValuesSource}.  Use
  *  this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each request (e.g. 
@@ -48,28 +46,49 @@ import org.apache.lucene.search.Weight;
 public class LongRangeFacetCounts extends RangeFacetCounts {
 
   /** Create {@code LongRangeFacetCounts}, using {@link
-   *  LongFieldSource} from the specified field. */
+   *  LongValuesSource} from the specified field. */
   public LongRangeFacetCounts(String field, FacetsCollector hits, LongRange... ranges) throws IOException {
-    this(field, new LongFieldSource(field), hits, ranges);
+    this(field, LongValuesSource.fromLongField(field), hits, ranges);
+  }
+
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, null, ranges);
   }
 
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
+  /**
+   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
+   *
+   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, Query, LongRange...)}
+   */
+  @Deprecated
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+    this(field, valueSource.asLongValuesSource(), hits, fastMatchQuery, ranges);
+  }
+
+
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges.  The filter must be
    *  random access (implement {@link DocIdSet#bits}). */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
     super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
-  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+  private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
 
     LongRange[] ranges = (LongRange[]) this.ranges;
 
@@ -77,7 +96,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
-      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      LongValues fv = valueSource.getValues(hits.context, null);
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
@@ -109,8 +128,8 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
           }
         }
         // Skip missing docs:
-        if (fv.exists(doc)) {
-          counter.add(fv.longVal(doc));
+        if (fv.advanceExact(doc)) {
+          counter.add(fv.longValue());
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
index 5f6de98..82b8088 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
@@ -16,10 +16,6 @@
  */
 package org.apache.lucene.facet.range;
 
-import org.apache.lucene.facet.DrillDownQuery; // javadocs
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Query;
-
 /** Base class for a single labeled range.
  *
  *  @lucene.experimental */
@@ -36,33 +32,6 @@ public abstract class Range {
     this.label = label;
   }
 
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to use
-   *  {@link org.apache.lucene.search.PointRangeQuery}. The provided fastMatchQuery,
-   *  if non-null, will first be consulted, and only if
-   *  that is set for each document will the range then be
-   *  checked. */
-  public abstract Query getQuery(Query fastMatchQuery, ValueSource valueSource);
-
-  /** Returns a new {@link Query} accepting only documents
-   *  in this range.  This query might not be very efficient
-   *  when run on its own since it is optimized towards
-   *  random-access, so it is best used either with
-   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
-   *  or when intersected with another query that can lead the
-   *  iteration.  If the {@link ValueSource} is static, e.g. an
-   *  indexed numeric field, then it may be more efficient to
-   *  use {@link org.apache.lucene.search.PointRangeQuery}. */
-  public Query getQuery(ValueSource valueSource) {
-    return getQuery(null, valueSource);
-  }
-
   /** Invoke this for a useless range. */
   protected void failNoMatch() {
     throw new IllegalArgumentException("range \"" + label + "\" matches nothing");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
deleted file mode 100644
index 238b74c..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FakeScorer.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Scorer;
-
-class FakeScorer extends Scorer {
-
-  float score;
-  int doc = -1;
-  int freq = 1;
-
-  FakeScorer() {
-    super(null);
-  }
-
-  @Override
-  public int docID() {
-    return doc;
-  }
-
-  @Override
-  public DocIdSetIterator iterator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int freq() throws IOException {
-    return freq;
-  }
-
-  @Override
-  public float score() throws IOException {
-    return score;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 4010c81..0a73ae5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -29,6 +28,8 @@ import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.IntsRef;
 
@@ -39,52 +40,94 @@ import org.apache.lucene.util.IntsRef;
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
   private final OrdinalsReader ordinalsReader;
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
    *  {@link ValueSource}, pulling ordinals using {@link
    *  DocValuesOrdinalsReader} against the default indexed
    *  facet field {@link
-   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}. */
+   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   *
+   *  @deprecated {@link #TaxonomyFacetSumValueSource(TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
                                      FacetsCollector fc, ValueSource valueSource) throws IOException {
     this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
   }
 
-  /** Aggreggates float facet values from the provided
+  /**
+   * Aggreggates double facet values from the provided
+   * {@link DoubleValuesSource}, pulling ordinals using {@link
+   * DocValuesOrdinalsReader} against the default indexed
+   * facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   */
+   public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
+                                     FacetsCollector fc, DoubleValuesSource valueSource) throws IOException {
+    this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
+  }
+
+  /**
+   * Aggreggates float facet values from the provided
    *  {@link ValueSource}, and pulls ordinals from the
-   *  provided {@link OrdinalsReader}. */
+   *  provided {@link OrdinalsReader}.
+   *
+   *  @deprecated use {@link #TaxonomyFacetSumValueSource(OrdinalsReader, TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
+   */
+  @Deprecated
   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
                                      FacetsConfig config, FacetsCollector fc, ValueSource valueSource) throws IOException {
     super(ordinalsReader.getIndexFieldName(), taxoReader, config);
     this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource);
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource.asDoubleValuesSource());
   }
 
-  private final void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, ValueSource valueSource) throws IOException {
-    final FakeScorer scorer = new FakeScorer();
-    Map<String, Scorer> context = new HashMap<>();
-    if (keepScores) {
-      context.put("scorer", scorer);
-    }
+  /**
+   * Aggreggates float facet values from the provided
+   *  {@link DoubleValuesSource}, and pulls ordinals from the
+   *  provided {@link OrdinalsReader}.
+   */
+   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
+                                     FacetsConfig config, FacetsCollector fc, DoubleValuesSource vs) throws IOException {
+    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
+    this.ordinalsReader = ordinalsReader;
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), vs);
+  }
+
+  private static DoubleValues scores(MatchingDocs hits) {
+    return new DoubleValues() {
+
+      int index = -1;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return hits.scores[index];
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        index++;
+        return true;
+      }
+    };
+  }
+
+  private void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, DoubleValuesSource valueSource) throws IOException {
+
     IntsRef scratch = new IntsRef();
     for(MatchingDocs hits : matchingDocs) {
       OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
-      
-      int scoresIdx = 0;
-      float[] scores = hits.scores;
-
-      FunctionValues functionValues = valueSource.getValues(context, hits.context);
+      DoubleValues scores = keepScores ? scores(hits) : null;
+      DoubleValues functionValues = valueSource.getValues(hits.context, scores);
       DocIdSetIterator docs = hits.bits.iterator();
       
       int doc;
       while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
         ords.get(doc, scratch);
-        if (keepScores) {
-          scorer.doc = doc;
-          scorer.score = scores[scoresIdx++];
-        }
-        float value = (float) functionValues.doubleVal(doc);
-        for(int i=0;i<scratch.length;i++) {
-          values[scratch.ints[i]] += value;
+        if (functionValues.advanceExact(doc)) {
+          float value = (float) functionValues.doubleValue();
+          for (int i = 0; i < scratch.length; i++) {
+            values[scratch.ints[i]] += value;
+          }
         }
       }
     }
@@ -92,9 +135,13 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
     rollup();
   }
 
-  /** {@link ValueSource} that returns the score for each
+  /**
+   * {@link ValueSource} that returns the score for each
    *  hit; use this to aggregate the sum of all hit scores
-   *  for each facet label.  */
+   *  for each facet label.
+   *
+   * @deprecated Use {@link DoubleValuesSource#SCORES}
+   */
   public static class ScoreValueSource extends ValueSource {
 
     /** Sole constructor. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index 7250ef4..ff207d3 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -28,8 +28,8 @@ import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.facet.DrillDownQuery;
-import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.FacetField;
 import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.FacetTestCase;
@@ -46,11 +46,11 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -708,7 +708,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
   }
 
-  public void testCustomDoublesValueSource() throws Exception {
+  public void testCustomDoubleValuesSource() throws Exception {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     
@@ -720,33 +720,30 @@ public class TestRangeFacetCounts extends FacetTestCase {
     // Test wants 3 docs in one segment:
     writer.forceMerge(1);
 
-    final ValueSource vs = new ValueSource() {
-        @SuppressWarnings("rawtypes")
-        @Override
-        public FunctionValues getValues(Map ignored, LeafReaderContext ignored2) {
-          return new DoubleDocValues(null) {
-            @Override
-            public double doubleVal(int doc) {
-              return doc+1;
-            }
-          };
-        }
+    final DoubleValuesSource vs = new DoubleValuesSource() {
 
-        @Override
-        public boolean equals(Object o) {
-          return o != null && getClass() == o.getClass();
-        }
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          int doc = -1;
+          @Override
+          public double doubleValue() throws IOException {
+            return doc + 1;
+          }
 
-        @Override
-        public int hashCode() {
-          return getClass().hashCode();
-        }
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            this.doc = doc;
+            return true;
+          }
+        };
+      }
 
-        @Override
-        public String description() {
-          throw new UnsupportedOperationException();
-        }
-      };
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
 
     FacetsConfig config = new FacetsConfig();
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 0ad90ba..31bf6e1 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -52,6 +52,7 @@ import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -266,7 +267,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     
     TopDocs td = FacetsCollector.search(newSearcher(r), csq, 10, fc);
 
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
     
     int expected = (int) (td.getMaxScore() * td.totalHits);
     assertEquals(expected, facets.getSpecificValue("dim", "a").intValue());
@@ -408,7 +409,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector.search(newSearcher(r), new MatchAllDocsQuery(), 10, fc);
     
     Facets facets1 = getTaxonomyFacetCounts(taxoReader, config, fc);
-    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, new TaxonomyFacetSumValueSource.ScoreValueSource());
+    Facets facets2 = new TaxonomyFacetSumValueSource(new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, DoubleValuesSource.SCORES);
 
     assertEquals(r.maxDoc(), facets1.getTopChildren(10, "a").value.intValue());
     assertEquals(r.maxDoc(), facets2.getTopChildren(10, "b").value.doubleValue(), 1E-10);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/713b65d1/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
index 4064fc9..5bf6324 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
@@ -17,13 +17,20 @@
 package org.apache.lucene.queries.function;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.SimpleFieldComparator;
 import org.apache.lucene.search.SortField;
 
@@ -78,6 +85,110 @@ public abstract class ValueSource {
     return context;
   }
 
+  private static class FakeScorer extends Scorer {
+
+    int current = -1;
+    float score = 0;
+
+    FakeScorer() {
+      super(null);
+    }
+
+    @Override
+    public int docID() {
+      return current;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Expose this ValueSource as a LongValuesSource
+   */
+  public LongValuesSource asLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new IdentityHashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        final FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new LongValues() {
+
+          @Override
+          public long longValue() throws IOException {
+            return fv.longVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc))
+              scorer.score = (float) scores.doubleValue();
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  /**
+   * Expose this ValueSource as a DoubleValuesSource
+   */
+  public DoubleValuesSource asDoubleValuesSource() {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        Map context = new HashMap<>();
+        FakeScorer scorer = new FakeScorer();
+        context.put("scorer", scorer);
+        FunctionValues fv = ValueSource.this.getValues(context, ctx);
+        return new DoubleValues() {
+
+          @Override
+          public double doubleValue() throws IOException {
+            return fv.doubleVal(scorer.current);
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            scorer.current = doc;
+            if (scores != null && scores.advanceExact(doc)) {
+              scorer.score = (float) scores.doubleValue();
+            }
+            else
+              scorer.score = 0;
+            return fv.exists(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return true;  // be on the safe side
+      }
+    };
+  }
 
   //
   // Sorting by function


[16/34] lucene-solr:jira/solr-9856: LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource

Posted by ab...@apache.org.
LUCENE-7611: Suggester uses LongValuesSource in place of ValueSource


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1a95c5ac
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1a95c5ac
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1a95c5ac

Branch: refs/heads/jira/solr-9856
Commit: 1a95c5acd0f69efb1a24b2c980a289289e703758
Parents: 713b65d
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Jan 6 11:03:09 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 11:52:25 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/LongValuesSource.java  |  27 ++
 .../suggest/DocumentValueSourceDictionary.java  |  72 ++++-
 .../DocumentValueSourceDictionaryTest.java      | 264 ++++++++++++++++++-
 .../DocumentExpressionDictionaryFactory.java    |   6 +-
 5 files changed, 352 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d0bedb7..30c9ab0 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -82,6 +82,9 @@ API Changes
 * LUCENE-7610: The facets module now uses the DoubleValuesSource API, and
   methods that take ValueSource parameters are deprecated (Alan Woodward)
 
+* LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
+  as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
index 9d00355..524822c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -74,6 +74,33 @@ public abstract class LongValuesSource {
     return fromLongField(field);
   }
 
+  /**
+   * Creates a LongValuesSource that always returns a constant value
+   */
+  public static LongValuesSource constant(long value) {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
   private static class FieldValuesSource extends LongValuesSource {
 
     final String field;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 2c0b8f4..656dc04 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -17,15 +17,15 @@
 package org.apache.lucene.search.suggest;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 
 
 /**
@@ -34,7 +34,7 @@ import org.apache.lucene.queries.function.ValueSource;
  * optionally contexts information
  * taken from stored fields in a Lucene index. Similar to 
  * {@link DocumentDictionary}, except it obtains the weight
- * of the terms in a document based on a {@link ValueSource}.
+ * of the terms in a document based on a {@link LongValuesSource}.
  * </p>
  * <b>NOTE:</b> 
  *  <ul>
@@ -46,44 +46,75 @@ import org.apache.lucene.queries.function.ValueSource;
  *    </li>
  *  </ul>
  *  <p>
- *  In practice the {@link ValueSource} will likely be obtained
+ *  In practice the {@link LongValuesSource} will likely be obtained
  *  using the lucene expression module. The following example shows
- *  how to create a {@link ValueSource} from a simple addition of two
+ *  how to create a {@link LongValuesSource} from a simple addition of two
  *  fields:
  *  <code>
  *    Expression expression = JavascriptCompiler.compile("f1 + f2");
  *    SimpleBindings bindings = new SimpleBindings();
  *    bindings.add(new SortField("f1", SortField.Type.LONG));
  *    bindings.add(new SortField("f2", SortField.Type.LONG));
- *    ValueSource valueSource = expression.getValueSource(bindings);
+ *    LongValuesSource valueSource = expression.getDoubleValuesSource(bindings).toLongValuesSource();
  *  </code>
  *  </p>
  *
  */
 public class DocumentValueSourceDictionary extends DocumentDictionary {
   
-  private final ValueSource weightsValueSource;
+  private final LongValuesSource weightsValueSource;
   
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
    * for the associated contexts and uses the <code>weightsValueSource</code> supplied 
    * to determine the score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload, String contexts) {
     super(reader, field, null, payload, contexts);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
+   * for the associated contexts and uses the <code>weightsValueSource</code> supplied
+   * to determine the score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload, String contexts) {
+    super(reader, field, null, payload, contexts);
     this.weightsValueSource = weightsValueSource;
   }
+
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
    * and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource, String payload) {
     super(reader, field, null, payload);
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms, <code>payloadField</code> for the corresponding payloads
+   * and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource, String payload) {
+    super(reader, field, null, payload);
     this.weightsValueSource = weightsValueSource;
   }
   
@@ -91,11 +122,25 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
    * score.
+   *
+   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource)}
    */
+  @Deprecated
   public DocumentValueSourceDictionary(IndexReader reader, String field,
                                        ValueSource weightsValueSource) {
     super(reader, field, null, null);
-    this.weightsValueSource = weightsValueSource;  
+    this.weightsValueSource = weightsValueSource.asLongValuesSource();
+  }
+
+  /**
+   * Creates a new dictionary with the contents of the fields named <code>field</code>
+   * for the terms and uses the <code>weightsValueSource</code> supplied to determine the
+   * score.
+   */
+  public DocumentValueSourceDictionary(IndexReader reader, String field,
+                                       LongValuesSource weightsValueSource) {
+    super(reader, field, null, null);
+    this.weightsValueSource = weightsValueSource;
   }
   
   @Override
@@ -105,7 +150,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   
   final class DocumentValueSourceInputIterator extends DocumentDictionary.DocumentInputIterator {
     
-    private FunctionValues currentWeightValues;
+    private LongValues currentWeightValues;
     /** leaves of the reader */
     private final List<LeafReaderContext> leaves;
     /** starting docIds of all the leaves */
@@ -123,7 +168,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       }
       starts[leaves.size()] = reader.maxDoc();
       currentWeightValues = (leaves.size() > 0) 
-          ? weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex))
+          ? weightsValueSource.getValues(leaves.get(currentLeafIndex), null)
           : null;
     }
     
@@ -140,13 +185,16 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
       if (subIndex != currentLeafIndex) {
         currentLeafIndex = subIndex;
         try {
-          currentWeightValues = weightsValueSource.getValues(new HashMap<String, Object>(), leaves.get(currentLeafIndex));
+          currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
       }
       try {
-        return currentWeightValues.longVal(docId - starts[subIndex]);
+        if (currentWeightValues.advanceExact(docId - starts[subIndex]))
+          return currentWeightValues.longValue();
+        else
+          return 0;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index 9e58a4e..92799cd 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -36,12 +36,16 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.SumFloatFunction;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -59,7 +63,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   static final String CONTEXTS_FIELD_NAME = "c1";
   
   @Test
-  public void testEmptyReader() throws IOException {
+  public void testValueSourceEmptyReader() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -78,9 +82,30 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
 
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceEmptyReader() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    // Make sure the index is created?
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    writer.commit();
+    writer.close();
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+
+    assertNull(inputIterator.next());
+    assertEquals(inputIterator.weight(), 0);
+    assertNull(inputIterator.payload());
+
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testBasic() throws IOException {
+  public void testValueSourceBasic() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -112,9 +137,83 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  private static LongValuesSource sum(String... fields) {
+    LongValuesSource[] sources = new LongValuesSource[fields.length];
+    for (int i = 0; i < fields.length; i++) {
+      sources[i] = LongValuesSource.fromLongField(fields[i]);
+    }
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        LongValues[] values = new LongValues[fields.length];
+        for (int i = 0; i < sources.length; i++) {
+          values[i] = sources[i].getValues(ctx, scores);
+        }
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            long v = 0;
+            for (LongValues value : values) {
+              v += value.longValue();
+            }
+            return v;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            boolean v = true;
+            for (LongValues value : values) {
+              v &= value.advanceExact(doc);
+            }
+            return v;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  @Test
+  public void testLongValuesSourceBasic() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValueSource = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValueSource, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithContext() throws IOException {
+  public void testValueSourceWithContext() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -153,7 +252,46 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
   }
 
   @Test
-  public void testWithoutPayload() throws IOException {
+  public void testLongValuesSourceWithContext() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+      Set<BytesRef> originalCtxs = new HashSet<>();
+      for (IndexableField ctxf: doc.getFields(CONTEXTS_FIELD_NAME)) {
+        originalCtxs.add(ctxf.binaryValue());
+      }
+      assertEquals(originalCtxs, inputIterator.contexts());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
+  @Test
+  public void testValueSourceWithoutPayload() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -183,9 +321,41 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithoutPayload() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      long w3 = doc.getField(WEIGHT_FIELD_NAME_3).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), (w1 + w2 + w3));
+      assertNull(inputIterator.payload());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
-  public void testWithDeletions() throws IOException {
+  public void testValueSourceWithDeletions() throws IOException {
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random());
     IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
@@ -238,6 +408,60 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     assertTrue(docs.isEmpty());
     IOUtils.close(ir, analyzer, dir);
   }
+
+  @Test
+  public void testLongValuesSourceWithDeletions() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    Random rand = random();
+    List<String> termsToDel = new ArrayList<>();
+    for(Document doc : docs.values()) {
+      if(rand.nextBoolean() && termsToDel.size() < docs.size()-1) {
+        termsToDel.add(doc.get(FIELD_NAME));
+      }
+      writer.addDocument(doc);
+    }
+    writer.commit();
+
+    Term[] delTerms = new Term[termsToDel.size()];
+    for(int i=0; i < termsToDel.size() ; i++) {
+      delTerms[i] = new Term(FIELD_NAME, termsToDel.get(i));
+    }
+
+    for(Term delTerm: delTerms) {
+      writer.deleteDocuments(delTerm);
+    }
+    writer.commit();
+    writer.close();
+
+    for(String termToDel: termsToDel) {
+      assertTrue(null!=docs.remove(termToDel));
+    }
+
+    IndexReader ir = DirectoryReader.open(dir);
+    assertTrue("NumDocs should be > 0 but was " + ir.numDocs(), ir.numDocs() > 0);
+    assertEquals(ir.numDocs(), docs.size());
+    LongValuesSource sumValues = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, sumValues, PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      long w1 = doc.getField(WEIGHT_FIELD_NAME_1).numericValue().longValue();
+      long w2 = doc.getField(WEIGHT_FIELD_NAME_2).numericValue().longValue();
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), w2+w1);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
   
   @Test
   public void testWithValueSource() throws IOException {
@@ -269,6 +493,36 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IOUtils.close(ir, analyzer, dir);
   }
 
+  @Test
+  public void testWithLongValuesSource() throws IOException {
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriterConfig iwc = newIndexWriterConfig(analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    Map<String, Document> docs = generateIndexDocuments(atLeast(100));
+    for(Document doc: docs.values()) {
+      writer.addDocument(doc);
+    }
+    writer.commit();
+    writer.close();
+
+    IndexReader ir = DirectoryReader.open(dir);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
+    InputIterator inputIterator = dictionary.getEntryIterator();
+    BytesRef f;
+    while((f = inputIterator.next())!=null) {
+      Document doc = docs.remove(f.utf8ToString());
+      assertTrue(f.equals(new BytesRef(doc.get(FIELD_NAME))));
+      assertEquals(inputIterator.weight(), 10);
+      IndexableField payloadField = doc.getField(PAYLOAD_FIELD_NAME);
+      if (payloadField == null) assertTrue(inputIterator.payload().length == 0);
+      else assertEquals(inputIterator.payload(), payloadField.binaryValue());
+    }
+    assertTrue(docs.isEmpty());
+    IOUtils.close(ir, analyzer, dir);
+  }
+
   private Map<String, Document> generateIndexDocuments(int ndocs) {
     Map<String, Document> docs = new HashMap<>();
     for(int i = 0; i < ndocs ; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1a95c5ac/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
index 24f1553..b0d7007 100644
--- a/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
+++ b/solr/core/src/java/org/apache/solr/spelling/suggest/DocumentExpressionDictionaryFactory.java
@@ -23,7 +23,7 @@ import java.util.Set;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
-import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.suggest.DocumentValueSourceDictionary;
@@ -93,7 +93,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
         sortFields), payloadField);
   }
 
-  public ValueSource fromExpression(String weightExpression, Set<SortField> sortFields) {
+  public LongValuesSource fromExpression(String weightExpression, Set<SortField> sortFields) {
     Expression expression = null;
     try {
       expression = JavascriptCompiler.compile(weightExpression);
@@ -104,7 +104,7 @@ public class DocumentExpressionDictionaryFactory extends DictionaryFactory {
     for (SortField sortField : sortFields) {
       bindings.add(sortField);
     }
-    return expression.getValueSource(bindings);
+    return expression.getDoubleValuesSource(bindings).toLongValuesSource();
   }
   
   private SortField.Type getSortFieldType(SolrCore core, String sortFieldName) {


[23/34] lucene-solr:jira/solr-9856: SOLR-9944: Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-9944: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-9856
Commit: ac14fc32e045d45b5129dc237f7e5472fc86e4a0
Parents: aae4217
Author: Joel Bernstein <jb...@apache.org>
Authored: Sat Jan 7 22:19:46 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sat Jan 7 22:19:46 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac14fc32/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0d61730..899dcd3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -401,6 +401,8 @@ Other Changes
 
 * SOLR-3990: Moves getIndexSize() from ReplicationHandler to SolrCore (Shawn Heisey)
 
+* SOLR-9944: Map the nodes function name to the GatherNodesStream (Joel Bernstein)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.


[26/34] lucene-solr:jira/solr-9856: LUCENE-7588: the parallell search method was failing to pass on the user's requested sort when merge-sorting all hits

Posted by ab...@apache.org.
LUCENE-7588: the parallell search method was failing to pass on the user's requested sort when merge-sorting all hits


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1aa9c425
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1aa9c425
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1aa9c425

Branch: refs/heads/jira/solr-9856
Commit: 1aa9c4251289e71ab8e87b03797b20f4a8fda0a5
Parents: 570880d
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:24:03 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:24:03 2017 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/facet/DrillSideways.java  | 37 ++++++++++----------
 1 file changed, 19 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aa9c425/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
index 61530bc..b2391f5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
@@ -16,6 +16,17 @@
  */
 package org.apache.lucene.facet;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
@@ -34,20 +45,10 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.util.ThreadInterruptedException;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
 /**
  * Computes drill down and sideways counts for the provided
  * {@link DrillDownQuery}.  Drill sideways counts include
@@ -234,8 +235,8 @@ public class DrillSideways {
 
       if (executor != null) { // We have an executor, let use the multi-threaded version
 
-        final CollectorManager<TopFieldCollector, TopDocs> collectorManager =
-                new CollectorManager<TopFieldCollector, TopDocs>() {
+        final CollectorManager<TopFieldCollector, TopFieldDocs> collectorManager =
+                new CollectorManager<TopFieldCollector, TopFieldDocs>() {
 
                   @Override
                   public TopFieldCollector newCollector() throws IOException {
@@ -243,16 +244,16 @@ public class DrillSideways {
                   }
 
                   @Override
-                  public TopDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
-                    final TopDocs[] topDocs = new TopDocs[collectors.size()];
+                  public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
+                    final TopFieldDocs[] topFieldDocs = new TopFieldDocs[collectors.size()];
                     int pos = 0;
                     for (TopFieldCollector collector : collectors)
-                      topDocs[pos++] = collector.topDocs();
-                    return TopDocs.merge(topN, topDocs);
+                      topFieldDocs[pos++] = collector.topDocs();
+                    return TopDocs.merge(sort, topN, topFieldDocs);
                   }
 
                 };
-        ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
+        ConcurrentDrillSidewaysResult<TopFieldDocs> r = search(query, collectorManager);
         return new DrillSidewaysResult(r.facets, r.collectorResult);
 
       } else {


[06/34] lucene-solr:jira/solr-9856: SOLR-8530: Add support for aggregate HAVING comparisons without single quotes

Posted by ab...@apache.org.
SOLR-8530: Add support for aggregate HAVING comparisons without single quotes


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

Branch: refs/heads/jira/solr-9856
Commit: b32cd82318f5c8817a8383e1be7534c772e6fa13
Parents: a810fb3
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Jan 5 20:36:32 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Jan 5 20:46:55 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/io/ops/EqualsOperation.java    | 2 +-
 .../solr/client/solrj/io/ops/GreaterThanEqualToOperation.java   | 2 +-
 .../apache/solr/client/solrj/io/ops/GreaterThanOperation.java   | 2 +-
 .../java/org/apache/solr/client/solrj/io/ops/LeafOperation.java | 1 -
 .../solr/client/solrj/io/ops/LessThanEqualToOperation.java      | 2 +-
 .../org/apache/solr/client/solrj/io/ops/LessThanOperation.java  | 2 +-
 .../apache/solr/client/solrj/io/stream/expr/StreamFactory.java  | 2 ++
 .../solr/client/solrj/io/stream/StreamExpressionTest.java       | 5 ++---
 8 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
index 8506f30..1958551 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
@@ -54,7 +54,7 @@ public class EqualsOperation extends LeafOperation {
 
   public StreamExpression toExpression(StreamFactory factory) throws IOException {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(quote(field));
+    expression.addParameter(field);
     expression.addParameter(Double.toString(val));
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
index 5e6dd85..87c8364 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
@@ -54,7 +54,7 @@ public class GreaterThanEqualToOperation extends LeafOperation {
 
   public StreamExpression toExpression(StreamFactory factory) throws IOException {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(quote(field));
+    expression.addParameter(field);
     expression.addParameter(Double.toString(val));
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
index 9c181c1..664438a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
@@ -54,7 +54,7 @@ public class GreaterThanOperation extends LeafOperation {
 
   public StreamExpression toExpression(StreamFactory factory) throws IOException {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(quote(field));
+    expression.addParameter(field);
     expression.addParameter(Double.toString(val));
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
index 9a5c407..691a328 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
@@ -45,7 +45,6 @@ public abstract class LeafOperation implements BooleanOperation {
 
   public LeafOperation(StreamExpression expression, StreamFactory factory) throws IOException {
     this.field = factory.getValueOperand(expression, 0);
-    this.field = this.field.replace("'","");
     this.val = Double.parseDouble(factory.getValueOperand(expression, 1));
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
index ca362cf..2da3274 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
@@ -54,7 +54,7 @@ public class LessThanEqualToOperation extends LeafOperation {
 
   public StreamExpression toExpression(StreamFactory factory) throws IOException {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(quote(field));
+    expression.addParameter(field);
     expression.addParameter(Double.toString(val));
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
index 433f45f..c1cec95 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
@@ -54,7 +54,7 @@ public class LessThanOperation extends LeafOperation {
 
   public StreamExpression toExpression(StreamFactory factory) throws IOException {
     StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(quote(field));
+    expression.addParameter(field);
     expression.addParameter(Double.toString(val));
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
index d2e72df..bf20a1e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
@@ -93,6 +93,8 @@ public class StreamFactory implements Serializable {
     if(null != parameter){ 
       if(parameter instanceof StreamExpressionValue){
         return ((StreamExpressionValue)parameter).getValue();
+      } else if(parameter instanceof StreamExpression) {
+        return parameter.toString();
       }
     }
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b32cd823/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 49814a3..5b806a8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -915,14 +915,13 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertTrue(t.getString("id").equals("9"));
 
 
-    stream = factory.constructStream("having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")), eq('sum(a_i)', 9))");
+    stream = factory.constructStream("having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")), and(eq(sum(a_i), 9),eq(sum(a_i), 9)))");
     context = new StreamContext();
     context.setSolrClientCache(solrClientCache);
     stream.setStreamContext(context);
     tuples = getTuples(stream);
 
     assert(tuples.size() == 1);
-
     t = tuples.get(0);
     assertTrue(t.getDouble("a_f") == 10.0D);
 
@@ -1024,7 +1023,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     t = tuples.get(1);
     assertTrue(t.getString("id").equals("9"));
 
-    stream = factory.constructStream("parallel("+COLLECTIONORALIAS+", workers=2, sort=\"a_f asc\", having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=a_f)), eq('sum(a_i)', 9)))");
+    stream = factory.constructStream("parallel("+COLLECTIONORALIAS+", workers=2, sort=\"a_f asc\", having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=a_f)), and(eq(sum(a_i), 9),eq(sum(a_i),9))))");
     context = new StreamContext();
     context.setSolrClientCache(solrClientCache);
     stream.setStreamContext(context);


[12/34] lucene-solr:jira/solr-9856: LUCENE-7617: Grouping collector API cleanup

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
new file mode 100644
index 0000000..2ff79a1
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.search.Sort;
+
+/**
+ * A factory object to create first and second-pass collectors, run by a {@link GroupingSearch}
+ * @param <T> the type the group value
+ */
+public abstract class Grouper<T> {
+
+  /**
+   * Create a first-pass collector
+   * @param sort  the order in which groups should be returned
+   * @param count how many groups to return
+   */
+  public abstract FirstPassGroupingCollector<T> getFirstPassCollector(Sort sort, int count) throws IOException;
+
+  /**
+   * Create an {@link AllGroupsCollector}
+   */
+  public abstract AllGroupsCollector<T> getAllGroupsCollector();
+
+  /**
+   * Create an {@link AllGroupHeadsCollector}
+   * @param sort a within-group sort order to determine which doc is the group head
+   */
+  public abstract AllGroupHeadsCollector<T> getGroupHeadsCollector(Sort sort);
+
+  /**
+   * Create a second-pass collector
+   */
+  public abstract SecondPassGroupingCollector<T> getSecondPassCollector(
+      Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
+      int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
index d0bab09..f4319d5 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
@@ -16,6 +16,11 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.CachingCollector;
 import org.apache.lucene.search.Collector;
@@ -25,25 +30,12 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
-import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.function.FunctionSecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
-import org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.function.FunctionGrouper;
+import org.apache.lucene.search.grouping.term.TermGrouper;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.mutable.MutableValue;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Convenience class to perform grouping in a non distributed environment.
  *
@@ -51,9 +43,7 @@ import java.util.Map;
  */
 public class GroupingSearch {
 
-  private final String groupField;
-  private final ValueSource groupFunction;
-  private final Map<?, ?> valueSourceContext;
+  private final Grouper grouper;
   private final Query groupEndDocs;
 
   private Sort groupSort = Sort.RELEVANCE;
@@ -70,7 +60,6 @@ public class GroupingSearch {
   private boolean cacheScores;
   private boolean allGroups;
   private boolean allGroupHeads;
-  private int initialSize = 128;
 
   private Collection<?> matchingGroups;
   private Bits matchingGroupHeads;
@@ -82,7 +71,11 @@ public class GroupingSearch {
    * @param groupField The name of the field to group by.
    */
   public GroupingSearch(String groupField) {
-    this(groupField, null, null, null);
+    this(new TermGrouper(groupField, 128), null);
+  }
+
+  public GroupingSearch(String groupField, int initialSize) {
+    this(new TermGrouper(groupField, initialSize), null);
   }
 
   /**
@@ -93,7 +86,7 @@ public class GroupingSearch {
    * @param valueSourceContext The context of the specified groupFunction
    */
   public GroupingSearch(ValueSource groupFunction, Map<?, ?> valueSourceContext) {
-    this(null, groupFunction, valueSourceContext, null);
+    this(new FunctionGrouper(groupFunction, valueSourceContext), null);
   }
 
   /**
@@ -103,13 +96,11 @@ public class GroupingSearch {
    * @param groupEndDocs The query that marks the last document in all doc blocks
    */
   public GroupingSearch(Query groupEndDocs) {
-    this(null, null, null, groupEndDocs);
+    this(null, groupEndDocs);
   }
 
-  private GroupingSearch(String groupField, ValueSource groupFunction, Map<?, ?> valueSourceContext, Query groupEndDocs) {
-    this.groupField = groupField;
-    this.groupFunction = groupFunction;
-    this.valueSourceContext = valueSourceContext;
+  private GroupingSearch(Grouper grouper, Query groupEndDocs) {
+    this.grouper = grouper;
     this.groupEndDocs = groupEndDocs;
   }
 
@@ -125,7 +116,7 @@ public class GroupingSearch {
    */
   @SuppressWarnings("unchecked")
   public <T> TopGroups<T> search(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
-    if (groupField != null || groupFunction != null) {
+    if (grouper != null) {
       return groupByFieldOrFunction(searcher, query, groupOffset, groupLimit);
     } else if (groupEndDocs != null) {
       return (TopGroups<T>) groupByDocBlock(searcher, query, groupOffset, groupLimit);
@@ -137,49 +128,13 @@ public class GroupingSearch {
   @SuppressWarnings({"unchecked", "rawtypes"})
   protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
     int topN = groupOffset + groupLimit;
-    final AbstractFirstPassGroupingCollector firstPassCollector;
-    final AbstractAllGroupsCollector allGroupsCollector;
-    final AbstractAllGroupHeadsCollector allGroupHeadsCollector;
-    if (groupFunction != null) {
-      firstPassCollector = new FunctionFirstPassGroupingCollector(groupFunction, valueSourceContext, groupSort, topN);
-      if (allGroups) {
-        allGroupsCollector = new FunctionAllGroupsCollector(groupFunction, valueSourceContext);
-      } else {
-        allGroupsCollector = null;
-      }
-      if (allGroupHeads) {
-        allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(groupFunction, valueSourceContext, sortWithinGroup);
-      } else {
-        allGroupHeadsCollector = null;
-      }
-    } else {
-      firstPassCollector = new TermFirstPassGroupingCollector(groupField, groupSort, topN);
-      if (allGroups) {
-        allGroupsCollector = new TermAllGroupsCollector(groupField, initialSize);
-      } else {
-        allGroupsCollector = null;
-      }
-      if (allGroupHeads) {
-        allGroupHeadsCollector = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup, initialSize);
-      } else {
-        allGroupHeadsCollector = null;
-      }
-    }
 
-    final Collector firstRound;
-    if (allGroupHeads || allGroups) {
-      List<Collector> collectors = new ArrayList<>();
-      collectors.add(firstPassCollector);
-      if (allGroups) {
-        collectors.add(allGroupsCollector);
-      }
-      if (allGroupHeads) {
-        collectors.add(allGroupHeadsCollector);
-      }
-      firstRound = MultiCollector.wrap(collectors.toArray(new Collector[collectors.size()]));
-    } else {
-      firstRound = firstPassCollector;
-    }
+    final FirstPassGroupingCollector firstPassCollector = grouper.getFirstPassCollector(groupSort, topN);
+    final AllGroupsCollector allGroupsCollector = allGroups ? grouper.getAllGroupsCollector() : null;
+    final AllGroupHeadsCollector allGroupHeadsCollector
+        = allGroupHeads ? grouper.getGroupHeadsCollector(sortWithinGroup) : null;
+
+    final Collector firstRound = MultiCollector.wrap(firstPassCollector, allGroupsCollector, allGroupHeadsCollector);
 
     CachingCollector cachedCollector = null;
     if (maxCacheRAMMB != null || maxDocsToCache != null) {
@@ -193,16 +148,9 @@ public class GroupingSearch {
       searcher.search(query, firstRound);
     }
 
-    if (allGroups) {
-      matchingGroups = allGroupsCollector.getGroups();
-    } else {
-      matchingGroups = Collections.emptyList();
-    }
-    if (allGroupHeads) {
-      matchingGroupHeads = allGroupHeadsCollector.retrieveGroupHeads(searcher.getIndexReader().maxDoc());
-    } else {
-      matchingGroupHeads = new Bits.MatchNoBits(searcher.getIndexReader().maxDoc());
-    }
+    matchingGroups = allGroups ? allGroupsCollector.getGroups() : Collections.emptyList();
+    matchingGroupHeads = allGroupHeads ? allGroupHeadsCollector.retrieveGroupHeads(searcher.getIndexReader().maxDoc())
+        : new Bits.MatchNoBits(searcher.getIndexReader().maxDoc());
 
     Collection<SearchGroup> topSearchGroups = firstPassCollector.getTopGroups(groupOffset, fillSortFields);
     if (topSearchGroups == null) {
@@ -210,12 +158,9 @@ public class GroupingSearch {
     }
 
     int topNInsideGroup = groupDocsOffset + groupDocsLimit;
-    AbstractSecondPassGroupingCollector secondPassCollector;
-    if (groupFunction != null) {
-      secondPassCollector = new FunctionSecondPassGroupingCollector((Collection) topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields, groupFunction, valueSourceContext);
-    } else {
-      secondPassCollector = new TermSecondPassGroupingCollector(groupField, (Collection) topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields);
-    }
+    SecondPassGroupingCollector secondPassCollector
+        = grouper.getSecondPassCollector(topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
+                                         includeScores, includeMaxScore, fillSortFields);
 
     if (cachedCollector != null && cachedCollector.isCached()) {
       cachedCollector.replay(secondPassCollector);
@@ -411,19 +356,4 @@ public class GroupingSearch {
     return matchingGroupHeads;
   }
 
-  /**
-   * Sets the initial size of some internal used data structures.
-   * This prevents growing data structures many times. This can improve the performance of the grouping at the cost of
-   * more initial RAM.
-   * <p>
-   * The {@link #setAllGroups} and {@link #setAllGroupHeads} features use this option.
-   * Defaults to 128.
-   *
-   * @param initialSize The initial size of some internal used data structures
-   * @return <code>this</code>
-   */
-  public GroupingSearch setInitialSize(int initialSize) {
-    this.initialSize = initialSize;
-    return this;
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
index b3a43cb..95a507c 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
@@ -16,28 +16,37 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
  * Represents a group that is found during the first pass search.
  *
  * @lucene.experimental
  */
-public class SearchGroup<GROUP_VALUE_TYPE> {
+public class SearchGroup<T> {
 
   /** The value that defines this group  */
-  public GROUP_VALUE_TYPE groupValue;
+  public T groupValue;
 
   /** The sort values used during sorting. These are the
    *  groupSort field values of the highest rank document
    *  (by the groupSort) within the group.  Can be
    * <code>null</code> if <code>fillFields=false</code> had
-   * been passed to {@link AbstractFirstPassGroupingCollector#getTopGroups} */
+   * been passed to {@link FirstPassGroupingCollector#getTopGroups} */
   public Object[] sortValues;
 
   @Override
@@ -327,7 +336,7 @@ public class SearchGroup<GROUP_VALUE_TYPE> {
    *  groupSort must match how the groups were sorted, and
    *  the provided SearchGroups must have been computed
    *  with fillFields=true passed to {@link
-   *  AbstractFirstPassGroupingCollector#getTopGroups}.
+   *  FirstPassGroupingCollector#getTopGroups}.
    *
    * <p>NOTE: this returns null if the topGroups is empty.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
new file mode 100644
index 0000000..f8feb75
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+
+/**
+ * SecondPassGroupingCollector is the second of two passes
+ * necessary to collect grouped docs.  This pass gathers the
+ * top N documents per top group computed from the
+ * first pass. Concrete subclasses define what a group is and how it
+ * is internally collected.
+ *
+ * <p>See {@link org.apache.lucene.search.grouping} for more
+ * details including a full code example.</p>
+ *
+ * @lucene.experimental
+ */
+public abstract class SecondPassGroupingCollector<T> extends SimpleCollector {
+
+  private final Collection<SearchGroup<T>> groups;
+  private final Sort groupSort;
+  private final Sort withinGroupSort;
+  private final int maxDocsPerGroup;
+  private final boolean needsScores;
+  protected final Map<T, SearchGroupDocs<T>> groupMap;
+
+  protected SearchGroupDocs<T>[] groupDocs;
+
+  private int totalHitCount;
+  private int totalGroupedHitCount;
+
+  public SecondPassGroupingCollector(Collection<SearchGroup<T>> groups, Sort groupSort, Sort withinGroupSort,
+                                     int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
+    throws IOException {
+
+    //System.out.println("SP init");
+    if (groups.isEmpty()) {
+      throw new IllegalArgumentException("no groups to collect (groups is empty)");
+    }
+
+    this.groups = Objects.requireNonNull(groups);
+    this.groupSort = Objects.requireNonNull(groupSort);
+    this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
+    this.maxDocsPerGroup = maxDocsPerGroup;
+    this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+
+    this.groupMap = new HashMap<>(groups.size());
+    for (SearchGroup<T> group : groups) {
+      //System.out.println("  prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+      final TopDocsCollector<?> collector;
+      if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector
+        // Sort by score
+        collector = TopScoreDocCollector.create(maxDocsPerGroup);
+      } else {
+        // Sort by fields
+        collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
+      }
+      groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector));
+    }
+  }
+
+  @Override
+  public boolean needsScores() {
+    return needsScores;
+  }
+
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {
+    for (SearchGroupDocs<T> group : groupMap.values()) {
+      group.leafCollector.setScorer(scorer);
+    }
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    totalHitCount++;
+    SearchGroupDocs<T> group = retrieveGroup(doc);
+    if (group != null) {
+      totalGroupedHitCount++;
+      group.leafCollector.collect(doc);
+    }
+  }
+
+  /**
+   * Returns the group the specified doc belongs to or <code>null</code> if no group could be retrieved.
+   *
+   * @param doc The specified doc
+   * @return the group the specified doc belongs to or <code>null</code> if no group could be retrieved
+   * @throws IOException If an I/O related error occurred
+   */
+  protected abstract SearchGroupDocs<T> retrieveGroup(int doc) throws IOException;
+
+  @Override
+  protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
+    //System.out.println("SP.setNextReader");
+    for (SearchGroupDocs<T> group : groupMap.values()) {
+      group.leafCollector = group.collector.getLeafCollector(readerContext);
+    }
+  }
+
+  public TopGroups<T> getTopGroups(int withinGroupOffset) {
+    @SuppressWarnings({"unchecked","rawtypes"})
+    final GroupDocs<T>[] groupDocsResult = (GroupDocs<T>[]) new GroupDocs[groups.size()];
+
+    int groupIDX = 0;
+    float maxScore = Float.MIN_VALUE;
+    for(SearchGroup<?> group : groups) {
+      final SearchGroupDocs<T> groupDocs = groupMap.get(group.groupValue);
+      final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
+      groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
+                                                                    topDocs.getMaxScore(),
+                                                                    topDocs.totalHits,
+                                                                    topDocs.scoreDocs,
+                                                                    groupDocs.groupValue,
+                                                                    group.sortValues);
+      maxScore = Math.max(maxScore, topDocs.getMaxScore());
+    }
+
+    return new TopGroups<>(groupSort.getSort(),
+                                           withinGroupSort.getSort(),
+                                           totalHitCount, totalGroupedHitCount, groupDocsResult,
+                                           maxScore);
+  }
+
+
+  // TODO: merge with SearchGroup or not?
+  // ad: don't need to build a new hashmap
+  // disad: blows up the size of SearchGroup if we need many of them, and couples implementations
+  public class SearchGroupDocs<T> {
+
+    public final T groupValue;
+    public final TopDocsCollector<?> collector;
+    public LeafCollector leafCollector;
+
+    public SearchGroupDocs(T groupValue, TopDocsCollector<?> collector) {
+      this.groupValue = groupValue;
+      this.collector = collector;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java
index a60c8f5..803482b 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroups.java
@@ -16,18 +16,18 @@
  */
 package org.apache.lucene.search.grouping;
 
+import java.io.IOException;
+
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldDocs;
 
-import java.io.IOException;
-
 /** Represents result returned by a grouping search.
  *
  * @lucene.experimental */
-public class TopGroups<GROUP_VALUE_TYPE> {
+public class TopGroups<T> {
   /** Number of documents matching the search */
   public final int totalHitCount;
 
@@ -38,7 +38,7 @@ public class TopGroups<GROUP_VALUE_TYPE> {
   public final Integer totalGroupCount;
 
   /** Group results in groupSort order */
-  public final GroupDocs<GROUP_VALUE_TYPE>[] groups;
+  public final GroupDocs<T>[] groups;
 
   /** How groups are sorted against each other */
   public final SortField[] groupSort;
@@ -50,7 +50,7 @@ public class TopGroups<GROUP_VALUE_TYPE> {
    *  <code>Float.NaN</code> if scores were not computed. */
   public final float maxScore;
 
-  public TopGroups(SortField[] groupSort, SortField[] withinGroupSort, int totalHitCount, int totalGroupedHitCount, GroupDocs<GROUP_VALUE_TYPE>[] groups, float maxScore) {
+  public TopGroups(SortField[] groupSort, SortField[] withinGroupSort, int totalHitCount, int totalGroupedHitCount, GroupDocs<T>[] groups, float maxScore) {
     this.groupSort = groupSort;
     this.withinGroupSort = withinGroupSort;
     this.totalHitCount = totalHitCount;
@@ -60,7 +60,7 @@ public class TopGroups<GROUP_VALUE_TYPE> {
     this.maxScore = maxScore;
   }
 
-  public TopGroups(TopGroups<GROUP_VALUE_TYPE> oldTopGroups, Integer totalGroupCount) {
+  public TopGroups(TopGroups<T> oldTopGroups, Integer totalGroupCount) {
     this.groupSort = oldTopGroups.groupSort;
     this.withinGroupSort = oldTopGroups.withinGroupSort;
     this.totalHitCount = oldTopGroups.totalHitCount;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
index 4c6071c..f4d4668 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
@@ -16,6 +16,11 @@
  */
 package org.apache.lucene.search.grouping.function;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -24,25 +29,20 @@ import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
 import org.apache.lucene.util.mutable.MutableValue;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
 /**
- * An implementation of {@link AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
+ * An implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
  * by {@link ValueSource}.
  *
  * @lucene.experimental
  */
-public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollector<FunctionAllGroupHeadsCollector.GroupHead> {
+public class FunctionAllGroupHeadsCollector extends AllGroupHeadsCollector<MutableValue> {
 
   private final ValueSource groupBy;
   private final Map<?, ?> vsContext;
-  private final Map<MutableValue, GroupHead> groups;
+  private final Map<MutableValue, FunctionGroupHead> groups;
   private final Sort sortWithinGroup;
 
   private FunctionValues.ValueFiller filler;
@@ -73,10 +73,10 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
   @Override
   protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
     filler.fillValue(doc);
-    GroupHead groupHead = groups.get(mval);
+    FunctionGroupHead groupHead = groups.get(mval);
     if (groupHead == null) {
       MutableValue groupValue = mval.duplicate();
-      groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
+      groupHead = new FunctionGroupHead(groupValue, sortWithinGroup, doc);
       groups.put(groupValue, groupHead);
       temporalResult.stop = true;
     } else {
@@ -86,14 +86,14 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
   }
 
   @Override
-  protected Collection<GroupHead> getCollectedGroupHeads() {
+  protected Collection<FunctionGroupHead> getCollectedGroupHeads() {
     return groups.values();
   }
 
   @Override
   public void setScorer(Scorer scorer) throws IOException {
     this.scorer = scorer;
-    for (GroupHead groupHead : groups.values()) {
+    for (FunctionGroupHead groupHead : groups.values()) {
       for (LeafFieldComparator comparator : groupHead.leafComparators) {
         comparator.setScorer(scorer);
       }
@@ -107,7 +107,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
     filler = values.getValueFiller();
     mval = filler.getValue();
 
-    for (GroupHead groupHead : groups.values()) {
+    for (FunctionGroupHead groupHead : groups.values()) {
       for (int i = 0; i < groupHead.comparators.length; i++) {
         groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
       }
@@ -117,13 +117,13 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
   /** Holds current head document for a single group.
    *
    * @lucene.experimental */
-  public class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<MutableValue> {
+  public class FunctionGroupHead extends AllGroupHeadsCollector.GroupHead<MutableValue> {
 
     final FieldComparator<?>[] comparators;
     final LeafFieldComparator[] leafComparators;
 
     @SuppressWarnings({"unchecked","rawtypes"})
-    private GroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
+    private FunctionGroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
       super(groupValue, doc + readerContext.docBase);
       final SortField[] sortFields = sort.getSort();
       comparators = new FieldComparator[sortFields.length];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
index 58418ed..1609d4d 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
@@ -19,7 +19,7 @@ package org.apache.lucene.search.grouping.function;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
 import org.apache.lucene.util.mutable.MutableValue;
 
 import java.io.IOException;
@@ -39,7 +39,7 @@ import java.util.TreeSet;
  *
  * @lucene.experimental
  */
-public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<MutableValue> {
+public class FunctionAllGroupsCollector extends AllGroupsCollector<MutableValue> {
 
   private final Map<?, ?> vsContext;
   private final ValueSource groupBy;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
index 32e9324..69ead07 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
@@ -16,27 +16,31 @@
  */
 package org.apache.lucene.search.grouping.function;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
+import org.apache.lucene.search.grouping.DistinctValuesCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.mutable.MutableValue;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
- * Function based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector}.
+ * Function based implementation of {@link DistinctValuesCollector}.
  *
  * @lucene.experimental
  */
-public class FunctionDistinctValuesCollector extends AbstractDistinctValuesCollector<FunctionDistinctValuesCollector.GroupCount> {
+public class FunctionDistinctValuesCollector extends DistinctValuesCollector<MutableValue> {
 
   private final Map<?, ?> vsContext;
   private final ValueSource groupSource;
   private final ValueSource countSource;
-  private final Map<MutableValue, GroupCount> groupMap;
+  private final Map<MutableValue, GroupCount<MutableValue>> groupMap;
 
   private FunctionValues.ValueFiller groupFiller;
   private FunctionValues.ValueFiller countFiller;
@@ -49,19 +53,19 @@ public class FunctionDistinctValuesCollector extends AbstractDistinctValuesColle
     this.countSource = countSource;
     groupMap = new LinkedHashMap<>();
     for (SearchGroup<MutableValue> group : groups) {
-      groupMap.put(group.groupValue, new GroupCount(group.groupValue));
+      groupMap.put(group.groupValue, new GroupCount<>(group.groupValue));
     }
   }
 
   @Override
-  public List<GroupCount> getGroups() {
+  public List<GroupCount<MutableValue>> getGroups() {
     return new ArrayList<>(groupMap.values());
   }
 
   @Override
   public void collect(int doc) throws IOException {
     groupFiller.fillValue(doc);
-    GroupCount groupCount = groupMap.get(groupMval);
+    GroupCount<MutableValue> groupCount = groupMap.get(groupMval);
     if (groupCount != null) {
       countFiller.fillValue(doc);
       groupCount.uniqueValues.add(countMval.duplicate());
@@ -78,15 +82,4 @@ public class FunctionDistinctValuesCollector extends AbstractDistinctValuesColle
     countMval = countFiller.getValue();
   }
 
-  /** Holds distinct values for a single group.
-   *
-   * @lucene.experimental */
-  public static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<MutableValue> {
-
-    GroupCount(MutableValue groupValue) {
-      super(groupValue);
-    }
-
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
index 0237e21..85376e6 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
@@ -20,19 +20,19 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
 import org.apache.lucene.util.mutable.MutableValue;
 
 import java.io.IOException;
 import java.util.Map;
 
 /**
- * Concrete implementation of {@link AbstractFirstPassGroupingCollector} that groups based on
+ * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
  * {@link ValueSource} instances.
  *
  * @lucene.experimental
  */
-public class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector<MutableValue> {
+public class FunctionFirstPassGroupingCollector extends FirstPassGroupingCollector<MutableValue> {
 
   private final ValueSource groupByVS;
   private final Map<?, ?> vsContext;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
new file mode 100644
index 0000000..5204dc2
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping.function;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.Grouper;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.util.mutable.MutableValue;
+
+/**
+ * Collector factory for grouping by ValueSource
+ */
+public class FunctionGrouper extends Grouper<MutableValue> {
+
+  private final ValueSource valueSource;
+  private final Map<?, ?> context;
+
+  /**
+   * Create a Grouper for the provided ValueSource and context
+   */
+  public FunctionGrouper(ValueSource valueSource, Map<?, ?> context) {
+    this.valueSource = valueSource;
+    this.context = context;
+  }
+
+  @Override
+  public FirstPassGroupingCollector<MutableValue> getFirstPassCollector(Sort sort, int count) throws IOException {
+    return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count);
+  }
+
+  @Override
+  public AllGroupHeadsCollector<MutableValue> getGroupHeadsCollector(Sort sort) {
+    return new FunctionAllGroupHeadsCollector(valueSource, context, sort);
+  }
+
+  @Override
+  public AllGroupsCollector<MutableValue> getAllGroupsCollector() {
+    return new FunctionAllGroupsCollector(valueSource, context);
+  }
+
+  @Override
+  public SecondPassGroupingCollector<MutableValue> getSecondPassCollector(Collection<SearchGroup<MutableValue>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+    return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, valueSource, context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
index d363267..45f2b37 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
@@ -20,7 +20,7 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.search.grouping.TopGroups; //javadoc
@@ -30,12 +30,12 @@ import java.util.Collection;
 import java.util.Map;
 
 /**
- * Concrete implementation of {@link AbstractSecondPassGroupingCollector} that groups based on
+ * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
  * {@link ValueSource} instances.
  *
  * @lucene.experimental
  */
-public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector<MutableValue> {
+public class FunctionSecondPassGroupingCollector extends SecondPassGroupingCollector<MutableValue> {
 
   private final ValueSource groupByVS;
   private final Map<?, ?> vsContext;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
index 14636ec..54e2399 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
@@ -31,19 +31,19 @@ import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.SentinelIntSet;
 
 /**
- * A base implementation of {@link org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
+ * A base implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
  * on a string based group field. More specifically this all concrete implementations of this base implementation
  * use {@link SortedDocValues}.
  *
  * @lucene.experimental
  */
-public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead<?>> extends AbstractAllGroupHeadsCollector<GH> {
+public abstract class TermAllGroupHeadsCollector extends AllGroupHeadsCollector<BytesRef> {
 
   private static final int DEFAULT_INITIAL_SIZE = 128;
 
@@ -67,7 +67,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
    * @param sortWithinGroup The sort within each group
    * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
    */
-  public static AbstractAllGroupHeadsCollector<?> create(String groupField, Sort sortWithinGroup) {
+  public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup) {
     return create(groupField, sortWithinGroup, DEFAULT_INITIAL_SIZE);
   }
 
@@ -82,7 +82,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
    *                    4 bytes * initialSize.
    * @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
    */
-  public static AbstractAllGroupHeadsCollector<?> create(String groupField, Sort sortWithinGroup, int initialSize) {
+  public static AllGroupHeadsCollector<BytesRef> create(String groupField, Sort sortWithinGroup, int initialSize) {
     boolean sortAllScore = true;
     boolean sortAllFieldValue = true;
 
@@ -113,7 +113,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
   }
 
   // A general impl that works for any group sort.
-  static class GeneralAllGroupHeadsCollector extends TermAllGroupHeadsCollector<GeneralAllGroupHeadsCollector.GroupHead> {
+  static class GeneralAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
 
     private final Sort sortWithinGroup;
     private final Map<BytesRef, GroupHead> groups;
@@ -199,7 +199,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
       }
     }
 
-    class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
+    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
 
       @SuppressWarnings({"unchecked", "rawtypes"})
       final FieldComparator[] comparators;
@@ -239,7 +239,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
 
 
   // AbstractAllGroupHeadsCollector optimized for ord fields and scores.
-  static class OrdScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector<OrdScoreAllGroupHeadsCollector.GroupHead> {
+  static class OrdScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
 
     private final SentinelIntSet ordSet;
     private final List<GroupHead> collectedGroups;
@@ -365,7 +365,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
       }
     }
 
-    class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
+    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
 
       BytesRefBuilder[] sortValues;
       int[] sortOrds;
@@ -452,7 +452,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
 
 
   // AbstractAllGroupHeadsCollector optimized for ord fields.
-  static class OrdAllGroupHeadsCollector extends TermAllGroupHeadsCollector<OrdAllGroupHeadsCollector.GroupHead> {
+  static class OrdAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
 
     private final SentinelIntSet ordSet;
     private final List<GroupHead> collectedGroups;
@@ -566,7 +566,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
       }
     }
 
-    class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
+    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
 
       BytesRefBuilder[] sortValues;
       int[] sortOrds;
@@ -635,7 +635,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
 
 
   // AbstractAllGroupHeadsCollector optimized for scores.
-  static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector<ScoreAllGroupHeadsCollector.GroupHead> {
+  static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
 
     final SentinelIntSet ordSet;
     final List<GroupHead> collectedGroups;
@@ -727,7 +727,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
       }
     }
 
-    class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
+    class GroupHead extends AllGroupHeadsCollector.GroupHead<BytesRef> {
 
       float[] scores;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
index 0009c36..125555a 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SentinelIntSet;
 
@@ -42,7 +42,7 @@ import org.apache.lucene.util.SentinelIntSet;
  *
  * @lucene.experimental
  */
-public class TermAllGroupsCollector extends AbstractAllGroupsCollector<BytesRef> {
+public class TermAllGroupsCollector extends AllGroupsCollector<BytesRef> {
 
   private static final int DEFAULT_INITIAL_SIZE = 128;
 
@@ -53,7 +53,7 @@ public class TermAllGroupsCollector extends AbstractAllGroupsCollector<BytesRef>
   private SortedDocValues index;
 
   /**
-   * Expert: Constructs a {@link AbstractAllGroupsCollector}
+   * Expert: Constructs a {@link AllGroupsCollector}
    *
    * @param groupField  The field to group by
    * @param initialSize The initial allocation size of the
@@ -69,7 +69,7 @@ public class TermAllGroupsCollector extends AbstractAllGroupsCollector<BytesRef>
   }
 
   /**
-   * Constructs a {@link AbstractAllGroupsCollector}. This sets the
+   * Constructs a {@link AllGroupsCollector}. This sets the
    * initial allocation size for the internal int set and group
    * list to 128.
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
index 7980c48..e5356a3 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
@@ -25,24 +25,24 @@ import java.util.List;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
+import org.apache.lucene.search.grouping.DistinctValuesCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SentinelIntSet;
 
 /**
- * A term based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector} that relies
+ * A term based implementation of {@link DistinctValuesCollector} that relies
  * on {@link SortedDocValues} to count the distinct values per group.
  *
  * @lucene.experimental
  */
-public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector<TermDistinctValuesCollector.GroupCount> {
+public class TermDistinctValuesCollector extends DistinctValuesCollector<BytesRef> {
 
   private final String groupField;
   private final String countField;
-  private final List<GroupCount> groups;
+  private final List<TermGroupCount> groups;
   private final SentinelIntSet ordSet;
-  private final GroupCount groupCounts[];
+  private final TermGroupCount groupCounts[];
 
   private SortedDocValues groupFieldTermIndex;
   private SortedDocValues countFieldTermIndex;
@@ -59,10 +59,10 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
     this.countField = countField;
     this.groups = new ArrayList<>(groups.size());
     for (SearchGroup<BytesRef> group : groups) {
-      this.groups.add(new GroupCount(group.groupValue));
+      this.groups.add(new TermGroupCount(group.groupValue));
     }
     ordSet = new SentinelIntSet(groups.size(), -2);
-    groupCounts = new GroupCount[ordSet.keys.length];
+    groupCounts = new TermGroupCount[ordSet.keys.length];
   }
 
   @Override
@@ -81,7 +81,7 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
       return;
     }
 
-    GroupCount gc = groupCounts[slot];
+    TermGroupCount gc = groupCounts[slot];
     if (doc > countFieldTermIndex.docID()) {
       countFieldTermIndex.advance(doc);
     }
@@ -119,8 +119,8 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
   }
 
   @Override
-  public List<GroupCount> getGroups() {
-    return groups;
+  public List<GroupCount<BytesRef>> getGroups() {
+    return new ArrayList<>(groups);
   }
 
   @Override
@@ -128,7 +128,7 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
     groupFieldTermIndex = DocValues.getSorted(context.reader(), groupField);
     countFieldTermIndex = DocValues.getSorted(context.reader(), countField);
     ordSet.clear();
-    for (GroupCount group : groups) {
+    for (TermGroupCount group : groups) {
       int groupOrd = group.groupValue == null ? -1 : groupFieldTermIndex.lookupTerm(group.groupValue);
       if (group.groupValue != null && groupOrd < 0) {
         continue;
@@ -150,11 +150,11 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
   /** Holds distinct values for a single group.
    *
    * @lucene.experimental */
-  public static class GroupCount extends AbstractDistinctValuesCollector.GroupCount<BytesRef> {
+  public static class TermGroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
 
     int[] ords;
 
-    GroupCount(BytesRef groupValue) {
+    TermGroupCount(BytesRef groupValue) {
       super(groupValue);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
index 063521f..3c35fa8 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
@@ -22,18 +22,18 @@ import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector} that groups based on
+ * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
  * field values and more specifically uses {@link SortedDocValues}
  * to collect groups.
  *
  * @lucene.experimental
  */
-public class TermFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector<BytesRef> {
+public class TermFirstPassGroupingCollector extends FirstPassGroupingCollector<BytesRef> {
 
   private SortedDocValues index;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
index efa75d6..cee327c 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
@@ -25,19 +25,19 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.grouping.AbstractGroupFacetCollector;
+import org.apache.lucene.search.grouping.GroupFacetCollector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.SentinelIntSet;
 import org.apache.lucene.util.UnicodeUtil;
 
 /**
- * An implementation of {@link AbstractGroupFacetCollector} that computes grouped facets based on the indexed terms
+ * An implementation of {@link GroupFacetCollector} that computes grouped facets based on the indexed terms
  * from DocValues.
  *
  * @lucene.experimental
  */
-public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollector {
+public abstract class TermGroupFacetCollector extends GroupFacetCollector {
 
   final List<GroupedFacetHit> groupedFacetHits;
   final SentinelIntSet segmentGroupedFacetHits;
@@ -190,7 +190,7 @@ public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollecto
       return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.termsEnum(), startFacetOrd, endFacetOrd);
     }
 
-    private static class SegmentResult extends AbstractGroupFacetCollector.SegmentResult {
+    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
 
       final TermsEnum tenum;
 
@@ -380,7 +380,7 @@ public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollecto
       return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldNumTerms, facetOrdTermsEnum, startFacetOrd, endFacetOrd);
     }
 
-    private static class SegmentResult extends AbstractGroupFacetCollector.SegmentResult {
+    private static class SegmentResult extends GroupFacetCollector.SegmentResult {
 
       final TermsEnum tenum;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
new file mode 100644
index 0000000..246ee78
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping.term;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.Grouper;
+import org.apache.lucene.search.grouping.SearchGroup;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Collector factory for grouping by term
+ */
+public class TermGrouper extends Grouper<BytesRef> {
+
+  private final String field;
+  private final int initialSize;
+
+  /**
+   * Create a new TermGrouper
+   * @param field the field to group on
+   */
+  public TermGrouper(String field) {
+    this(field, 128);
+  }
+
+  /**
+   * Create a new TermGrouper
+   * @param field       the field to group on
+   * @param initialSize the initial size of various internal datastructures
+   */
+  public TermGrouper(String field, int initialSize) {
+    this.field = field;
+    this.initialSize = initialSize;
+  }
+
+  @Override
+  public FirstPassGroupingCollector<BytesRef> getFirstPassCollector(Sort sort, int count) throws IOException {
+    return new TermFirstPassGroupingCollector(field, sort, count);
+  }
+
+  @Override
+  public AllGroupHeadsCollector<BytesRef> getGroupHeadsCollector(Sort sort) {
+    return TermAllGroupHeadsCollector.create(field, sort, initialSize);
+  }
+
+  @Override
+  public AllGroupsCollector<BytesRef> getAllGroupsCollector() {
+    return new TermAllGroupsCollector(field, initialSize);
+  }
+
+  @Override
+  public SecondPassGroupingCollector<BytesRef> getSecondPassCollector(
+      Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
+      int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
+    return new TermSecondPassGroupingCollector(field, groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
index c7b16e9..75d2210 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
@@ -23,19 +23,19 @@ import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SentinelIntSet;
 
 /**
- * Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector} that groups based on
+ * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
  * field values and more specifically uses {@link SortedDocValues}
  * to collect grouped docs.
  *
  * @lucene.experimental
  */
-public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector<BytesRef> {
+public class TermSecondPassGroupingCollector extends SecondPassGroupingCollector<BytesRef> {
 
   private final String groupField;
   private final SentinelIntSet ordSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
index 316676b..0c99456 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
@@ -52,6 +52,7 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
@@ -138,7 +139,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
     int maxDoc = reader.maxDoc();
 
     Sort sortWithinGroup = new Sort(new SortField("id_1", SortField.Type.INT, true));
-    AbstractAllGroupHeadsCollector<?> allGroupHeadsCollector = createRandomCollector(groupField, sortWithinGroup);
+    AllGroupHeadsCollector<?> allGroupHeadsCollector = createRandomCollector(groupField, sortWithinGroup);
     indexSearcher.search(new TermQuery(new Term("content", "random")), allGroupHeadsCollector);
     assertTrue(arrayContains(new int[]{2, 3, 5, 7}, allGroupHeadsCollector.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, allGroupHeadsCollector.retrieveGroupHeads(maxDoc), maxDoc));
@@ -326,7 +327,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
         final String searchTerm = "real" + random().nextInt(3);
         boolean sortByScoreOnly = random().nextBoolean();
         Sort sortWithinGroup = getRandomSort(sortByScoreOnly);
-        AbstractAllGroupHeadsCollector<?> allGroupHeadsCollector = createRandomCollector("group", sortWithinGroup);
+        AllGroupHeadsCollector<?> allGroupHeadsCollector = createRandomCollector("group", sortWithinGroup);
         s.search(new TermQuery(new Term("content", searchTerm)), allGroupHeadsCollector);
         int[] expectedGroupHeads = createExpectedGroupHeads(searchTerm, groupDocs, sortWithinGroup, sortByScoreOnly, fieldIdToDocID);
         int[] actualGroupHeads = allGroupHeadsCollector.retrieveGroupHeads();
@@ -402,8 +403,9 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
     return true;
   }
 
-  private boolean openBitSetContains(int[] expectedDocs, FixedBitSet actual, int maxDoc) throws IOException {
-    if (expectedDocs.length != actual.cardinality()) {
+  private boolean openBitSetContains(int[] expectedDocs, Bits actual, int maxDoc) throws IOException {
+    assert actual instanceof FixedBitSet;
+    if (expectedDocs.length != ((FixedBitSet)actual).cardinality()) {
       return false;
     }
 
@@ -510,8 +512,8 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private AbstractAllGroupHeadsCollector<?> createRandomCollector(String groupField, Sort sortWithinGroup) {
-    AbstractAllGroupHeadsCollector<? extends AbstractAllGroupHeadsCollector.GroupHead> collector;
+  private AllGroupHeadsCollector<?> createRandomCollector(String groupField, Sort sortWithinGroup) {
+    AllGroupHeadsCollector<?> collector;
     if (random().nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
       collector =  new FunctionAllGroupHeadsCollector(vs, new HashMap<>(), sortWithinGroup);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
index 30a147e..ab70fad 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
@@ -102,7 +102,7 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
     IndexSearcher indexSearcher = newSearcher(w.getReader());
     w.close();
 
-    AbstractAllGroupsCollector<?> allGroupsCollector = createRandomCollector(groupField);
+    AllGroupsCollector<?> allGroupsCollector = createRandomCollector(groupField);
     indexSearcher.search(new TermQuery(new Term("content", "random")), allGroupsCollector);
     assertEquals(4, allGroupsCollector.getGroupCount());
 
@@ -123,8 +123,8 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
     doc.add(new SortedDocValuesField(groupField, new BytesRef(value)));
   }
 
-  private AbstractAllGroupsCollector<?> createRandomCollector(String groupField) {
-    AbstractAllGroupsCollector<?> selected;
+  private AllGroupsCollector<?> createRandomCollector(String groupField) {
+    AllGroupsCollector<?> selected;
     if (random().nextBoolean()) {
       selected = new TermAllGroupsCollector(groupField);
     } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
index 313e92f..ba43ca8 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/DistinctValuesCollectorTest.java
@@ -126,10 +126,10 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     IndexSearcher indexSearcher = newSearcher(w.getReader());
     w.close();
 
-    Comparator<AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> cmp = new Comparator<AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>>() {
+    Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>> cmp = new Comparator<DistinctValuesCollector.GroupCount<Comparable<Object>>>() {
 
       @Override
-      public int compare(AbstractDistinctValuesCollector.GroupCount<Comparable<Object>> groupCount1, AbstractDistinctValuesCollector.GroupCount<Comparable<Object>> groupCount2) {
+      public int compare(DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount1, DistinctValuesCollector.GroupCount<Comparable<Object>> groupCount2) {
         if (groupCount1.groupValue == null) {
           if (groupCount2.groupValue == null) {
             return 0;
@@ -145,13 +145,13 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     };
 
     // === Search for content:random
-    AbstractFirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(new Sort(), groupField, 10);
+    FirstPassGroupingCollector<Comparable<Object>> firstCollector = createRandomFirstPassCollector(new Sort(), groupField, 10);
     indexSearcher.search(new TermQuery(new Term("content", "random")), firstCollector);
-    AbstractDistinctValuesCollector<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> distinctValuesCollector
+    DistinctValuesCollector<Comparable<Object>> distinctValuesCollector
         = createDistinctCountCollector(firstCollector, groupField, countField);
     indexSearcher.search(new TermQuery(new Term("content", "random")), distinctValuesCollector);
 
-    List<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
+    List<DistinctValuesCollector.GroupCount<Comparable<Object>>> gcs = distinctValuesCollector.getGroups();
     Collections.sort(gcs, cmp);
     assertEquals(4, gcs.size());
 
@@ -240,15 +240,15 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
         Sort groupSort = new Sort(new SortField("id", SortField.Type.STRING));
         int topN = 1 + random.nextInt(10);
 
-        List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
+        List<DistinctValuesCollector.GroupCount<Comparable<?>>> expectedResult = createExpectedResult(context, term, groupSort, topN);
 
-        AbstractFirstPassGroupingCollector<Comparable<?>> firstCollector = createRandomFirstPassCollector(groupSort, groupField, topN);
+        FirstPassGroupingCollector<Comparable<?>> firstCollector = createRandomFirstPassCollector(groupSort, groupField, topN);
         searcher.search(new TermQuery(new Term("content", term)), firstCollector);
-        AbstractDistinctValuesCollector<? extends AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> distinctValuesCollector
+        DistinctValuesCollector<Comparable<?>> distinctValuesCollector
             = createDistinctCountCollector(firstCollector, groupField, countField);
         searcher.search(new TermQuery(new Term("content", term)), distinctValuesCollector);
         @SuppressWarnings("unchecked")
-        List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> actualResult = (List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>>) distinctValuesCollector.getGroups();
+        List<DistinctValuesCollector.GroupCount<Comparable<?>>> actualResult = distinctValuesCollector.getGroups();
 
         if (VERBOSE) {
           System.out.println("Index iter=" + indexIter);
@@ -265,8 +265,8 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
 
         assertEquals(expectedResult.size(), actualResult.size());
         for (int i = 0; i < expectedResult.size(); i++) {
-          AbstractDistinctValuesCollector.GroupCount<Comparable<?>> expected = expectedResult.get(i);
-          AbstractDistinctValuesCollector.GroupCount<Comparable<?>> actual = actualResult.get(i);
+          DistinctValuesCollector.GroupCount<Comparable<?>> expected = expectedResult.get(i);
+          DistinctValuesCollector.GroupCount<Comparable<?>> actual = actualResult.get(i);
           assertValues(expected.groupValue, actual.groupValue);
           assertEquals(expected.uniqueValues.size(), actual.uniqueValues.size());
           List<Comparable<?>> expectedUniqueValues = new ArrayList<>(expected.uniqueValues);
@@ -283,9 +283,9 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
     }
   }
 
-  private void printGroups(List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> results) {
+  private void printGroups(List<? extends DistinctValuesCollector.GroupCount<Comparable<?>>> results) {
     for(int i=0;i<results.size();i++) {
-      AbstractDistinctValuesCollector.GroupCount<Comparable<?>> group = results.get(i);
+      DistinctValuesCollector.GroupCount<Comparable<?>> group = results.get(i);
       Object gv = group.groupValue;
       if (gv instanceof BytesRef) {
         System.out.println(i + ": groupValue=" + ((BytesRef) gv).utf8ToString());
@@ -350,31 +350,31 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase {
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private <T extends Comparable> AbstractDistinctValuesCollector<AbstractDistinctValuesCollector.GroupCount<T>> createDistinctCountCollector(AbstractFirstPassGroupingCollector<T> firstPassGroupingCollector,
-                                                                      String groupField,
-                                                                      String countField) throws IOException {
+  private <T extends Comparable> DistinctValuesCollector<T> createDistinctCountCollector(FirstPassGroupingCollector<T> firstPassGroupingCollector,
+                                                                                                                             String groupField,
+                                                                                                                             String countField) throws IOException {
     Random random = random();
     Collection<SearchGroup<T>> searchGroups = firstPassGroupingCollector.getTopGroups(0, false);
     if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
-      return (AbstractDistinctValuesCollector) new FunctionDistinctValuesCollector(new HashMap<>(), new BytesRefFieldSource(groupField), new BytesRefFieldSource(countField), (Collection) searchGroups);
+      return (DistinctValuesCollector) new FunctionDistinctValuesCollector(new HashMap<>(), new BytesRefFieldSource(groupField), new BytesRefFieldSource(countField), (Collection) searchGroups);
     } else {
-      return (AbstractDistinctValuesCollector) new TermDistinctValuesCollector(groupField, countField, (Collection) searchGroups);
+      return (DistinctValuesCollector) new TermDistinctValuesCollector(groupField, countField, (Collection) searchGroups);
     }
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private <T> AbstractFirstPassGroupingCollector<T> createRandomFirstPassCollector(Sort groupSort, String groupField, int topNGroups) throws IOException {
+  private <T> FirstPassGroupingCollector<T> createRandomFirstPassCollector(Sort groupSort, String groupField, int topNGroups) throws IOException {
     Random random = random();
     if (random.nextBoolean()) {
-      return (AbstractFirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<>(), groupSort, topNGroups);
+      return (FirstPassGroupingCollector<T>) new FunctionFirstPassGroupingCollector(new BytesRefFieldSource(groupField), new HashMap<>(), groupSort, topNGroups);
     } else {
-      return (AbstractFirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
+      return (FirstPassGroupingCollector<T>) new TermFirstPassGroupingCollector(groupField, groupSort, topNGroups);
     }
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private List<AbstractDistinctValuesCollector.GroupCount<Comparable<?>>> createExpectedResult(IndexContext context,  String term, Sort groupSort, int topN) {
-    class GroupCount extends AbstractDistinctValuesCollector.GroupCount<BytesRef> {
+  private List<DistinctValuesCollector.GroupCount<Comparable<?>>> createExpectedResult(IndexContext context, String term, Sort groupSort, int topN) {
+    class GroupCount extends DistinctValuesCollector.GroupCount<BytesRef> {
       GroupCount(BytesRef groupValue, Collection<BytesRef> uniqueValues) {
         super(groupValue);
         this.uniqueValues.addAll(uniqueValues);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
index 82bbd39..46141bc 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java
@@ -103,7 +103,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
     IndexSearcher indexSearcher = newSearcher(w.getReader());
 
     List<TermGroupFacetCollector.FacetEntry> entries;
-    AbstractGroupFacetCollector groupedAirportFacetCollector;
+    GroupFacetCollector groupedAirportFacetCollector;
     TermGroupFacetCollector.GroupedFacetResult airportResult;
     
     for (int limit : new int[] { 2, 10, 100, Integer.MAX_VALUE }) {
@@ -136,7 +136,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
       assertEquals(1, entries.get(0).getCount());
     }
 
-    AbstractGroupFacetCollector groupedDurationFacetCollector = createRandomCollector(useDv ? "hotel_dv" : "hotel", useDv ? "duration_dv" : "duration", null, false);
+    GroupFacetCollector groupedDurationFacetCollector = createRandomCollector(useDv ? "hotel_dv" : "hotel", useDv ? "duration_dv" : "duration", null, false);
     indexSearcher.search(new MatchAllDocsQuery(), groupedDurationFacetCollector);
     TermGroupFacetCollector.GroupedFacetResult durationResult = groupedDurationFacetCollector.mergeSegmentResults(10, 0, false);
     assertEquals(4, durationResult.getTotalCount());
@@ -344,7 +344,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
 
     w.close();
     IndexSearcher indexSearcher = newSearcher(DirectoryReader.open(dir));
-    AbstractGroupFacetCollector groupedAirportFacetCollector = createRandomCollector(groupField + "_dv", "airport", null, true);
+    GroupFacetCollector groupedAirportFacetCollector = createRandomCollector(groupField + "_dv", "airport", null, true);
     indexSearcher.search(new MatchAllDocsQuery(), groupedAirportFacetCollector);
     TermGroupFacetCollector.GroupedFacetResult airportResult = groupedAirportFacetCollector.mergeSegmentResults(10, 0, false);
     assertEquals(3, airportResult.getTotalCount());
@@ -404,7 +404,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
         }
 
         GroupedFacetResult expectedFacetResult = createExpectedFacetResult(searchTerm, context, offset, limit, minCount, orderByCount, facetPrefix);
-        AbstractGroupFacetCollector groupFacetCollector = createRandomCollector("group", "facet", facetPrefix, multipleFacetsPerDocument);
+        GroupFacetCollector groupFacetCollector = createRandomCollector("group", "facet", facetPrefix, multipleFacetsPerDocument);
         searcher.search(new TermQuery(new Term("content", searchTerm)), groupFacetCollector);
         TermGroupFacetCollector.GroupedFacetResult actualFacetResult = groupFacetCollector.mergeSegmentResults(size, minCount, orderByCount);
 
@@ -704,7 +704,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
     return new GroupedFacetResult(totalCount, totalMissCount, entriesResult);
   }
 
-  private AbstractGroupFacetCollector createRandomCollector(String groupField, String facetField, String facetPrefix, boolean multipleFacetsPerDocument) {
+  private GroupFacetCollector createRandomCollector(String groupField, String facetField, String facetPrefix, boolean multipleFacetsPerDocument) {
     BytesRef facetPrefixBR = facetPrefix == null ? null : new BytesRef(facetPrefix);
     return TermGroupFacetCollector.createTermGroupFacetCollector(groupField, facetField, multipleFacetsPerDocument, facetPrefixBR, random().nextInt(1024));
   }


[11/34] lucene-solr:jira/solr-9856: LUCENE-7617: Grouping collector API cleanup

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
----------------------------------------------------------------------
diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
index 72c858f..f079b85 100644
--- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
+++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
@@ -144,10 +144,10 @@ public class TestGrouping extends LuceneTestCase {
 
     final Sort groupSort = Sort.RELEVANCE;
 
-    final AbstractFirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
+    final FirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, 10);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c1);
 
-    final AbstractSecondPassGroupingCollector<?> c2 = createSecondPassCollector(c1, groupField, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
+    final SecondPassGroupingCollector<?> c2 = createSecondPassCollector(c1, groupField, groupSort, Sort.RELEVANCE, 0, 5, true, true, true);
     indexSearcher.search(new TermQuery(new Term("content", "random")), c2);
 
     final TopGroups<?> groups = c2.getTopGroups(0);
@@ -195,8 +195,8 @@ public class TestGrouping extends LuceneTestCase {
     doc.add(new SortedDocValuesField(groupField, new BytesRef(value)));
   }
 
-  private AbstractFirstPassGroupingCollector<?> createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
-    AbstractFirstPassGroupingCollector<?> selected;
+  private FirstPassGroupingCollector<?> createRandomFirstPassCollector(String groupField, Sort groupSort, int topDocs) throws IOException {
+    FirstPassGroupingCollector<?> selected;
     if (random().nextBoolean()) {
       ValueSource vs = new BytesRefFieldSource(groupField);
       selected = new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
@@ -209,7 +209,7 @@ public class TestGrouping extends LuceneTestCase {
     return selected;
   }
 
-  private AbstractFirstPassGroupingCollector<?> createFirstPassCollector(String groupField, Sort groupSort, int topDocs, AbstractFirstPassGroupingCollector<?> firstPassGroupingCollector) throws IOException {
+  private FirstPassGroupingCollector<?> createFirstPassCollector(String groupField, Sort groupSort, int topDocs, FirstPassGroupingCollector<?> firstPassGroupingCollector) throws IOException {
     if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
       ValueSource vs = new BytesRefFieldSource(groupField);
       return new FunctionFirstPassGroupingCollector(vs, new HashMap<>(), groupSort, topDocs);
@@ -219,37 +219,37 @@ public class TestGrouping extends LuceneTestCase {
   }
 
   @SuppressWarnings({"unchecked","rawtypes"})
-  private <T> AbstractSecondPassGroupingCollector<T> createSecondPassCollector(AbstractFirstPassGroupingCollector firstPassGroupingCollector,
-                                                                        String groupField,
-                                                                        Sort groupSort,
-                                                                        Sort sortWithinGroup,
-                                                                        int groupOffset,
-                                                                        int maxDocsPerGroup,
-                                                                        boolean getScores,
-                                                                        boolean getMaxScores,
-                                                                        boolean fillSortFields) throws IOException {
+  private <T> SecondPassGroupingCollector<T> createSecondPassCollector(FirstPassGroupingCollector firstPassGroupingCollector,
+                                                                       String groupField,
+                                                                       Sort groupSort,
+                                                                       Sort sortWithinGroup,
+                                                                       int groupOffset,
+                                                                       int maxDocsPerGroup,
+                                                                       boolean getScores,
+                                                                       boolean getMaxScores,
+                                                                       boolean fillSortFields) throws IOException {
 
     if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
       Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
-      return (AbstractSecondPassGroupingCollector) new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
+      return (SecondPassGroupingCollector) new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
     } else {
       ValueSource vs = new BytesRefFieldSource(groupField);
       Collection<SearchGroup<MutableValue>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
-      return (AbstractSecondPassGroupingCollector) new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
+      return (SecondPassGroupingCollector) new FunctionSecondPassGroupingCollector(searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, vs, new HashMap());
     }
   }
 
   // Basically converts searchGroups from MutableValue to BytesRef if grouping by ValueSource
   @SuppressWarnings("unchecked")
-  private AbstractSecondPassGroupingCollector<?> createSecondPassCollector(AbstractFirstPassGroupingCollector<?> firstPassGroupingCollector,
-                                                                        String groupField,
-                                                                        Collection<SearchGroup<BytesRef>> searchGroups,
-                                                                        Sort groupSort,
-                                                                        Sort sortWithinGroup,
-                                                                        int maxDocsPerGroup,
-                                                                        boolean getScores,
-                                                                        boolean getMaxScores,
-                                                                        boolean fillSortFields) throws IOException {
+  private SecondPassGroupingCollector<?> createSecondPassCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
+                                                                   String groupField,
+                                                                   Collection<SearchGroup<BytesRef>> searchGroups,
+                                                                   Sort groupSort,
+                                                                   Sort sortWithinGroup,
+                                                                   int maxDocsPerGroup,
+                                                                   boolean getScores,
+                                                                   boolean getMaxScores,
+                                                                   boolean fillSortFields) throws IOException {
     if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
       return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
     } else {
@@ -272,8 +272,8 @@ public class TestGrouping extends LuceneTestCase {
     }
   }
 
-  private AbstractAllGroupsCollector<?> createAllGroupsCollector(AbstractFirstPassGroupingCollector<?> firstPassGroupingCollector,
-                                                              String groupField) {
+  private AllGroupsCollector<?> createAllGroupsCollector(FirstPassGroupingCollector<?> firstPassGroupingCollector,
+                                                         String groupField) {
     if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
       return new TermAllGroupsCollector(groupField);
     } else {
@@ -305,7 +305,7 @@ public class TestGrouping extends LuceneTestCase {
     }
   }
 
-  private Collection<SearchGroup<BytesRef>> getSearchGroups(AbstractFirstPassGroupingCollector<?> c, int groupOffset, boolean fillFields) throws IOException {
+  private Collection<SearchGroup<BytesRef>> getSearchGroups(FirstPassGroupingCollector<?> c, int groupOffset, boolean fillFields) throws IOException {
     if (TermFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
       return ((TermFirstPassGroupingCollector) c).getTopGroups(groupOffset, fillFields);
     } else if (FunctionFirstPassGroupingCollector.class.isAssignableFrom(c.getClass())) {
@@ -328,7 +328,7 @@ public class TestGrouping extends LuceneTestCase {
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
-  private TopGroups<BytesRef> getTopGroups(AbstractSecondPassGroupingCollector c, int withinGroupOffset) {
+  private TopGroups<BytesRef> getTopGroups(SecondPassGroupingCollector c, int withinGroupOffset) {
     if (c.getClass().isAssignableFrom(TermSecondPassGroupingCollector.class)) {
       return ((TermSecondPassGroupingCollector) c).getTopGroups(withinGroupOffset);
     } else if (c.getClass().isAssignableFrom(FunctionSecondPassGroupingCollector.class)) {
@@ -874,11 +874,11 @@ public class TestGrouping extends LuceneTestCase {
         if (VERBOSE) {
           System.out.println("  groupField=" + groupField);
         }
-        final AbstractFirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, groupOffset+topNGroups);
+        final FirstPassGroupingCollector<?> c1 = createRandomFirstPassCollector(groupField, groupSort, groupOffset+topNGroups);
         final CachingCollector cCache;
         final Collector c;
         
-        final AbstractAllGroupsCollector<?> allGroupsCollector;
+        final AllGroupsCollector<?> allGroupsCollector;
         if (doAllGroups) {
           allGroupsCollector = createAllGroupsCollector(c1, groupField);
         } else {
@@ -953,7 +953,7 @@ public class TestGrouping extends LuceneTestCase {
         
         final TopGroups<BytesRef> topGroupsShards = searchShards(s, shards.subSearchers, query, groupSort, docSort,
             groupOffset, topNGroups, docOffset, docsPerGroup, getScores, getMaxScores, true, false);
-        final AbstractSecondPassGroupingCollector<?> c2;
+        final SecondPassGroupingCollector<?> c2;
         if (topGroups != null) {
           
           if (VERBOSE) {
@@ -1172,8 +1172,8 @@ public class TestGrouping extends LuceneTestCase {
     // Run 1st pass collector to get top groups per shard
     final Weight w = topSearcher.createNormalizedWeight(query, getScores);
     final List<Collection<SearchGroup<BytesRef>>> shardGroups = new ArrayList<>();
-    List<AbstractFirstPassGroupingCollector<?>> firstPassGroupingCollectors = new ArrayList<>();
-    AbstractFirstPassGroupingCollector<?> firstPassCollector = null;
+    List<FirstPassGroupingCollector<?>> firstPassGroupingCollectors = new ArrayList<>();
+    FirstPassGroupingCollector<?> firstPassCollector = null;
     boolean shardsCanUseIDV = canUseIDV;
 
     String groupField = "group";
@@ -1223,7 +1223,7 @@ public class TestGrouping extends LuceneTestCase {
       @SuppressWarnings({"unchecked","rawtypes"})
       final TopGroups<BytesRef>[] shardTopGroups = new TopGroups[subSearchers.length];
       for(int shardIDX=0;shardIDX<subSearchers.length;shardIDX++) {
-        final AbstractSecondPassGroupingCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
+        final SecondPassGroupingCollector<?> secondPassCollector = createSecondPassCollector(firstPassGroupingCollectors.get(shardIDX),
             groupField, mergedTopGroups, groupSort, docSort, docOffset + topNDocs, getScores, getMaxScores, true);
         subSearchers[shardIDX].search(w, secondPassCollector);
         shardTopGroups[shardIDX] = getTopGroups(secondPassCollector, 0);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index f29a767..641b1f3 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -51,7 +51,7 @@ import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.term.TermAllGroupsCollector;
 import org.apache.lucene.search.grouping.term.TermGroupFacetCollector;
 import org.apache.lucene.util.BytesRef;
@@ -282,7 +282,7 @@ public class SimpleFacets {
       } else {
         return base;
       }
-      AbstractAllGroupHeadsCollector allGroupHeadsCollector = grouping.getCommands().get(0).createAllGroupCollector();
+      AllGroupHeadsCollector allGroupHeadsCollector = grouping.getCommands().get(0).createAllGroupCollector();
       searcher.search(base.getTopFilter(), allGroupHeadsCollector);
       return new BitDocSet(allGroupHeadsCollector.retrieveGroupHeads(searcher.maxDoc()));
     } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/solr/core/src/java/org/apache/solr/search/Grouping.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java
index 8d6f3ca..75011e7 100644
--- a/solr/core/src/java/org/apache/solr/search/Grouping.java
+++ b/solr/core/src/java/org/apache/solr/search/Grouping.java
@@ -46,7 +46,7 @@ import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
-import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
@@ -323,7 +323,7 @@ public class Grouping {
       cmd.prepare();
     }
 
-    AbstractAllGroupHeadsCollector<?> allGroupHeadsCollector = null;
+    AllGroupHeadsCollector<?> allGroupHeadsCollector = null;
     List<Collector> collectors = new ArrayList<>(commands.size());
     for (Command cmd : commands) {
       Collector collector = cmd.createFirstPassCollector();
@@ -513,7 +513,7 @@ public class Grouping {
    * Note: Maybe the creating the response structure should be done in something like a ReponseBuilder???
    * Warning NOT thread save!
    */
-  public abstract class Command<GROUP_VALUE_TYPE> {
+  public abstract class Command<T> {
 
     public String key;       // the name to use for this group in the response
     public Sort withinGroupSort;   // the sort of the documents *within* a single group.
@@ -527,7 +527,7 @@ public class Grouping {
     public boolean main;     // use as the main result in simple format (grouped.main=true param)
     public TotalCount totalCount = TotalCount.ungrouped;
 
-    TopGroups<GROUP_VALUE_TYPE> result;
+    TopGroups<T> result;
 
 
     /**
@@ -565,7 +565,7 @@ public class Grouping {
      * @return a collector that is able to return the most relevant document of all groups.
      * @throws IOException If I/O related errors occur
      */
-    public AbstractAllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
+    public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
       return null;
     }
 
@@ -774,7 +774,7 @@ public class Grouping {
      * {@inheritDoc}
      */
     @Override
-    public AbstractAllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
+    public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
       Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
       return TermAllGroupHeadsCollector.create(groupBy, sortWithinGroup);
     }
@@ -992,7 +992,7 @@ public class Grouping {
     }
 
     @Override
-    public AbstractAllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
+    public AllGroupHeadsCollector<?> createAllGroupCollector() throws IOException {
       Sort sortWithinGroup = withinGroupSort != null ? withinGroupSort : Sort.RELEVANCE;
       return new FunctionAllGroupHeadsCollector(groupBy, context, sortWithinGroup);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
index 716f5df..74c2b70 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
@@ -31,7 +31,7 @@ import org.apache.lucene.search.MultiCollector;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TimeLimitingCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
-import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
+import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.term.TermAllGroupHeadsCollector;
 import org.apache.solr.common.util.NamedList;
@@ -170,7 +170,7 @@ public class CommandHandler {
     SchemaField sf = searcher.getSchema().getField(field);
     FieldType fieldType = sf.getType();
     
-    final AbstractAllGroupHeadsCollector allGroupHeadsCollector;
+    final AllGroupHeadsCollector allGroupHeadsCollector;
     if (fieldType.getNumericType() != null) {
       ValueSource vs = fieldType.getValueSource(sf, null);
       allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getSortWithinGroup());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
index fc04599..46f8009 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java
@@ -19,8 +19,8 @@ package org.apache.solr.search.grouping.distributed.command;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
-import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
+import org.apache.lucene.search.grouping.AllGroupsCollector;
+import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
 import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
@@ -81,8 +81,8 @@ public class SearchGroupsFieldCommand implements Command<SearchGroupsFieldComman
   private final int topNGroups;
   private final boolean includeGroupCount;
 
-  private AbstractFirstPassGroupingCollector firstPassGroupingCollector;
-  private AbstractAllGroupsCollector allGroupsCollector;
+  private FirstPassGroupingCollector firstPassGroupingCollector;
+  private AllGroupsCollector allGroupsCollector;
 
   private SearchGroupsFieldCommand(SchemaField field, Sort groupSort, int topNGroups, boolean includeGroupCount) {
     this.field = field;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da30f21f/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
index 2660b21..0bdb0ed 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
@@ -19,7 +19,7 @@ package org.apache.solr.search.grouping.distributed.command;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
+import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
@@ -106,7 +106,7 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
   private final int maxDocPerGroup;
   private final boolean needScores;
   private final boolean needMaxScore;
-  private AbstractSecondPassGroupingCollector secondPassCollector;
+  private SecondPassGroupingCollector secondPassCollector;
 
   private TopGroupsFieldCommand(SchemaField field,
                                 Sort groupSort,


[32/34] lucene-solr:jira/solr-9856: SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching

Posted by ab...@apache.org.
SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching


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

Branch: refs/heads/jira/solr-9856
Commit: b0177312032e039673bfbbd42cd1dca09fb93833
Parents: 322ad88
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Jan 9 08:33:06 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Jan 9 08:33:06 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                      | 2 ++
 solr/core/src/java/org/apache/solr/core/SolrCore.java | 4 ++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0177312/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 11151d7..49d24df 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -406,6 +406,8 @@ Other Changes
 
 * SOLR-9944: Map the nodes function name to the GatherNodesStream (Joel Bernstein)
 
+* SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching (Ishan Chattopadhyaya)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0177312/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 08072e1..697e008 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1687,7 +1687,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   public IndexFingerprint getIndexFingerprint(SolrIndexSearcher searcher, LeafReaderContext ctx, long maxVersion)
       throws IOException {
     IndexFingerprint f = null;
-    f = perSegmentFingerprintCache.get(ctx.reader().getCoreCacheKey());
+    f = perSegmentFingerprintCache.get(ctx.reader().getCombinedCoreAndDeletesKey());
     // fingerprint is either not cached or
     // if we want fingerprint only up to a version less than maxVersionEncountered in the segment, or
     // documents were deleted from segment for which fingerprint was cached
@@ -1698,7 +1698,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       // cache fingerprint for the segment only if all the versions in the segment are included in the fingerprint
       if (f.getMaxVersionEncountered() == f.getMaxInHash()) {
         log.info("Caching fingerprint for searcher:{} leafReaderContext:{} mavVersion:{}", searcher, ctx, maxVersion);
-        perSegmentFingerprintCache.put(ctx.reader().getCoreCacheKey(), f);
+        perSegmentFingerprintCache.put(ctx.reader().getCombinedCoreAndDeletesKey(), f);
       }
 
     } else {


[18/34] lucene-solr:jira/solr-9856: LUCENE-7610: Remove deprecated facet ValueSource methods

Posted by ab...@apache.org.
LUCENE-7610: Remove deprecated facet ValueSource methods


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

Branch: refs/heads/jira/solr-9856
Commit: ce8b678ba19a53c43033a235bdca54e5a68adcc8
Parents: e5f39f6
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 12:25:15 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 13:07:13 2017 +0000

----------------------------------------------------------------------
 .../apache/lucene/facet/range/DoubleRange.java  |  9 ---
 .../facet/range/DoubleRangeFacetCounts.java     | 23 ------
 .../apache/lucene/facet/range/LongRange.java    | 10 ---
 .../facet/range/LongRangeFacetCounts.java       | 21 ------
 .../taxonomy/TaxonomyFacetSumValueSource.java   | 75 +-------------------
 .../facet/range/TestRangeFacetCounts.java       |  8 +--
 .../TestTaxonomyFacetSumValueSource.java        | 49 +++----------
 7 files changed, 13 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index ce377f5..50e771d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -21,7 +21,6 @@ import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -178,14 +177,6 @@ public final class DoubleRange extends Range {
   }
 
   /**
-   * @deprecated Use {@link #getQuery(Query, DoubleValuesSource)}
-   */
-  @Deprecated
-  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
-    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asDoubleValuesSource());
-  }
-
-  /**
    * Create a Query that matches documents in this range
    *
    * The query will check all documents that match the provided match query,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 63fc935..5ed11a9 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -25,7 +25,6 @@ import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
@@ -61,15 +60,6 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
   }
 
   /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
-   * */
-  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
-    this(field, valueSource, hits, null, ranges);
-  }
-
-  /**
    * Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource}
    */
   public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
@@ -78,19 +68,6 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
   /**
    * Create {@code RangeFacetCounts}, using the provided
-   * {@link ValueSource}, and using the provided Query as
-   * a fastmatch: only documents matching the query are
-   * checked for the matching ranges.
-   *
-   * @deprecated Use ({@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, Query, DoubleRange...)}
-   */
-  @Deprecated
-  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
-    this(field, valueSource.asDoubleValuesSource(), hits, fastMatchQuery, ranges);
-  }
-
-  /**
-   * Create {@code RangeFacetCounts}, using the provided
    * {@link DoubleValuesSource}, and using the provided Query as
    * a fastmatch: only documents matching the query are
    * checked for the matching ranges.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 20c408d..582d76f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -21,7 +21,6 @@ import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -169,15 +168,6 @@ public final class LongRange extends Range {
 
   }
 
-
-  /**
-   * @deprecated Use {@link #getQuery(Query, LongValuesSource)}
-   */
-  @Deprecated
-  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
-    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asLongValuesSource());
-  }
-
   /**
    * Create a Query that matches documents in this range
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index a3cfc71..c9c42a3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -51,33 +51,12 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
     this(field, LongValuesSource.fromLongField(field), hits, ranges);
   }
 
-  /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, LongRange...)}
-   */
-  @Deprecated
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
-    this(field, valueSource.asLongValuesSource(), hits, null, ranges);
-  }
-
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
   public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
-  /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, Query, LongRange...)}
-   */
-  @Deprecated
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
-    this(field, valueSource.asLongValuesSource(), hits, fastMatchQuery, ranges);
-  }
-
-
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 0a73ae5..6bc77c2 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -18,19 +18,14 @@ package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.IntsRef;
 
 /** Aggregates sum of values from {@link
@@ -42,21 +37,6 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
 
   /**
    * Aggreggates double facet values from the provided
-   *  {@link ValueSource}, pulling ordinals using {@link
-   *  DocValuesOrdinalsReader} against the default indexed
-   *  facet field {@link
-   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
-   *
-   *  @deprecated {@link #TaxonomyFacetSumValueSource(TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
-   */
-  @Deprecated
-  public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
-                                     FacetsCollector fc, ValueSource valueSource) throws IOException {
-    this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
-  }
-
-  /**
-   * Aggreggates double facet values from the provided
    * {@link DoubleValuesSource}, pulling ordinals using {@link
    * DocValuesOrdinalsReader} against the default indexed
    * facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
@@ -64,29 +44,14 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
    public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
                                      FacetsCollector fc, DoubleValuesSource valueSource) throws IOException {
     this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
-  }
-
-  /**
-   * Aggreggates float facet values from the provided
-   *  {@link ValueSource}, and pulls ordinals from the
-   *  provided {@link OrdinalsReader}.
-   *
-   *  @deprecated use {@link #TaxonomyFacetSumValueSource(OrdinalsReader, TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
-   */
-  @Deprecated
-  public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
-                                     FacetsConfig config, FacetsCollector fc, ValueSource valueSource) throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
-    this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource.asDoubleValuesSource());
-  }
+   }
 
   /**
    * Aggreggates float facet values from the provided
    *  {@link DoubleValuesSource}, and pulls ordinals from the
    *  provided {@link OrdinalsReader}.
    */
-   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
+  public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
                                      FacetsConfig config, FacetsCollector fc, DoubleValuesSource vs) throws IOException {
     super(ordinalsReader.getIndexFieldName(), taxoReader, config);
     this.ordinalsReader = ordinalsReader;
@@ -134,41 +99,5 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
 
     rollup();
   }
-
-  /**
-   * {@link ValueSource} that returns the score for each
-   *  hit; use this to aggregate the sum of all hit scores
-   *  for each facet label.
-   *
-   * @deprecated Use {@link DoubleValuesSource#SCORES}
-   */
-  public static class ScoreValueSource extends ValueSource {
-
-    /** Sole constructor. */
-    public ScoreValueSource() {
-    }
-
-    @Override
-    public FunctionValues getValues(@SuppressWarnings("rawtypes") Map context, LeafReaderContext readerContext) throws IOException {
-      final Scorer scorer = (Scorer) context.get("scorer");
-      if (scorer == null) {
-        throw new IllegalStateException("scores are missing; be sure to pass keepScores=true to FacetsCollector");
-      }
-      return new DoubleDocValues(this) {
-        @Override
-        public double doubleVal(int document) {
-          try {
-            return scorer.score();
-          } catch (IOException exception) {
-            throw new RuntimeException(exception);
-          }
-        }
-      };
-    }
-    
-    @Override public boolean equals(Object o) { return o == this; }
-    @Override public int hashCode() { return System.identityHashCode(this); }
-    @Override public String description() { return "score()"; }
-  }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index ff207d3..31f9e59 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -46,13 +46,11 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -437,7 +435,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
       } else {
         fastMatchQuery = null;
       }
-      ValueSource vs = new LongFieldSource("field");
+      LongValuesSource vs = LongValuesSource.fromLongField("field");
       Facets facets = new LongRangeFacetCounts("field", vs, sfc, fastMatchQuery, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
@@ -580,7 +578,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
       } else {
         fastMatchFilter = null;
       }
-      ValueSource vs = new DoubleFieldSource("field");
+      DoubleValuesSource vs = DoubleValuesSource.fromDoubleField("field");
       Facets facets = new DoubleRangeFacetCounts("field", vs, sfc, fastMatchFilter, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 31bf6e1..2af9925 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.facet.taxonomy;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -37,18 +36,12 @@ import org.apache.lucene.facet.FacetsConfig;
 import org.apache.lucene.facet.LabelAndValue;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
@@ -56,7 +49,6 @@ import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -121,7 +113,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // Facets.search utility methods:
     searcher.search(new MatchAllDocsQuery(), c);
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
 
     // Retrieve & verify results:
     assertEquals("dim=Author path=[] value=145.0 childCount=4\n  Lisa (50.0)\n  Frank (45.0)\n  Susan (40.0)\n  Bob (10.0)\n", facets.getTopChildren(10, "Author").toString());
@@ -181,7 +173,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector c = new FacetsCollector();
     searcher.search(new MatchAllDocsQuery(), c);    
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -224,7 +216,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector c = new FacetsCollector();
     searcher.search(new MatchAllDocsQuery(), c);    
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, config, c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, config, c, DoubleValuesSource.fromIntField("num"));
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -295,7 +287,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector sfc = new FacetsCollector();
     newSearcher(r).search(new MatchAllDocsQuery(), sfc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, new LongFieldSource("price"));
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
     iw.close();
@@ -319,34 +311,12 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     
     DirectoryReader r = DirectoryReader.open(iw);
     DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
-
-    ValueSource valueSource = new ValueSource() {
-      @Override
-      public FunctionValues getValues(@SuppressWarnings("rawtypes") Map context, LeafReaderContext readerContext) throws IOException {
-        final Scorer scorer = (Scorer) context.get("scorer");
-        assert scorer != null;
-        return new DoubleDocValues(this) {
-          @Override
-          public double doubleVal(int document) {
-            try {
-              return scorer.score();
-            } catch (IOException exception) {
-              throw new RuntimeException(exception);
-            }
-          }
-        };
-      }
-
-      @Override public boolean equals(Object o) { return o == this; }
-      @Override public int hashCode() { return System.identityHashCode(this); }
-      @Override public String description() { return "score()"; }
-    };
     
     FacetsCollector fc = new FacetsCollector(true);
     // score documents by their 'price' field - makes asserting the correct counts for the categories easier
     Query q = new FunctionQuery(new LongFieldSource("price"));
     FacetsCollector.search(newSearcher(r), q, 10, fc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, valueSource);
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
     
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
@@ -374,10 +344,9 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     DirectoryReader r = DirectoryReader.open(iw);
     DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
 
-    ValueSource valueSource = new LongFieldSource("price");
     FacetsCollector sfc = new FacetsCollector();
     newSearcher(r).search(new MatchAllDocsQuery(), sfc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, valueSource);
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
     
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
@@ -447,8 +416,6 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // NRT open
     TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
 
-    ValueSource values = new FloatFieldSource("value");
-
     int iters = atLeast(100);
     for(int iter=0;iter<iters;iter++) {
       String searchToken = tokens[random().nextInt(tokens.length)];
@@ -457,7 +424,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
       }
       FacetsCollector fc = new FacetsCollector();
       FacetsCollector.search(searcher, new TermQuery(new Term("content", searchToken)), 10, fc);
-      Facets facets = new TaxonomyFacetSumValueSource(tr, config, fc, values);
+      Facets facets = new TaxonomyFacetSumValueSource(tr, config, fc, DoubleValuesSource.fromFloatField("value"));
 
       // Slow, yet hopefully bug-free, faceting:
       @SuppressWarnings({"rawtypes","unchecked"}) Map<String,Float>[] expectedValues = new HashMap[numDims];
@@ -473,7 +440,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
               if (v == null) {
                 expectedValues[j].put(doc.dims[j], doc.value);
               } else {
-                expectedValues[j].put(doc.dims[j], v.floatValue() + doc.value);
+                expectedValues[j].put(doc.dims[j], v + doc.value);
               }
             }
           }


[21/34] lucene-solr:jira/solr-9856: SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.

Posted by ab...@apache.org.
SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.


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

Branch: refs/heads/jira/solr-9856
Commit: d817fd43eccd67a5d73c3bbc49561de65d3fc9cb
Parents: 67261d2
Author: Steve Rowe <sa...@apache.org>
Authored: Sat Jan 7 16:39:20 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Sat Jan 7 16:39:20 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../java/org/apache/solr/update/UpdateLog.java  |   4 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java  |   3 +-
 ...dd-schema-fields-update-processor-chains.xml |   8 +-
 .../collection1/conf/solrconfig-schemaless.xml  |  45 ++----
 .../schema/TestSchemalessBufferedUpdates.java   | 160 +++++++++++++++++++
 solr/example/files/conf/solrconfig.xml          |   5 +-
 .../basic_configs/conf/solrconfig.xml           |   6 +-
 .../conf/solrconfig.xml                         |   5 +-
 9 files changed, 190 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1f7f09a..0d61730 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -337,6 +337,9 @@ Bug Fixes
 
 * SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules (Tim Owen via noble)
 
+* SOLR-9883: Example schemaless solr config files can lead to invalid tlog replays: when updates are buffered,
+  update processors ordered before DistributedUpdateProcessor, e.g. field normalization, are never run. (Steve Rowe)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 9c0f1cf..b79290d 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1429,7 +1429,7 @@ public class UpdateLog implements PluginInfoInitialized {
             }
           } catch (IOException ex) {
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (ClassCastException cl) {
             recoveryInfo.errors++;
@@ -1440,7 +1440,7 @@ public class UpdateLog implements PluginInfoInitialized {
               throw ex;
             }
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (Exception ex) {
             recoveryInfo.errors++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
index 4f68bcc..4758972 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
@@ -120,7 +120,8 @@ import static org.apache.solr.core.ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG;
  *   &lt;/lst&gt;
  * &lt;/processor&gt;</pre>
  */
-public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware {
+public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory
+    implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String TYPE_MAPPING_PARAM = "typeMapping";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
index 8d91d28..e574575 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
@@ -66,8 +66,6 @@
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.AddSchemaFieldsUpdateProcessorFactory">
       <str name="defaultFieldType">text</str>
       <lst name="typeMapping">
@@ -96,12 +94,12 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="parse-and-add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -154,6 +152,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
index 46aa2a4..8247d69 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
@@ -1,5 +1,4 @@
-<?xml version="1.0" ?>
-
+<?xml version="1.0" encoding="UTF-8" ?>
 <!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
@@ -16,7 +15,7 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-                                                           
+
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
@@ -33,47 +32,26 @@
   <updateHandler>
     <updateLog enable="true">
       <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog> 
+    </updateLog>
   </updateHandler>
 
   <requestHandler name="standard" class="solr.StandardRequestHandler">
     <bool name="httpCaching">true</bool>
   </requestHandler>
 
-
-  <requestHandler name="/update" class="solr.UpdateRequestHandler">
+  <initParams path="/update/**">
     <lst name="defaults">
       <str name="update.chain">add-unknown-fields-to-the-schema</str>
     </lst>
-  </requestHandler>
-
-  <query>
-    <!-- custom cache currently used by block join -->
-    <cache name="perSegFilter"
-           class="solr.search.LRUCache"
-           size="10"
-           initialSize="0"
-           autowarmCount="10"
-           regenerator="solr.NoOpRegenerator" />
-  </query>
+  </initParams>
 
-  <!-- Add unknown fields to the schema
-  
-       An example field type guessing update processor that will
-       attempt to parse string-typed field values as Booleans, Longs,
-       Doubles, or Dates, and then add schema fields with the guessed
-       field types.  
-       
-       This requires that the schema is both managed and mutable, by
-       declaring schemaFactory as ManagedIndexSchemaFactory, with
-       mutable specified as true. 
-       
-       See http://wiki.apache.org/solr/GuessingFieldTypes
-    -->
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.UUIDUpdateProcessorFactory" />
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
+    <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
+      <str name="pattern">[^\w-\.]</str>
+      <str name="replacement">_</str>
+    </processor>
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -119,7 +97,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
-
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
new file mode 100644
index 0000000..c2e8b2e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.schema;
+
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DirectUpdateHandler2;
+import org.apache.solr.update.UpdateLog;
+import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+
+public class TestSchemalessBufferedUpdates extends SolrTestCaseJ4 {
+
+  // means that we've seen the leader and have version info (i.e. we are a non-leader replica)
+  private static final String FROM_LEADER = DistribPhase.FROMLEADER.toString();
+  private static final String UPDATE_CHAIN = "add-unknown-fields-to-the-schema";
+  private static final int TIMEOUT = 10;
+
+  private static final String collection = "collection1";
+  private static final String confDir = collection + "/conf";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    File tmpSolrHome = createTempDir().toFile();
+    File tmpConfDir = new File(tmpSolrHome, confDir);
+    File testHomeConfDir = new File(TEST_HOME(), confDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-schemaless.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-add-schema-fields-update-processor.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig.snippet.randomindexconfig.xml"), tmpConfDir);
+    initCore("solrconfig-schemaless.xml", "schema-add-schema-fields-update-processor.xml", tmpSolrHome.getPath());
+  }
+
+  @Test
+  public void test() throws Exception {
+    DirectUpdateHandler2.commitOnClose = false;
+    final Semaphore logReplay = new Semaphore(0);
+    final Semaphore logReplayFinish = new Semaphore(0);
+    UpdateLog.testing_logReplayHook = () -> {
+      try {
+        assertTrue(logReplay.tryAcquire(TIMEOUT, TimeUnit.SECONDS));
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    };
+    UpdateLog.testing_logReplayFinishHook = logReplayFinish::release;
+
+    SolrQueryRequest req = req();
+    UpdateHandler uhandler = req.getCore().getUpdateHandler();
+    UpdateLog ulog = uhandler.getUpdateLog();
+
+    try {
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      // Invalid date will be normalized by ParseDateField URP
+      updateJ(jsonAdd(processAdd(sdoc("id","1", "f_dt","2017-01-04"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==1");
+
+      ulog.bufferUpdates();
+      assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
+
+      // If the ParseDateField URP isn't ahead of the DUP, then the date won't be normalized in the buffered tlog entry,
+      // and the doc won't be indexed on the replaying replica - a warning is logged as follows:
+      // WARN [...] o.a.s.u.UpdateLog REYPLAY_ERR: IOException reading log
+      //            org.apache.solr.common.SolrException: Invalid Date String:'2017-01-05'
+      //              at org.apache.solr.util.DateMathParser.parseMath(DateMathParser.java:234)
+      //              at org.apache.solr.schema.TrieField.createField(TrieField.java:725) [...]
+      updateJ(jsonAdd(processAdd(sdoc("id","2", "f_dt","2017-01-05"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+
+      Future<UpdateLog.RecoveryInfo> rinfoFuture = ulog.applyBufferedUpdates();
+
+      assertTrue(rinfoFuture != null);
+
+      assertEquals(UpdateLog.State.APPLYING_BUFFERED, ulog.getState());
+
+      logReplay.release(1000);
+
+      UpdateLog.RecoveryInfo rinfo = rinfoFuture.get();
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==2");
+    } finally {
+      DirectUpdateHandler2.commitOnClose = true;
+      UpdateLog.testing_logReplayHook = null;
+      UpdateLog.testing_logReplayFinishHook = null;
+      req().close();
+    }
+  }
+
+  private SolrInputDocument processAdd(final SolrInputDocument docIn) throws IOException {
+    UpdateRequestProcessorChain processorChain = h.getCore().getUpdateProcessingChain(UPDATE_CHAIN);
+    assertNotNull("Undefined URP chain '" + UPDATE_CHAIN + "'", processorChain);
+    List <UpdateRequestProcessorFactory> factoriesUpToDUP = new ArrayList<>();
+    for (UpdateRequestProcessorFactory urpFactory : processorChain.getProcessors()) {
+      factoriesUpToDUP.add(urpFactory);
+      if (urpFactory.getClass().equals(DistributedUpdateProcessorFactory.class)) 
+        break;
+    }
+    UpdateRequestProcessorChain chainUpToDUP = new UpdateRequestProcessorChain(factoriesUpToDUP, h.getCore());
+    assertNotNull("URP chain '" + UPDATE_CHAIN + "'", chainUpToDUP);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = req();
+    try {
+      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = docIn;
+      UpdateRequestProcessor processor = chainUpToDUP.createProcessor(req, rsp);
+      processor.processAdd(cmd);
+      if (cmd.solrDoc.get("f_dt").getValue() instanceof Date) {
+        // Non-JSON types (Date in this case) aren't handled properly in noggit-0.6.  Although this is fixed in 
+        // https://github.com/yonik/noggit/commit/ec3e732af7c9425e8f40297463cbe294154682b1 to call obj.toString(), 
+        // Date::toString produces a Date representation that Solr doesn't like, so we convert using Instant::toString
+        cmd.solrDoc.get("f_dt").setValue(((Date) cmd.solrDoc.get("f_dt").getValue()).toInstant().toString(), 1.0f);
+      }
+      return cmd.solrDoc;
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+      req.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index 3fd825e..f83c235 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -1193,9 +1193,6 @@
   <updateRequestProcessorChain name="files-update-processor">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1261,6 +1258,8 @@
       <!--</lst>-->
     </processor>
 
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
index 4ef902f..e009aab 100644
--- a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
@@ -1187,9 +1187,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1239,6 +1236,9 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
index 4b0899b..2ca1b7f 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
@@ -1186,9 +1186,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1238,6 +1235,8 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 


[17/34] lucene-solr:jira/solr-9856: SOLR-9928 Unwrap Directory consistently whenever it's passed as an argument.

Posted by ab...@apache.org.
SOLR-9928 Unwrap Directory consistently whenever it's passed as an argument.


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

Branch: refs/heads/jira/solr-9856
Commit: e5f39f62f76677a5f500af4f323c0c31afb26228
Parents: 1a95c5a
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Sat Jan 7 13:22:37 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Sat Jan 7 13:24:22 2017 +0100

----------------------------------------------------------------------
 .../solr/core/MetricsDirectoryFactory.java      | 60 ++++++++------------
 1 file changed, 25 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5f39f62/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
index 8706c61..f441579 100644
--- a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
@@ -74,21 +74,28 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
     }
   }
 
-  @Override
-  public void doneWithDirectory(Directory dir) throws IOException {
-    // unwrap
+  /**
+   * Unwrap just one level if the argument is a {@link MetricsDirectory}
+   * @param dir directory
+   * @return delegate if the instance was a {@link MetricsDirectory}, otherwise unchanged.
+   */
+  private static Directory unwrap(Directory dir) {
     if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
+      return ((MetricsDirectory)dir).getDelegate();
+    } else {
+      return dir;
     }
+  }
+
+  @Override
+  public void doneWithDirectory(Directory dir) throws IOException {
+    dir = unwrap(dir);
     in.doneWithDirectory(dir);
   }
 
   @Override
   public void addCloseListener(Directory dir, CachingDirectoryFactory.CloseListener closeListener) {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.addCloseListener(dir, closeListener);
   }
 
@@ -115,19 +122,13 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void remove(Directory dir) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.remove(dir);
   }
 
   @Override
   public void remove(Directory dir, boolean afterCoreClose) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.remove(dir, afterCoreClose);
   }
 
@@ -152,8 +153,9 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
   }
 
   @Override
-  public long size(Directory directory) throws IOException {
-    return in.size(directory);
+  public long size(Directory dir) throws IOException {
+    dir = unwrap(dir);
+    return in.size(dir);
   }
 
   @Override
@@ -183,6 +185,8 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void move(Directory fromDir, Directory toDir, String fileName, IOContext ioContext) throws IOException {
+    fromDir = unwrap(fromDir);
+    toDir = unwrap(toDir);
     in.move(fromDir, toDir, fileName, ioContext);
   }
 
@@ -198,10 +202,7 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory) dir).getDelegate();
-    }
-
+    dir = unwrap(dir);
     in.renameWithOverwrite(dir, fileName, toName);
   }
 
@@ -221,16 +222,8 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
   }
 
   @Override
-  protected Directory getBaseDir(Directory dir) {
-    return in.getBaseDir(dir);
-  }
-
-  @Override
   public void incRef(Directory dir) {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.incRef(dir);
   }
 
@@ -248,10 +241,7 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void release(Directory dir) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.release(dir);
   }
 


[33/34] lucene-solr:jira/solr-9856: SOLR-9854 Relax test assertions.

Posted by ab...@apache.org.
SOLR-9854 Relax test assertions.


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

Branch: refs/heads/jira/solr-9856
Commit: b9827bcba9ee4e1b0dea8a18c6efe4322a400b09
Parents: b017731
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 9 11:16:40 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 9 11:16:40 2017 +0100

----------------------------------------------------------------------
 .../src/test/org/apache/solr/update/SolrIndexMetricsTest.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b9827bcb/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
index e17b1bd..4f5ea69 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
@@ -67,13 +67,13 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
 
     // check basic index meters
     Timer timer = (Timer)metrics.get("INDEX.merge.minor");
-    assertEquals("minorMerge: " + timer.getCount(), 4, timer.getCount());
+    assertTrue("minorMerge: " + timer.getCount(), timer.getCount() >= 3);
     timer = (Timer)metrics.get("INDEX.merge.major");
     assertEquals("majorMerge: " + timer.getCount(), 0, timer.getCount());
     Meter meter = (Meter)metrics.get("INDEX.merge.major.docs");
     assertEquals("majorMergeDocs: " + meter.getCount(), 0, meter.getCount());
     meter = (Meter)metrics.get("INDEX.flush");
-    assertEquals("flush: " + meter.getCount(), 19, meter.getCount());
+    assertTrue("flush: " + meter.getCount(), meter.getCount() > 10);
 
     // check basic directory meters
     meter = (Meter)metrics.get("DIRECTORY.total.reads");


[07/34] lucene-solr:jira/solr-9856: SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules

Posted by ab...@apache.org.
SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules


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

Branch: refs/heads/jira/solr-9856
Commit: cd4f908d5ba223e615920be73285b7c5cc57704a
Parents: b32cd82
Author: Noble Paul <no...@apache.org>
Authored: Sat Jan 7 01:40:47 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Sat Jan 7 01:40:47 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../apache/solr/cloud/rule/ReplicaAssigner.java | 31 ++++----------------
 .../java/org/apache/solr/cloud/rule/Rule.java   |  4 +--
 .../apache/solr/cloud/rule/RuleEngineTest.java  |  2 +-
 4 files changed, 10 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd4f908d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b747876..1f7f09a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -335,6 +335,8 @@ Bug Fixes
 * SOLR-9931: JSON Facet API hll (hyper-log-log) function returned 0 for non-empty buckets with no field values
   in local mode, but nothing for distributed mode.  Both modes now return 0.  (yonik)
 
+* SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules (Tim Owen via noble)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd4f908d/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
index 4ecda47..3eab8b4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
@@ -17,7 +17,6 @@
 package org.apache.solr.cloud.rule;
 
 import java.lang.invoke.MethodHandles;
-
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collections;
@@ -39,7 +38,6 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 import org.apache.solr.common.cloud.rule.Snitch;
 import org.apache.solr.common.cloud.rule.SnitchContext;
-import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.slf4j.Logger;
@@ -48,7 +46,6 @@ import org.slf4j.LoggerFactory;
 import static java.util.Collections.singletonList;
 import static org.apache.solr.cloud.rule.Rule.MatchStatus.*;
 import static org.apache.solr.cloud.rule.Rule.Phase.*;
-import static org.apache.solr.common.util.StrUtils.formatString;
 import static org.apache.solr.common.util.Utils.getDeepCopy;
 
 public class ReplicaAssigner {
@@ -103,7 +100,6 @@ public class ReplicaAssigner {
     this.participatingLiveNodes = new ArrayList<>(participatingLiveNodes);
     this.nodeVsTags = getTagsForNodes(cc, snitches);
     this.shardVsNodes = getDeepCopy(shardVsNodes, 2);
-    validateTags(nodeVsTags);
 
     if (clusterState != null) {
       Map<String, DocCollection> collections = clusterState.getCollectionsMap();
@@ -284,21 +280,6 @@ public class ReplicaAssigner {
     return result;
   }
 
-  private void validateTags(Map<String, Map<String, Object>> nodeVsTags) {
-    List<String> errors = new ArrayList<>();
-    for (Rule rule : rules) {
-      for (Map.Entry<String, Map<String, Object>> e : nodeVsTags.entrySet()) {
-        if (e.getValue().get(rule.tag.name) == null) {
-          errors.add(formatString("The value for tag {0} is not available for node {1}", rule.tag.name, e.getKey()));
-        }
-      }
-    }
-    if (!errors.isEmpty()) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, StrUtils.join(errors, ','));
-    }
-  }
-
-
   /**
    * get all permutations for the int[] whose items are 0..level
    */
@@ -422,14 +403,12 @@ public class ReplicaAssigner {
           context.exception = new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Not all tags were obtained from node " + node);
         } else {
-          if (context.getTags().keySet().containsAll(context.snitchInfo.getTagNames())) {
-            Map<String, Object> tags = result.get(node);
-            if (tags == null) {
-              tags = new HashMap<>();
-              result.put(node, tags);
-            }
-            tags.putAll(context.getTags());
+          Map<String, Object> tags = result.get(node);
+          if (tags == null) {
+            tags = new HashMap<>();
+            result.put(node, tags);
           }
+          tags.putAll(context.getTags());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd4f908d/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java b/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
index 97947cf..87bbe69 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
@@ -202,7 +202,7 @@ public class Rule {
 
       @Override
       public boolean canMatch(Object ruleVal, Object testVal) {
-        return compareNum(ruleVal, testVal) == 1;
+        return testVal != null && compareNum(ruleVal, testVal) == 1;
       }
 
     },
@@ -214,7 +214,7 @@ public class Rule {
 
       @Override
       public boolean canMatch(Object ruleVal, Object testVal) {
-        return compareNum(ruleVal, testVal) == -1;
+        return testVal != null && compareNum(ruleVal, testVal) == -1;
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd4f908d/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
index 7c33541..8b0a788 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
@@ -85,7 +85,7 @@ public class RuleEngineTest extends SolrTestCaseJ4{
         new HashMap(), new ArrayList<>(MockSnitch.nodeVsTags.keySet()), null, null ).getNodeMappings();
     assertNotNull(mapping);
 
-    rules = parseRules("[{role:'!overseer'}]" );
+    rules = parseRules("[{role:'!overseer'}, {'freedisk':'>1'}]" );
     Map<String, Object> snitchSession = new HashMap<>();
     List<String> preferredOverseerNodes = ImmutableList.of("127.0.0.1:49947_", "127.0.0.1:49952_");
     ReplicaAssigner replicaAssigner = new ReplicaAssigner(


[31/34] lucene-solr:jira/solr-9856: LUCENE-7611: Remove queries javadoc link from suggester help page

Posted by ab...@apache.org.
LUCENE-7611: Remove queries javadoc link from suggester help page


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/322ad889
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/322ad889
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/322ad889

Branch: refs/heads/jira/solr-9856
Commit: 322ad889604688db9d22ba7dfa1e389a01e34857
Parents: 8bc151d
Author: Alan Woodward <ro...@apache.org>
Authored: Sun Jan 8 20:07:30 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sun Jan 8 20:07:30 2017 +0000

----------------------------------------------------------------------
 lucene/suggest/build.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/322ad889/lucene/suggest/build.xml
----------------------------------------------------------------------
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index bc4bed5..47d4a63 100644
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -38,7 +38,6 @@
     <invoke-module-javadoc>
       <links>
         <link href="../analyzers-common"/>
-        <link href="../queries"/>
       </links>
     </invoke-module-javadoc>
   </target>


[03/34] lucene-solr:jira/solr-9856: SOLR-9877: Add a null check for target

Posted by ab...@apache.org.
SOLR-9877: Add a null check for target


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/3eab1b48
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3eab1b48
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3eab1b48

Branch: refs/heads/jira/solr-9856
Commit: 3eab1b4839e30d5a82923afeff1bc19bf8e6b25f
Parents: e5264c4
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 5 23:08:27 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 5 23:08:39 2017 +0530

----------------------------------------------------------------------
 .../apache/solr/util/stats/InstrumentedHttpRequestExecutor.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3eab1b48/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index 91ddd8a..0caa2d1 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -78,7 +78,9 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
       String schemeHostPort = null;
       if (request instanceof HttpRequestWrapper) {
         HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
-        schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" +  wrapper.getTarget().getPort();
+        if (wrapper.getTarget() != null)  {
+          schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" +  wrapper.getTarget().getPort();
+        }
       }
       final URIBuilder url = new URIBuilder(requestLine.getUri());
       return SolrMetricManager.mkName((schemeHostPort != null ? schemeHostPort : "") + url.removeQuery().build().toString() + "." + methodNameString(request), scope);


[29/34] lucene-solr:jira/solr-9856: SOLR-9859: Don't log error on NoSuchFileException (Cao Manh Dat)

Posted by ab...@apache.org.
SOLR-9859: Don't log error on NoSuchFileException (Cao Manh Dat)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/25290ab5
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/25290ab5
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/25290ab5

Branch: refs/heads/jira/solr-9856
Commit: 25290ab5d6af25c05cbbb4738f49329273a7d693
Parents: f985fca
Author: markrmiller <ma...@apache.org>
Authored: Sun Jan 8 09:21:43 2017 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Sun Jan 8 09:21:43 2017 -0500

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/core/DirectoryFactory.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/25290ab5/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 136a0a6..ac18d7e 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -22,6 +22,7 @@ import java.io.FileFilter;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.NoSuchFileException;
 import java.util.Collection;
 import java.util.Collections;
 
@@ -191,7 +192,7 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
     try {
       dir.deleteFile(toName);
-    } catch (FileNotFoundException e) {
+    } catch (FileNotFoundException | NoSuchFileException e) {
 
     } catch (Exception e) {
       log.error("Exception deleting file", e);


[27/34] lucene-solr:jira/solr-9856: TokenStreamToAutomaton failed to handle certain holes correctly

Posted by ab...@apache.org.
TokenStreamToAutomaton failed to handle certain holes correctly


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

Branch: refs/heads/jira/solr-9856
Commit: e64111c6545d8e05241bc87eb4623c5ed44db312
Parents: 1aa9c42
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:26:08 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:26:08 2017 -0500

----------------------------------------------------------------------
 .../apache/lucene/analysis/TokenStreamToAutomaton.java  | 11 ++++++++++-
 .../org/apache/lucene/analysis/TestGraphTokenizers.java | 12 ++++++++++++
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e64111c6/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
index 071fa4a..64bac66 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
@@ -113,6 +113,7 @@ public class TokenStreamToAutomaton {
     final RollingBuffer<Position> positions = new Positions();
 
     int pos = -1;
+    int freedPos = 0;
     Position posData = null;
     int maxOffset = 0;
     while (in.incrementToken()) {
@@ -150,7 +151,15 @@ public class TokenStreamToAutomaton {
             addHoles(builder, positions, pos);
           }
         }
-        positions.freeBefore(pos);
+        while (freedPos <= pos) {
+          Position freePosData = positions.get(freedPos);
+          // don't free this position yet if we may still need to fill holes over it:
+          if (freePosData.arriving == -1 || freePosData.leaving == -1) {
+            break;
+          }
+          positions.freeBefore(freedPos);
+          freedPos++;
+        }
       }
 
       final int endPos = pos + posLengthAtt.getPositionLength();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e64111c6/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
index 78fb127..8899dd1 100644
--- a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
+++ b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
@@ -585,4 +585,16 @@ public class TestGraphTokenizers extends BaseTokenStreamTestCase {
       Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
       Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
   }
+
+  public void testTokenStreamGraphWithHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 1, 8),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
+                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
+  }
 }


[28/34] lucene-solr:jira/solr-9856: promote this test case to core

Posted by ab...@apache.org.
promote this test case to core


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

Branch: refs/heads/jira/solr-9856
Commit: f985fcaa23cb9ef96ed823e5bf7957049e0d9461
Parents: e64111c
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:38:37 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:38:37 2017 -0500

----------------------------------------------------------------------
 .../lucene/analysis/TestGraphTokenizers.java    | 600 +++++++++++++++++++
 .../lucene/analysis/TestGraphTokenizers.java    | 600 -------------------
 2 files changed, 600 insertions(+), 600 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f985fcaa/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
new file mode 100644
index 0000000..8899dd1
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
@@ -0,0 +1,600 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.analysis;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
+public class TestGraphTokenizers extends BaseTokenStreamTestCase {
+
+  // Makes a graph TokenStream from the string; separate
+  // positions with single space, multiple tokens at the same
+  // position with /, and add optional position length with
+  // :.  EG "a b c" is a simple chain, "a/x b c" adds 'x'
+  // over 'a' at position 0 with posLen=1, "a/x:3 b c" adds
+  // 'x' over a with posLen=3.  Tokens are in normal-form!
+  // So, offsets are computed based on the first token at a
+  // given position.  NOTE: each token must be a single
+  // character!  We assume this when computing offsets...
+  
+  // NOTE: all input tokens must be length 1!!!  This means
+  // you cannot turn on MockCharFilter when random
+  // testing...
+
+  private static class GraphTokenizer extends Tokenizer {
+    private List<Token> tokens;
+    private int upto;
+    private int inputLength;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+    private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      tokens = null;
+      upto = 0;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (tokens == null) {
+        fillTokens();
+      }
+      //System.out.println("graphTokenizer: incr upto=" + upto + " vs " + tokens.size());
+      if (upto == tokens.size()) {
+        //System.out.println("  END @ " + tokens.size());
+        return false;
+      } 
+      final Token t = tokens.get(upto++);
+      //System.out.println("  return token=" + t);
+      clearAttributes();
+      termAtt.append(t.toString());
+      offsetAtt.setOffset(t.startOffset(), t.endOffset());
+      posIncrAtt.setPositionIncrement(t.getPositionIncrement());
+      posLengthAtt.setPositionLength(t.getPositionLength());
+      return true;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      // NOTE: somewhat... hackish, but we need this to
+      // satisfy BTSTC:
+      final int lastOffset;
+      if (tokens != null && !tokens.isEmpty()) {
+        lastOffset = tokens.get(tokens.size()-1).endOffset();
+      } else {
+        lastOffset = 0;
+      }
+      offsetAtt.setOffset(correctOffset(lastOffset),
+                          correctOffset(inputLength));
+    }
+
+    private void fillTokens() throws IOException {
+      final StringBuilder sb = new StringBuilder();
+      final char[] buffer = new char[256];
+      while (true) {
+        final int count = input.read(buffer);
+        if (count == -1) {
+          break;
+        }
+        sb.append(buffer, 0, count);
+        //System.out.println("got count=" + count);
+      }
+      //System.out.println("fillTokens: " + sb);
+
+      inputLength = sb.length();
+
+      final String[] parts = sb.toString().split(" ");
+
+      tokens = new ArrayList<>();
+      int pos = 0;
+      int maxPos = -1;
+      int offset = 0;
+      //System.out.println("again");
+      for(String part : parts) {
+        final String[] overlapped = part.split("/");
+        boolean firstAtPos = true;
+        int minPosLength = Integer.MAX_VALUE;
+        for(String part2 : overlapped) {
+          final int colonIndex = part2.indexOf(':');
+          final String token;
+          final int posLength;
+          if (colonIndex != -1) {
+            token = part2.substring(0, colonIndex);
+            posLength = Integer.parseInt(part2.substring(1+colonIndex));
+          } else {
+            token = part2;
+            posLength = 1;
+          }
+          maxPos = Math.max(maxPos, pos + posLength);
+          minPosLength = Math.min(minPosLength, posLength);
+          final Token t = new Token(token, offset, offset + 2*posLength - 1);
+          t.setPositionLength(posLength);
+          t.setPositionIncrement(firstAtPos ? 1:0);
+          firstAtPos = false;
+          //System.out.println("  add token=" + t + " startOff=" + t.startOffset() + " endOff=" + t.endOffset());
+          tokens.add(t);
+        }
+        pos += minPosLength;
+        offset = 2 * pos;
+      }
+      assert maxPos <= pos: "input string mal-formed: posLength>1 tokens hang over the end";
+    }
+  }
+
+  public void testMockGraphTokenFilterBasic() throws Exception {
+
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random(), a, false, "a b c d e f g h i j k");
+    }
+  }
+
+  public void testMockGraphTokenFilterOnGraphInput() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new GraphTokenizer();
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random(), a, false, "a/x:3 c/y:2 d e f/z:4 g h i j k");
+    }
+  }
+
+  // Just deletes (leaving hole) token 'a':
+  private final static class RemoveATokens extends TokenFilter {
+    private int pendingPosInc;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+    public RemoveATokens(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      pendingPosInc = 0;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      while (true) {
+        final boolean gotOne = input.incrementToken();
+        if (!gotOne) {
+          return false;
+        } else if (termAtt.toString().equals("a")) {
+          pendingPosInc += posIncAtt.getPositionIncrement();
+        } else {
+          posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+          pendingPosInc = 0;
+          return true;
+        }
+      }
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t3 = new RemoveATokens(t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      Random random = random();
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new RemoveATokens(t);
+            final TokenStream t3 = new MockGraphTokenFilter(random(), t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      Random random = random();
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  // Two MockGraphTokenFilters
+  public void testDoubleMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHolesRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t2 = new MockHoleInjectingTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHolesRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockHoleInjectingTokenFilter(random(), t);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  private static Token token(String term, int posInc, int posLength) {
+    final Token t = new Token(term, 0, 0);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  private static Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
+    final Token t = new Token(term, startOffset, endOffset);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  public void testSingleToken() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+      });
+    assertSameLanguage(s2a("abc"), ts);
+  }
+
+  public void testMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("b", 3, 1),
+      });
+    assertSameLanguage(join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")), ts);
+  }
+
+  public void testSynOverMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("x", 0, 3),
+        token("b", 3, 1),
+      });
+    final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
+    final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  // for debugging!
+  /*
+  private static void toDot(Automaton a) throws IOException {
+    final String s = a.toDot();
+    Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    w.write(s);
+    w.close();
+    System.out.println("TEST: saved to /x/tmp/out.dot");
+  }
+  */
+
+  private static final Automaton SEP_A = Automata.makeChar(TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton HOLE_A = Automata.makeChar(TokenStreamToAutomaton.HOLE);
+
+  private Automaton join(String ... strings) {
+    List<Automaton> as = new ArrayList<>();
+    for(String s : strings) {
+      as.add(s2a(s));
+      as.add(SEP_A);
+    }
+    as.remove(as.size()-1);
+    return Operations.concatenate(as);
+  }
+
+  private Automaton join(Automaton ... as) {
+    return Operations.concatenate(Arrays.asList(as));
+  }
+
+  private Automaton s2a(String s) {
+    return Automata.makeString(s);
+  }
+
+  public void testTwoTokens() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 1, 1),
+      });
+    assertSameLanguage(join("abc", "def"), ts);
+  }
+
+  public void testHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 2, 1),
+      });
+    assertSameLanguage(join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def")), ts);
+  }
+
+  public void testOverlappedTokensSausage() throws Exception {
+
+    // Two tokens on top of each other (sausage):
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 1)
+      });
+    final Automaton a1 = s2a("abc");
+    final Automaton a2 = s2a("xyz");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testOverlappedTokensLattice() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 2),
+        token("def", 1, 1),
+      });
+    final Automaton a1 = s2a("xyz");
+    final Automaton a2 = join("abc", "def");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testSynOverHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 2),
+        token("b", 2, 1),
+      });
+    final Automaton a1 = Operations.union(join(s2a("a"), SEP_A, HOLE_A), s2a("X"));
+    final Automaton expected = Operations.concatenate(a1, join(SEP_A, s2a("b")));
+    assertSameLanguage(expected, ts);
+  }
+
+  public void testSynOverHole2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("xyz", 1, 1),
+        token("abc", 0, 3),
+        token("def", 2, 1),
+      });
+    final Automaton expected = Operations.union(
+      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")), s2a("abc"));
+    assertSameLanguage(expected, ts);
+  }
+
+  public void testOverlappedTokensLattice2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 3),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    final Automaton a1 = s2a("xyz");
+    final Automaton a2 = join("abc", "def", "ghi");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testToDot() throws Exception {
+    final TokenStream ts = new CannedTokenStream(new Token[] {token("abc", 1, 1, 0, 4)});
+    StringWriter w = new StringWriter();
+    new TokenStreamToDot("abcd", ts, new PrintWriter(w)).toDot();
+    assertTrue(w.toString().indexOf("abc / abcd") != -1);
+  }
+
+  public void testStartsWithHole() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 2, 1),
+      });
+    assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
+  }
+
+  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
+
+  public void testSynHangingOverEnd() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 10),
+      });
+    assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
+  }
+
+  private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
+    assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
+  }
+
+  private void assertSameLanguage(Automaton expected, Automaton actual) {
+    assertTrue(Operations.sameLanguage(
+      Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
+      Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
+  }
+
+  public void testTokenStreamGraphWithHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 1, 8),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
+                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f985fcaa/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
deleted file mode 100644
index 8899dd1..0000000
--- a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
+++ /dev/null
@@ -1,600 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.analysis;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.Operations;
-
-import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
-
-public class TestGraphTokenizers extends BaseTokenStreamTestCase {
-
-  // Makes a graph TokenStream from the string; separate
-  // positions with single space, multiple tokens at the same
-  // position with /, and add optional position length with
-  // :.  EG "a b c" is a simple chain, "a/x b c" adds 'x'
-  // over 'a' at position 0 with posLen=1, "a/x:3 b c" adds
-  // 'x' over a with posLen=3.  Tokens are in normal-form!
-  // So, offsets are computed based on the first token at a
-  // given position.  NOTE: each token must be a single
-  // character!  We assume this when computing offsets...
-  
-  // NOTE: all input tokens must be length 1!!!  This means
-  // you cannot turn on MockCharFilter when random
-  // testing...
-
-  private static class GraphTokenizer extends Tokenizer {
-    private List<Token> tokens;
-    private int upto;
-    private int inputLength;
-
-    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
-    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
-    private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
-
-    @Override
-    public void reset() throws IOException {
-      super.reset();
-      tokens = null;
-      upto = 0;
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (tokens == null) {
-        fillTokens();
-      }
-      //System.out.println("graphTokenizer: incr upto=" + upto + " vs " + tokens.size());
-      if (upto == tokens.size()) {
-        //System.out.println("  END @ " + tokens.size());
-        return false;
-      } 
-      final Token t = tokens.get(upto++);
-      //System.out.println("  return token=" + t);
-      clearAttributes();
-      termAtt.append(t.toString());
-      offsetAtt.setOffset(t.startOffset(), t.endOffset());
-      posIncrAtt.setPositionIncrement(t.getPositionIncrement());
-      posLengthAtt.setPositionLength(t.getPositionLength());
-      return true;
-    }
-
-    @Override
-    public void end() throws IOException {
-      super.end();
-      // NOTE: somewhat... hackish, but we need this to
-      // satisfy BTSTC:
-      final int lastOffset;
-      if (tokens != null && !tokens.isEmpty()) {
-        lastOffset = tokens.get(tokens.size()-1).endOffset();
-      } else {
-        lastOffset = 0;
-      }
-      offsetAtt.setOffset(correctOffset(lastOffset),
-                          correctOffset(inputLength));
-    }
-
-    private void fillTokens() throws IOException {
-      final StringBuilder sb = new StringBuilder();
-      final char[] buffer = new char[256];
-      while (true) {
-        final int count = input.read(buffer);
-        if (count == -1) {
-          break;
-        }
-        sb.append(buffer, 0, count);
-        //System.out.println("got count=" + count);
-      }
-      //System.out.println("fillTokens: " + sb);
-
-      inputLength = sb.length();
-
-      final String[] parts = sb.toString().split(" ");
-
-      tokens = new ArrayList<>();
-      int pos = 0;
-      int maxPos = -1;
-      int offset = 0;
-      //System.out.println("again");
-      for(String part : parts) {
-        final String[] overlapped = part.split("/");
-        boolean firstAtPos = true;
-        int minPosLength = Integer.MAX_VALUE;
-        for(String part2 : overlapped) {
-          final int colonIndex = part2.indexOf(':');
-          final String token;
-          final int posLength;
-          if (colonIndex != -1) {
-            token = part2.substring(0, colonIndex);
-            posLength = Integer.parseInt(part2.substring(1+colonIndex));
-          } else {
-            token = part2;
-            posLength = 1;
-          }
-          maxPos = Math.max(maxPos, pos + posLength);
-          minPosLength = Math.min(minPosLength, posLength);
-          final Token t = new Token(token, offset, offset + 2*posLength - 1);
-          t.setPositionLength(posLength);
-          t.setPositionIncrement(firstAtPos ? 1:0);
-          firstAtPos = false;
-          //System.out.println("  add token=" + t + " startOff=" + t.startOffset() + " endOff=" + t.endOffset());
-          tokens.add(t);
-        }
-        pos += minPosLength;
-        offset = 2 * pos;
-      }
-      assert maxPos <= pos: "input string mal-formed: posLength>1 tokens hang over the end";
-    }
-  }
-
-  public void testMockGraphTokenFilterBasic() throws Exception {
-
-    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      checkAnalysisConsistency(random(), a, false, "a b c d e f g h i j k");
-    }
-  }
-
-  public void testMockGraphTokenFilterOnGraphInput() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new GraphTokenizer();
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      checkAnalysisConsistency(random(), a, false, "a/x:3 c/y:2 d e f/z:4 g h i j k");
-    }
-  }
-
-  // Just deletes (leaving hole) token 'a':
-  private final static class RemoveATokens extends TokenFilter {
-    private int pendingPosInc;
-
-    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
-
-    public RemoveATokens(TokenStream in) {
-      super(in);
-    }
-
-    @Override
-    public void reset() throws IOException {
-      super.reset();
-      pendingPosInc = 0;
-    }
-
-    @Override
-    public void end() throws IOException {
-      super.end();
-      posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      while (true) {
-        final boolean gotOne = input.incrementToken();
-        if (!gotOne) {
-          return false;
-        } else if (termAtt.toString().equals("a")) {
-          pendingPosInc += posIncAtt.getPositionIncrement();
-        } else {
-          posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
-          pendingPosInc = 0;
-          return true;
-        }
-      }
-    }
-  }
-
-  public void testMockGraphTokenFilterBeforeHoles() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t3 = new RemoveATokens(t2);
-            return new TokenStreamComponents(t, t3);
-          }
-        };
-
-      Random random = random();
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
-    }
-  }
-
-  public void testMockGraphTokenFilterAfterHoles() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new RemoveATokens(t);
-            final TokenStream t3 = new MockGraphTokenFilter(random(), t2);
-            return new TokenStreamComponents(t, t3);
-          }
-        };
-
-      Random random = random();
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
-    }
-  }
-
-  public void testMockGraphTokenFilterRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  // Two MockGraphTokenFilters
-  public void testDoubleMockGraphTokenFilterRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  public void testMockGraphTokenFilterBeforeHolesRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t2 = new MockHoleInjectingTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  public void testMockGraphTokenFilterAfterHolesRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockHoleInjectingTokenFilter(random(), t);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  private static Token token(String term, int posInc, int posLength) {
-    final Token t = new Token(term, 0, 0);
-    t.setPositionIncrement(posInc);
-    t.setPositionLength(posLength);
-    return t;
-  }
-
-  private static Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
-    final Token t = new Token(term, startOffset, endOffset);
-    t.setPositionIncrement(posInc);
-    t.setPositionLength(posLength);
-    return t;
-  }
-
-  public void testSingleToken() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-      });
-    assertSameLanguage(s2a("abc"), ts);
-  }
-
-  public void testMultipleHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("b", 3, 1),
-      });
-    assertSameLanguage(join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")), ts);
-  }
-
-  public void testSynOverMultipleHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("x", 0, 3),
-        token("b", 3, 1),
-      });
-    final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
-    final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  // for debugging!
-  /*
-  private static void toDot(Automaton a) throws IOException {
-    final String s = a.toDot();
-    Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
-    w.write(s);
-    w.close();
-    System.out.println("TEST: saved to /x/tmp/out.dot");
-  }
-  */
-
-  private static final Automaton SEP_A = Automata.makeChar(TokenStreamToAutomaton.POS_SEP);
-  private static final Automaton HOLE_A = Automata.makeChar(TokenStreamToAutomaton.HOLE);
-
-  private Automaton join(String ... strings) {
-    List<Automaton> as = new ArrayList<>();
-    for(String s : strings) {
-      as.add(s2a(s));
-      as.add(SEP_A);
-    }
-    as.remove(as.size()-1);
-    return Operations.concatenate(as);
-  }
-
-  private Automaton join(Automaton ... as) {
-    return Operations.concatenate(Arrays.asList(as));
-  }
-
-  private Automaton s2a(String s) {
-    return Automata.makeString(s);
-  }
-
-  public void testTwoTokens() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("def", 1, 1),
-      });
-    assertSameLanguage(join("abc", "def"), ts);
-  }
-
-  public void testHole() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("def", 2, 1),
-      });
-    assertSameLanguage(join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def")), ts);
-  }
-
-  public void testOverlappedTokensSausage() throws Exception {
-
-    // Two tokens on top of each other (sausage):
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 1)
-      });
-    final Automaton a1 = s2a("abc");
-    final Automaton a2 = s2a("xyz");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testOverlappedTokensLattice() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 2),
-        token("def", 1, 1),
-      });
-    final Automaton a1 = s2a("xyz");
-    final Automaton a2 = join("abc", "def");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testSynOverHole() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("X", 0, 2),
-        token("b", 2, 1),
-      });
-    final Automaton a1 = Operations.union(join(s2a("a"), SEP_A, HOLE_A), s2a("X"));
-    final Automaton expected = Operations.concatenate(a1, join(SEP_A, s2a("b")));
-    assertSameLanguage(expected, ts);
-  }
-
-  public void testSynOverHole2() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("xyz", 1, 1),
-        token("abc", 0, 3),
-        token("def", 2, 1),
-      });
-    final Automaton expected = Operations.union(
-      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")), s2a("abc"));
-    assertSameLanguage(expected, ts);
-  }
-
-  public void testOverlappedTokensLattice2() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 3),
-        token("def", 1, 1),
-        token("ghi", 1, 1),
-      });
-    final Automaton a1 = s2a("xyz");
-    final Automaton a2 = join("abc", "def", "ghi");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testToDot() throws Exception {
-    final TokenStream ts = new CannedTokenStream(new Token[] {token("abc", 1, 1, 0, 4)});
-    StringWriter w = new StringWriter();
-    new TokenStreamToDot("abcd", ts, new PrintWriter(w)).toDot();
-    assertTrue(w.toString().indexOf("abc / abcd") != -1);
-  }
-
-  public void testStartsWithHole() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 2, 1),
-      });
-    assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
-  }
-
-  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
-
-  public void testSynHangingOverEnd() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("X", 0, 10),
-      });
-    assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
-  }
-
-  private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
-    assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
-  }
-
-  private void assertSameLanguage(Automaton expected, Automaton actual) {
-    assertTrue(Operations.sameLanguage(
-      Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
-      Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
-  }
-
-  public void testTokenStreamGraphWithHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 1, 8),
-        token("def", 1, 1),
-        token("ghi", 1, 1),
-      });
-    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
-                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
-  }
-}


[10/34] lucene-solr:jira/solr-9856: LUCENE-7614: ComplexPhraseQueryParser ignores quotes around single terms phrases

Posted by ab...@apache.org.
LUCENE-7614: ComplexPhraseQueryParser ignores quotes around single terms phrases


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/52f2a77b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/52f2a77b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/52f2a77b

Branch: refs/heads/jira/solr-9856
Commit: 52f2a77b78fc95bc98d664411cda63d58606df52
Parents: 024c403
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Thu Jan 5 23:39:46 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Sat Jan 7 00:42:04 2017 +0300

----------------------------------------------------------------------
 lucene/CHANGES.txt                                             | 3 +++
 .../queryparser/complexPhrase/ComplexPhraseQueryParser.java    | 4 +++-
 .../queryparser/complexPhrase/TestComplexPhraseQuery.java      | 6 ++++++
 3 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52f2a77b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 67d8ae5..b74056f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -198,6 +198,9 @@ Improvements
 * LUCENE-7401: Changed the way BKD trees pick the split dimension in order to
   ensure all dimensions are indexed. (Adrien Grand)
 
+* LUCENE-7614: Complex Phrase Query parser ignores double quotes around single token 
+  prefix, wildcard, range queries (Mikhail Khludnev) 
+
 Optimizations
 
 * LUCENE-7568: Optimize merging when index sorting is used but the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52f2a77b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
index 1a7e5e1..6e18960 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
@@ -255,7 +255,9 @@ public class ComplexPhraseQueryParser extends QueryParser {
     public Query rewrite(IndexReader reader) throws IOException {
       final Query contents = this.contents[0];
       // ArrayList spanClauses = new ArrayList();
-      if (contents instanceof TermQuery) {
+      if (contents instanceof TermQuery 
+          || contents instanceof MultiTermQuery
+          ) {
         return contents;
       }
       // Build a sequence of Span clauses arranged in a SpanNear - child

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52f2a77b/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
index 66078b0..28b600b 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
@@ -72,6 +72,12 @@ public class TestComplexPhraseQuery extends LuceneTestCase {
     checkBadQuery("\"jo* \"smith\" \""); // phrases inside phrases is bad
   }
 
+  public void testSingleTermPhrase() throws Exception {
+    checkMatches("\"joh*\" \"tom\"", "1,2,3,4"); 
+    checkMatches("+\"j*\" +\"tom\"", "4"); 
+    checkMatches("\"jo*\" \"[sma TO smZ]\" ", "1,2,3");
+    checkMatches("+\"j*hn\" +\"sm*h\"", "1,3"); 
+  }
 
   public void testUnOrderedProximitySearches() throws Exception {
 


[19/34] lucene-solr:jira/solr-9856: LUCENE-7611: Remove queries dependency from suggester module

Posted by ab...@apache.org.
LUCENE-7611: Remove queries dependency from suggester module


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8f4fee3a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8f4fee3a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8f4fee3a

Branch: refs/heads/jira/solr-9856
Commit: 8f4fee3ad1c0027587d0de96f59cf61b2df67bc8
Parents: ce8b678
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 12:32:17 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 13:07:13 2017 +0000

----------------------------------------------------------------------
 dev-tools/idea/lucene/suggest/suggest.iml       |  1 -
 .../lucene/search/DoubleValuesSource.java       | 27 ++++++++++++
 lucene/suggest/build.xml                        |  3 +-
 .../suggest/DocumentValueSourceDictionary.java  | 45 --------------------
 .../DocumentValueSourceDictionaryTest.java      | 25 +++++------
 5 files changed, 38 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/dev-tools/idea/lucene/suggest/suggest.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/suggest/suggest.iml b/dev-tools/idea/lucene/suggest/suggest.iml
index 576ea8b..5e58bc2 100644
--- a/dev-tools/idea/lucene/suggest/suggest.iml
+++ b/dev-tools/idea/lucene/suggest/suggest.iml
@@ -13,7 +13,6 @@
     <orderEntry type="sourceFolder" forTests="false" />
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
-    <orderEntry type="module" module-name="queries" />
     <orderEntry type="module" module-name="analysis-common" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
index 4ac8fc1..d4be4e9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -150,6 +150,33 @@ public abstract class DoubleValuesSource {
   };
 
   /**
+   * Creates a DoubleValuesSource that always returns a constant value
+   */
+  public static DoubleValuesSource constant(double value) {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          @Override
+          public double doubleValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  /**
    * Returns a DoubleValues instance that wraps scores returned by a Scorer
    */
   public static DoubleValues fromScorer(Scorer scorer) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/build.xml
----------------------------------------------------------------------
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index 5babe06..bc4bed5 100644
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -30,11 +30,10 @@
 
   <path id="classpath">
     <pathelement path="${analyzers-common.jar}"/>
-    <pathelement path="${queries.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 656dc04..2291ac9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -23,7 +23,6 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.LongValues;
 import org.apache.lucene.search.LongValuesSource;
 
@@ -63,21 +62,6 @@ import org.apache.lucene.search.LongValuesSource;
 public class DocumentValueSourceDictionary extends DocumentDictionary {
   
   private final LongValuesSource weightsValueSource;
-  
-  /**
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
-   * for the associated contexts and uses the <code>weightsValueSource</code> supplied 
-   * to determine the score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String, String)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource, String payload, String contexts) {
-    super(reader, field, null, payload, contexts);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
 
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
@@ -94,21 +78,6 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
-   * and uses the <code>weightsValueSource</code> supplied to determine the 
-   * score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource, String payload) {
-    super(reader, field, null, payload);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
-
-  /**
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms, <code>payloadField</code> for the corresponding payloads
    * and uses the <code>weightsValueSource</code> supplied to determine the
    * score.
    */
@@ -117,20 +86,6 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
     super(reader, field, null, payload);
     this.weightsValueSource = weightsValueSource;
   }
-  
-  /** 
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
-   * score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource) {
-    super(reader, field, null, null);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
 
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index 92799cd..55970e4 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -39,10 +39,6 @@ import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
-import org.apache.lucene.queries.function.valuesource.SumFloatFunction;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.LongValues;
 import org.apache.lucene.search.LongValuesSource;
@@ -73,7 +69,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.commit();
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
 
     assertNull(inputIterator.next());
@@ -119,8 +115,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -227,8 +223,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -305,8 +301,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd));
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -390,9 +386,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IndexReader ir = DirectoryReader.open(dir);
     assertTrue("NumDocs should be > 0 but was " + ir.numDocs(), ir.numDocs() > 0);
     assertEquals(ir.numDocs(), docs.size());
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2)};
-
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -478,7 +473,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {


[34/34] lucene-solr:jira/solr-9856: Merge branch 'master' into jira/solr-9856

Posted by ab...@apache.org.
Merge branch 'master' into jira/solr-9856


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8616344a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8616344a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8616344a

Branch: refs/heads/jira/solr-9856
Commit: 8616344ab47312c1c09c4a82ca888e896aa2c4ec
Parents: ca2e2be b9827bc
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 9 13:48:18 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 9 13:48:18 2017 +0100

----------------------------------------------------------------------
 dev-tools/idea/lucene/suggest/suggest.iml       |   1 -
 lucene/CHANGES.txt                              |  24 +
 .../lucene/analysis/TokenStreamToAutomaton.java |  11 +-
 .../apache/lucene/index/AutomatonTermsEnum.java |   3 +
 .../lucene/search/DoubleValuesSource.java       |  27 +
 .../apache/lucene/search/LongValuesSource.java  |  27 +
 .../lucene/analysis/TestGraphTokenizers.java    | 600 +++++++++++++++++++
 .../org/apache/lucene/index/TestTermsEnum.java  |   8 +
 .../demo/facet/DistanceFacetsExample.java       |  18 +-
 .../ExpressionAggregationFacetsExample.java     |   2 +-
 lucene/expressions/build.xml                    |  11 -
 .../org/apache/lucene/expressions/Bindings.java |  14 +-
 .../apache/lucene/expressions/Expression.java   |  21 +-
 .../expressions/ExpressionComparator.java       | 100 ----
 .../expressions/ExpressionFunctionValues.java   |  35 +-
 .../lucene/expressions/ExpressionRescorer.java  |  33 +-
 .../lucene/expressions/ExpressionSortField.java |  77 ---
 .../expressions/ExpressionValueSource.java      |  70 +--
 .../lucene/expressions/ScoreFunctionValues.java |  46 --
 .../lucene/expressions/ScoreValueSource.java    |  61 --
 .../lucene/expressions/SimpleBindings.java      |  30 +-
 .../expressions/js/JavascriptCompiler.java      |  15 +-
 .../apache/lucene/expressions/package-info.java |   5 +-
 .../lucene/expressions/TestDemoExpressions.java |  44 +-
 .../expressions/TestExpressionSortField.java    |   2 +-
 .../expressions/TestExpressionValueSource.java  | 111 +---
 .../expressions/js/TestCustomFunctions.java     |  20 +-
 .../expressions/js/TestJavascriptFunction.java  |   2 +-
 .../js/TestJavascriptOperations.java            |   2 +-
 .../org/apache/lucene/facet/DrillSideways.java  |  37 +-
 .../org/apache/lucene/facet/package-info.java   |   2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |  28 +-
 .../facet/range/DoubleRangeFacetCounts.java     |  63 +-
 .../apache/lucene/facet/range/LongRange.java    |  28 +-
 .../facet/range/LongRangeFacetCounts.java       |  26 +-
 .../org/apache/lucene/facet/range/Range.java    |  31 -
 .../lucene/facet/taxonomy/FakeScorer.java       |  53 --
 .../taxonomy/TaxonomyFacetSumValueSource.java   | 114 ++--
 .../facet/range/TestRangeFacetCounts.java       |  61 +-
 .../TestTaxonomyFacetSumValueSource.java        |  54 +-
 .../AbstractAllGroupHeadsCollector.java         | 176 ------
 .../grouping/AbstractAllGroupsCollector.java    |  67 ---
 .../AbstractDistinctValuesCollector.java        |  59 --
 .../AbstractFirstPassGroupingCollector.java     | 354 -----------
 .../grouping/AbstractGroupFacetCollector.java   | 319 ----------
 .../AbstractSecondPassGroupingCollector.java    | 162 -----
 .../search/grouping/AllGroupHeadsCollector.java | 176 ++++++
 .../search/grouping/AllGroupsCollector.java     |  67 +++
 .../search/grouping/CollectedSearchGroup.java   |   2 +-
 .../grouping/DistinctValuesCollector.java       |  59 ++
 .../grouping/FirstPassGroupingCollector.java    | 363 +++++++++++
 .../lucene/search/grouping/GroupDocs.java       |   8 +-
 .../search/grouping/GroupFacetCollector.java    | 324 ++++++++++
 .../apache/lucene/search/grouping/Grouper.java  |  56 ++
 .../lucene/search/grouping/GroupingSearch.java  | 130 +---
 .../lucene/search/grouping/SearchGroup.java     |  23 +-
 .../grouping/SecondPassGroupingCollector.java   | 169 ++++++
 .../lucene/search/grouping/TopGroups.java       |  12 +-
 .../FunctionAllGroupHeadsCollector.java         |  32 +-
 .../function/FunctionAllGroupsCollector.java    |   4 +-
 .../FunctionDistinctValuesCollector.java        |  35 +-
 .../FunctionFirstPassGroupingCollector.java     |   6 +-
 .../grouping/function/FunctionGrouper.java      |  69 +++
 .../FunctionSecondPassGroupingCollector.java    |   6 +-
 .../term/TermAllGroupHeadsCollector.java        |  26 +-
 .../grouping/term/TermAllGroupsCollector.java   |   8 +-
 .../term/TermDistinctValuesCollector.java       |  26 +-
 .../term/TermFirstPassGroupingCollector.java    |   6 +-
 .../grouping/term/TermGroupFacetCollector.java  |  10 +-
 .../search/grouping/term/TermGrouper.java       |  81 +++
 .../term/TermSecondPassGroupingCollector.java   |   6 +-
 .../grouping/AllGroupHeadsCollectorTest.java    |  14 +-
 .../search/grouping/AllGroupsCollectorTest.java |   6 +-
 .../grouping/DistinctValuesCollectorTest.java   |  46 +-
 .../grouping/GroupFacetCollectorTest.java       |  10 +-
 .../lucene/search/grouping/TestGrouping.java    |  70 +--
 .../uhighlight/LengthGoalBreakIterator.java     | 185 ++++++
 .../lucene/search/uhighlight/Passage.java       |   1 +
 .../uhighlight/LengthGoalBreakIteratorTest.java | 104 ++++
 .../lucene/queries/function/ValueSource.java    | 111 ++++
 .../complexPhrase/ComplexPhraseQueryParser.java |   4 +-
 .../complexPhrase/TestComplexPhraseQuery.java   |   6 +
 lucene/suggest/build.xml                        |   4 +-
 .../suggest/DocumentValueSourceDictionary.java  |  58 +-
 .../DocumentValueSourceDictionaryTest.java      | 287 ++++++++-
 .../lucene/analysis/TestGraphTokenizers.java    | 588 ------------------
 solr/CHANGES.txt                                |  23 +-
 solr/contrib/ltr/example/README.md              | 118 +++-
 solr/contrib/ltr/example/user_queries.txt       |  12 +-
 .../apache/solr/cloud/rule/ReplicaAssigner.java |  31 +-
 .../java/org/apache/solr/cloud/rule/Rule.java   |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |   5 +-
 .../org/apache/solr/core/DirectoryFactory.java  |   3 +-
 .../solr/core/MetricsDirectoryFactory.java      |  58 +-
 .../src/java/org/apache/solr/core/SolrCore.java |   4 +-
 .../org/apache/solr/core/SolrInfoMBean.java     |   4 +-
 .../solr/core/StandardDirectoryFactory.java     |   3 +-
 .../org/apache/solr/handler/GraphHandler.java   |   1 +
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../component/HttpShardHandlerFactory.java      |   8 +-
 .../solr/highlight/UnifiedSolrHighlighter.java  |  12 +-
 .../org/apache/solr/request/SimpleFacets.java   |   4 +-
 .../java/org/apache/solr/search/Grouping.java   |  14 +-
 .../org/apache/solr/search/facet/HLLAgg.java    |  13 +-
 .../apache/solr/search/facet/UniqueSlotAcc.java |   2 +-
 .../solr/search/grouping/CommandHandler.java    |   4 +-
 .../command/SearchGroupsFieldCommand.java       |   8 +-
 .../command/TopGroupsFieldCommand.java          |   4 +-
 .../DocumentExpressionDictionaryFactory.java    |   6 +-
 .../java/org/apache/solr/update/UpdateLog.java  |   4 +-
 .../apache/solr/update/UpdateShardHandler.java  |  11 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java  |   3 +-
 .../stats/InstrumentedHttpRequestExecutor.java  |   6 +-
 ...dd-schema-fields-update-processor-chains.xml |   8 +-
 .../collection1/conf/solrconfig-schemaless.xml  |  45 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |   2 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |   5 +-
 .../highlight/TestUnifiedSolrHighlighter.java   |  24 +-
 .../schema/TestSchemalessBufferedUpdates.java   | 160 +++++
 .../solr/search/facet/TestJsonFacets.java       |  27 +
 .../solr/update/SolrIndexMetricsTest.java       |   4 +-
 solr/example/files/conf/solrconfig.xml          |   5 +-
 .../basic_configs/conf/solrconfig.xml           |   6 +-
 .../conf/solrconfig.xml                         |   5 +-
 .../client/solrj/io/ops/EqualsOperation.java    |   2 +-
 .../io/ops/GreaterThanEqualToOperation.java     |   2 +-
 .../solrj/io/ops/GreaterThanOperation.java      |   2 +-
 .../solr/client/solrj/io/ops/LeafOperation.java |   1 -
 .../solrj/io/ops/LessThanEqualToOperation.java  |   2 +-
 .../client/solrj/io/ops/LessThanOperation.java  |   2 +-
 .../solrj/io/stream/expr/StreamFactory.java     |   2 +
 .../solr/common/params/HighlightParams.java     |   2 +-
 .../solrj/io/graph/GraphExpressionTest.java     |   3 +-
 .../solrj/io/stream/StreamExpressionTest.java   |   5 +-
 134 files changed, 3835 insertions(+), 3097 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8616344a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index 35797f3,421a4ef..04c8395
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@@ -32,7 -32,7 +32,7 @@@ public interface SolrInfoMBean 
    /**
     * Category of {@link SolrCore} component.
     */
-   enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, OTHER }
 -  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, HTTP, OTHER }
++  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
  
    /**
     * Top-level group of beans for a subsystem.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8616344a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 2598949,b79290d..c40eafd
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@@ -1472,16 -1427,9 +1472,16 @@@ public class UpdateLog implements Plugi
                loglog.error("REPLAY_ERR: Exception replaying log", rsp.getException());
                throw rsp.getException();
              }
 +            if (state == State.REPLAYING) {
 +              replayOpsMeter.mark();
 +            } else if (state == State.APPLYING_BUFFERED) {
 +              applyingBufferedOpsMeter.mark();
 +            } else {
 +              // XXX should not happen?
 +            }
            } catch (IOException ex) {
              recoveryInfo.errors++;
-             loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+             loglog.warn("REPLAY_ERR: IOException reading log", ex);
              // could be caused by an incomplete flush if recovering from log
            } catch (ClassCastException cl) {
              recoveryInfo.errors++;