You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2017/12/14 07:58:08 UTC

[1/5] lucene-solr:jira/solr-11702: SOLR-11702: When new collection is created, we should try to remove old terms nodes

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-11702 93767bb45 -> f881a62a6


SOLR-11702: When new collection is created, we should try to remove old terms nodes


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

Branch: refs/heads/jira/solr-11702
Commit: 09135b28b8cf9ba320af8d345a3413c376f12f5b
Parents: 93767bb
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu Dec 14 14:39:20 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu Dec 14 14:39:20 2017 +0700

----------------------------------------------------------------------
 .../org/apache/solr/cloud/CreateCollectionCmd.java |  9 ++++++++-
 .../java/org/apache/solr/cloud/ZkShardTerms.java   |  3 ++-
 .../org/apache/solr/cloud/ZkShardTermsTest.java    | 17 ++++++++++++-----
 3 files changed, 22 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/09135b28/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 2c4f01e..60ad61a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -378,7 +378,14 @@ public class CreateCollectionCmd implements Cmd {
   public static void createCollectionZkNode(SolrZkClient zkClient, String collection, Map<String,String> params) {
     log.debug("Check for collection zkNode:" + collection);
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
-
+    String termsPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/terms";
+    try {
+      if (zkClient.exists(termsPath, true)) {
+        zkClient.clean(termsPath);
+      }
+    } catch (KeeperException | InterruptedException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error deleting old term nodes for collection from Zookeeper", e);
+    }
     try {
       if (!zkClient.exists(collectionPath, true)) {
         log.debug("Creating collection in ZooKeeper:" + collection);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/09135b28/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index 4b69370..0a4cb9e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -36,6 +36,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreDescriptor;
@@ -66,7 +67,7 @@ public class ZkShardTerms implements AutoCloseable{
   }
 
   public ZkShardTerms(String collection, String shard, SolrZkClient client) {
-    this.znodePath = "/collections/" + collection + "/terms/" + shard;
+    this.znodePath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/terms/" + shard;
     this.collection = collection;
     this.shard = shard;
     this.stateManager = new ZkDistribStateManager(client);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/09135b28/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
index 3ef2adf..32a4723 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
@@ -27,11 +27,13 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Supplier;
 
+import org.apache.calcite.rel.core.Collect;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -58,6 +60,13 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
 
   public void testParticipationOfReplicas() throws IOException, SolrServerException {
     String collection = "collection1";
+    try (ZkShardTerms zkShardTerms = new ZkShardTerms(collection, "shard2", cluster.getZkClient())) {
+      zkShardTerms.registerTerm("replica1");
+      zkShardTerms.registerTerm("replica2");
+      zkShardTerms.ensureTermsIsHigher("replica1", Collections.singleton("replica2"));
+    }
+
+    // When new collection is created, the old term nodes will be removed
     CollectionAdminRequest.createCollection(collection, 2, 2)
         .setCreateNodeSet(cluster.getJettySolrRunner(0).getNodeName())
         .setMaxShardsPerNode(1000)
@@ -144,18 +153,16 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
       threads[i].start();
     }
 
-    AtomicInteger maxTerm = new AtomicInteger();
+    long maxTerm = 0;
     try (ZkShardTerms shardTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient())) {
       shardTerms.registerTerm("leader");
       TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
       while (!timeOut.hasTimedOut()) {
-        if (shardTerms.ensureTermsIsHigher("leader", new HashSet<>(failedReplicas))) {
-          maxTerm.incrementAndGet();
-        }
+        maxTerm++;
         assertEquals(shardTerms.getTerms().get("leader"), Collections.max(shardTerms.getTerms().values()));
         Thread.sleep(100);
       }
-      assertEquals(maxTerm.get(), Collections.max(shardTerms.getTerms().values()).longValue());
+      assertTrue(maxTerm >= Collections.max(shardTerms.getTerms().values()).longValue());
     }
     stop.set(true);
     for (Thread thread : threads) {


[4/5] lucene-solr:jira/solr-11702: merge master

Posted by da...@apache.org.
merge master


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

Branch: refs/heads/jira/solr-11702
Commit: e80554a87a2f27b1edaf486a1fab457631bff295
Parents: 09135b2
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu Dec 14 14:41:15 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu Dec 14 14:41:15 2017 +0700

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  10 +
 .../index/DocumentsWriterFlushControl.java      |   2 +-
 .../search/similarities/AfterEffectB.java       |  12 +-
 .../search/similarities/AfterEffectL.java       |   4 +-
 .../lucene/search/similarities/Axiomatic.java   |  62 ++-
 .../search/similarities/AxiomaticF1EXP.java     |  36 ++
 .../search/similarities/AxiomaticF1LOG.java     |  34 ++
 .../search/similarities/AxiomaticF2EXP.java     |  36 ++
 .../search/similarities/AxiomaticF2LOG.java     |  35 ++
 .../search/similarities/AxiomaticF3EXP.java     |  31 ++
 .../search/similarities/AxiomaticF3LOG.java     |  30 ++
 .../lucene/search/similarities/BasicModel.java  |  16 +-
 .../lucene/search/similarities/BasicModelG.java |  22 +
 .../search/similarities/BasicModelIF.java       |  15 +
 .../search/similarities/BasicModelIn.java       |  10 +-
 .../search/similarities/BasicModelIne.java      |  21 +
 .../search/similarities/BooleanSimilarity.java  |   2 +-
 .../search/similarities/DFISimilarity.java      |  34 ++
 .../search/similarities/DFRSimilarity.java      |  17 +-
 .../search/similarities/IBSimilarity.java       |  19 +-
 .../similarities/LMDirichletSimilarity.java     |  30 +-
 .../similarities/LMJelinekMercerSimilarity.java |  23 +
 .../lucene/search/similarities/LambdaDF.java    |   9 +-
 .../lucene/search/similarities/LambdaTTF.java   |  12 +-
 .../search/similarities/Normalization.java      |  10 +-
 .../search/similarities/NormalizationH1.java    |  17 +
 .../search/similarities/NormalizationH2.java    |  17 +
 .../search/similarities/NormalizationH3.java    |  19 +
 .../search/similarities/NormalizationZ.java     |  16 +
 .../spatial/spatial4j/Geo3dCircleShape.java     |  15 -
 .../spatial4j/Geo3dDistanceCalculator.java      |  60 +--
 .../lucene/spatial/spatial4j/Geo3dShape.java    |   6 +-
 .../spatial/spatial4j/Geo3dShapeFactory.java    |  43 +-
 .../Geo3dShapeRectRelationTestCase.java         | 264 ------------
 .../Geo3dShapeSphereModelRectRelationTest.java  |   8 +-
 .../Geo3dShapeWGS84ModelRectRelationTest.java   |  10 +-
 .../spatial4j/ShapeRectRelationTestCase.java    | 201 +++++++++
 .../lucene/spatial3d/geom/GeoCircleFactory.java |  29 +-
 .../lucene/spatial3d/geom/GeoExactCircle.java   |  55 +--
 .../spatial3d/geom/GeoStandardCircle.java       |   4 +-
 .../lucene/spatial3d/geom/PlanetModel.java      |   7 +
 solr/CHANGES.txt                                |  16 +-
 solr/bin/solr                                   |   4 +-
 .../solr/handler/component/FacetComponent.java  |  27 +-
 .../apache/solr/cloud/TestCloudPivotFacet.java  |   9 -
 .../DistributedFacetPivotLongTailTest.java      | 115 ++---
 ...ibutedFacetSimpleRefinementLongTailTest.java | 428 +++++++++++++++++++
 solr/solr-ref-guide/src/collections-api.adoc    |  22 +-
 solr/solr-ref-guide/src/installing-solr.adoc    |  14 +-
 solr/solr-ref-guide/src/solr-upgrade-notes.adoc |  35 +-
 .../src/solrcloud-autoscaling-api.adoc          |  73 +++-
 .../apache/solr/common/params/FacetParams.java  |   6 +-
 .../UsingSolrJRefGuideExamplesTest.java         |  75 +++-
 53 files changed, 1531 insertions(+), 596 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e629be0..c08e215 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -37,6 +37,9 @@ Changes in Runtime Behavior
   divide-by-zero hacks.  IndexSearcher.termStatistics/collectionStatistics return null
   instead of returning bogus values for a non-existent term or field. (Robert Muir)
 
+* LUCENE-7996: FunctionQuery and FunctionScoreQuery now return a score of 0
+  when the function produces a negative value. (Adrien Grand)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.
@@ -44,6 +47,9 @@ Improvements
   Add missing range checks for similarity parameters.
   Improve BM25 and ClassicSimilarity's explanations. (Robert Muir)
 
+* LUCENE-8011: Improved similarity explanations.
+  (Mayya Sharipova via Adrien Grand)
+
 Optimizations
 
 * LUCENE-8040: Optimize IndexSearcher.collectionStatistics, avoiding MultiFields/MultiTerms
@@ -66,6 +72,10 @@ Improvements
   disk. This change adds an expert setting to opt ouf of this behavior unless
   flusing is falling behind. (Simon Willnauer)
 
+* LUCENE-8086: spatial-extras Geo3dFactory: Use GeoExactCircle with
+  configurable precision for non-spherical planet models.
+  (Ignacio Vera via David Smiley)
+
 ======================= Lucene 7.2.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
index 761db0e..bf55991 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
@@ -716,7 +716,7 @@ final class DocumentsWriterFlushControl implements Accountable {
     return infoStream;
   }
 
-  ThreadState findLargestNonPendingWriter() {
+  synchronized ThreadState findLargestNonPendingWriter() {
     ThreadState maxRamUsingThreadState = null;
     long maxRamSoFar = 0;
     Iterator<ThreadState> activePerThreadsIterator = allActiveThreadStates();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectB.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectB.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectB.java
index 6678cd9..66ee1b9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectB.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectB.java
@@ -39,10 +39,14 @@ public class AfterEffectB extends AfterEffect {
   public final Explanation explain(BasicStats stats, double tfn) {
     return Explanation.match(
         (float) (scoreTimes1pTfn(stats) / (1 + tfn)),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match((float) tfn, "tfn"),
-        Explanation.match(stats.getTotalTermFreq(), "totalTermFreq"),
-        Explanation.match(stats.getDocFreq(), "docFreq"));
+        getClass().getSimpleName()
+            + ", computed as (F + 1) / (n * (tfn + 1)) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"),
+        Explanation.match(stats.getTotalTermFreq(),
+            "F, total number of occurrences of term across all documents + 1"),
+        Explanation.match(stats.getDocFreq(),
+            "n, number of documents containing term + 1"),
+        Explanation.match((float) tfn, "tfn, normalized term frequency"));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectL.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectL.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectL.java
index 60a1b1d..283c3ac 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectL.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AfterEffectL.java
@@ -37,8 +37,8 @@ public class AfterEffectL extends AfterEffect {
   public final Explanation explain(BasicStats stats, double tfn) {
     return Explanation.match(
         (float) (scoreTimes1pTfn(stats) / (1 + tfn)),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match((float) tfn, "tfn"));
+        getClass().getSimpleName() + ", computed as 1 / (tfn + 1) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"));
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
index 2a7f353..403773e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
@@ -118,16 +118,19 @@ public abstract class Axiomatic extends SimilarityBase {
   protected void explain(List<Explanation> subs, BasicStats stats, int doc,
                          double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
-      subs.add(Explanation.match((float) stats.getBoost(), "boost"));
+      subs.add(Explanation.match((float) stats.getBoost(),
+          "boost, query boost"));
     }
 
-    subs.add(Explanation.match(this.k, "k"));
-    subs.add(Explanation.match(this.s, "s"));
-    subs.add(Explanation.match(this.queryLen, "queryLen"));
-    subs.add(Explanation.match((float) tf(stats, freq, docLen), "tf"));
-    subs.add(Explanation.match((float) ln(stats, freq, docLen), "ln"));
-    subs.add(Explanation.match((float) tfln(stats, freq, docLen), "tfln"));
-    subs.add(Explanation.match((float) idf(stats, freq, docLen), "idf"));
+    subs.add(Explanation.match(this.k,
+        "k, hyperparam for the primitive weighting function"));
+    subs.add(Explanation.match(this.s,
+        "s, hyperparam for the growth function"));
+    subs.add(Explanation.match(this.queryLen, "queryLen, query length"));
+    subs.add(tfExplain(stats, freq, docLen));
+    subs.add(lnExplain(stats, freq, docLen));
+    subs.add(tflnExplain(stats, freq, docLen));
+    subs.add(idfExplain(stats, freq, docLen));
     subs.add(Explanation.match((float) gamma(stats, freq, docLen), "gamma"));
     super.explain(subs, stats, doc, freq, docLen);
   }
@@ -162,4 +165,47 @@ public abstract class Axiomatic extends SimilarityBase {
    * compute the gamma component (only for F3EXp and F3LOG)
    */
   protected abstract double gamma(BasicStats stats, double freq, double docLen);
+
+
+  /**
+   * Explain the score of the term frequency component for a single document
+   * @param stats the corpus level statistics
+   * @param freq number of occurrences of term in the document
+   * @param docLen the document length
+   * @return Explanation of how the tf component was computed
+   */
+  protected abstract Explanation tfExplain(BasicStats stats,
+                                           double freq, double docLen);
+
+  /**
+   * Explain the score of the document length component for a single document
+   * @param stats the corpus level statistics
+   * @param freq number of occurrences of term in the document
+   * @param docLen the document length
+   * @return Explanation of how the ln component was computed
+   */
+  protected abstract Explanation lnExplain(BasicStats stats,
+                                           double freq, double docLen);
+
+  /**
+   * Explain the score of the mixed term frequency and
+   * document length component for a single document
+   * @param stats the corpus level statistics
+   * @param freq number of occurrences of term in the document
+   * @param docLen the document length
+   * @return Explanation of how the tfln component was computed
+   */
+  protected abstract Explanation tflnExplain(BasicStats stats,
+                                             double freq, double docLen);
+
+  /**
+   * Explain the score of the inverted document frequency component
+   * for a single document
+   * @param stats the corpus level statistics
+   * @param freq number of occurrences of term in the document
+   * @param docLen the document length
+   * @return Explanation of how the idf component was computed
+   */
+  protected abstract Explanation idfExplain(BasicStats stats, double freq, double docLen);
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1EXP.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1EXP.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1EXP.java
index 0619b4e..c026feb 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1EXP.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1EXP.java
@@ -16,6 +16,9 @@
  */
 package org.apache.lucene.search.similarities;
 
+
+import org.apache.lucene.search.Explanation;
+
 /**
  * F1EXP is defined as Sum(tf(term_doc_freq)*ln(docLen)*IDF(term))
  * where IDF(t) = pow((N+1)/df(t), k) N=total num of docs, df=doc freq
@@ -92,4 +95,37 @@ public class AxiomaticF1EXP extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return 0.0;
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency computed as 1 + log(1 + log(freq)) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"));
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length computed as (avgdl + s) / (avgdl + dl * s) from:",
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgdl, average length of field across all documents"),
+        Explanation.match((float) docLen,
+            "dl, length of field"));
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, equals to 1");
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as " +
+            "Math.pow((N + 1) / n, k) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1LOG.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1LOG.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1LOG.java
index f7a02da..2e19255 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1LOG.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF1LOG.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.similarities;
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * F1LOG is defined as Sum(tf(term_doc_freq)*ln(docLen)*IDF(term))
  * where IDF(t) = ln((N+1)/df(t)) N=total num of docs, df=doc freq
@@ -85,4 +87,36 @@ public class AxiomaticF1LOG extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return 0.0;
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency computed as 1 + log(1 + log(freq)) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"));
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length computed as (avgdl + s) / (avgdl + dl * s) from:",
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgdl, average length of field across all documents"),
+        Explanation.match((float) docLen,
+            "dl, length of field"));
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, equals to 1");
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as log((N + 1) / n) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2EXP.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2EXP.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2EXP.java
index 0a3e4ad..bd28048 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2EXP.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2EXP.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.similarities;
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * F2EXP is defined as Sum(tfln(term_doc_freq, docLen)*IDF(term))
  * where IDF(t) = pow((N+1)/df(t), k) N=total num of docs, df=doc freq
@@ -91,4 +93,38 @@ public class AxiomaticF2EXP extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return 0.0;
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency, equals to 1");
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length, equals to 1");
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, " +
+        "computed as freq / (freq + s + s * dl / avgdl) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"),
+        Explanation.match((float) docLen,
+            "dl, length of field"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgdl, average length of field across all documents"));
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as " +
+            "Math.pow((N + 1) / n, k) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2LOG.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2LOG.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2LOG.java
index 2fc5e11..4780d1e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2LOG.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF2LOG.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.similarities;
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * F2EXP is defined as Sum(tfln(term_doc_freq, docLen)*IDF(term))
  * where IDF(t) = ln((N+1)/df(t)) N=total num of docs, df=doc freq
@@ -83,4 +85,37 @@ public class AxiomaticF2LOG extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return 0.0;
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency, equals to 1");
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length, equals to 1");
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, " +
+            "computed as freq / (freq + s + s * dl / avgdl) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"),
+        Explanation.match((float) docLen,
+            "dl, length of field"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgdl, average length of field across all documents"));
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as log((N + 1) / n) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3EXP.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3EXP.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3EXP.java
index 5c76317..635dc68 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3EXP.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3EXP.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.similarities;
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * F3EXP is defined as Sum(tf(term_doc_freq)*IDF(term)-gamma(docLen, queryLen))
  * where IDF(t) = pow((N+1)/df(t), k) N=total num of docs, df=doc freq
@@ -91,4 +93,33 @@ public class AxiomaticF3EXP extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return (docLen - this.queryLen) * this.s * this.queryLen / stats.getAvgFieldLength();
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency computed as 1 + log(1 + log(freq)) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"));
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length, equals to 1");
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, equals to 1");
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as " +
+            "Math.pow((N + 1) / n, k) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3LOG.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3LOG.java b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3LOG.java
index 22a50b1..4753e4e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3LOG.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/AxiomaticF3LOG.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search.similarities;
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * F3EXP is defined as Sum(tf(term_doc_freq)*IDF(term)-gamma(docLen, queryLen))
  * where IDF(t) = ln((N+1)/df(t)) N=total num of docs, df=doc freq
@@ -80,4 +82,32 @@ public class AxiomaticF3LOG extends Axiomatic {
   protected double gamma(BasicStats stats, double freq, double docLen) {
     return (docLen - this.queryLen) * this.s * this.queryLen / stats.getAvgFieldLength();
   }
+
+  @Override
+  protected Explanation tfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tf(stats, freq, docLen),
+        "tf, term frequency computed as 1 + log(1 + log(freq)) from:",
+        Explanation.match((float) freq,
+            "freq, number of occurrences of term in the document"));
+  };
+
+  @Override
+  protected Explanation lnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) ln(stats, freq, docLen),
+        "ln, document length, equals to 1");
+  };
+
+  protected Explanation tflnExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) tfln(stats, freq, docLen),
+        "tfln, mixed term frequency and document length, equals to 1");
+  };
+
+  protected Explanation idfExplain(BasicStats stats, double freq, double docLen){
+    return Explanation.match((float) idf(stats, freq, docLen),
+        "idf, inverted document frequency computed as log((N + 1) / n) from:",
+        Explanation.match((float) stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match((float) stats.getDocFreq(),
+            "n, number of documents containing term"));
+  };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModel.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModel.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModel.java
index 51d4571..7bcdd24 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModel.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModel.java
@@ -41,21 +41,13 @@ public abstract class BasicModel {
    * non-decreasing with {@code tfn}. */
   public abstract double score(BasicStats stats, double tfn, double aeTimes1pTfn);
   
+
   /**
    * Returns an explanation for the score.
-   * <p>Most basic models use the number of documents and the total term
-   * frequency to compute Inf<sub>1</sub>. This method provides a generic
-   * explanation for such models. Subclasses that use other statistics must
-   * override this method.</p>
+   * Subclasses must override this method.
    */
-  public Explanation explain(BasicStats stats, double tfn, double aeTimes1pTfn) {
-    return Explanation.match(
-        (float) (score(stats, tfn, aeTimes1pTfn) * (1 + tfn) / aeTimes1pTfn),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match(stats.getNumberOfDocuments(), "numberOfDocuments"),
-        Explanation.match(stats.getTotalTermFreq(), "totalTermFreq"));
-  }
-  
+  public abstract Explanation explain (BasicStats stats, double tfn, double aeTimes1pTfn);
+
   /**
    * Subclasses must override this method to return the code of the
    * basic model formula. Refer to the original paper for the list. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelG.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelG.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelG.java
index ce87196..72c0219 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelG.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelG.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 import static org.apache.lucene.search.similarities.SimilarityBase.log2;
 
 /**
@@ -49,6 +51,26 @@ public class BasicModelG extends BasicModel {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tfn, double aeTimes1pTfn) {
+    double F = stats.getTotalTermFreq() + 1;
+    double N = stats.getNumberOfDocuments();
+    double lambda = F / (N + F);
+    Explanation explLambda = Explanation.match((float) lambda,
+        "lambda, computed as F / (N + F) from:",
+        Explanation.match((float) F,
+            "F, total number of occurrences of term across all docs + 1"),
+        Explanation.match((float) N,
+            "N, total number of documents with field"));
+
+    return Explanation.match(
+        (float) (score(stats, tfn, aeTimes1pTfn) * (1 + tfn) / aeTimes1pTfn),
+        getClass().getSimpleName() + ", computed as " +
+            "log2(lambda + 1) + tfn * log2((1 + lambda) / lambda) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"),
+        explLambda);
+  }
+
+  @Override
   public String toString() {
     return "G";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIF.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIF.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIF.java
index 16781cd..974420e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIF.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIF.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 import static org.apache.lucene.search.similarities.SimilarityBase.log2;
 
 /**
@@ -43,6 +45,19 @@ public class BasicModelIF extends BasicModel {
   }
 
   @Override
+    public Explanation explain(BasicStats stats, double tfn, double aeTimes1pTfn) {
+    return Explanation.match(
+        (float) (score(stats, tfn, aeTimes1pTfn) * (1 + tfn) / aeTimes1pTfn),
+        getClass().getSimpleName() + ", computed as " +
+            "tfn * log2(1 + (N + 1) / (F + 0.5)) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"),
+        Explanation.match(stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match(stats.getTotalTermFreq(),
+            "F, total number of occurrences of term across all documents"));
+  }
+
+  @Override
   public String toString() {
     return "I(F)";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIn.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIn.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIn.java
index 5f1e181..82b65f4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIn.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIn.java
@@ -47,9 +47,13 @@ public class BasicModelIn extends BasicModel {
   public final Explanation explain(BasicStats stats, double tfn, double aeTimes1pTfn) {
     return Explanation.match(
         (float) (score(stats, tfn, aeTimes1pTfn) * (1 + tfn) / aeTimes1pTfn),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match(stats.getNumberOfDocuments(), "numberOfDocuments"),
-        Explanation.match(stats.getDocFreq(), "docFreq"));
+        getClass().getSimpleName() +
+            ", computed as tfn * log2((N + 1) / (n + 0.5)) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"),
+        Explanation.match(stats.getNumberOfDocuments(),
+            "N, total number of documents with field"),
+        Explanation.match(stats.getDocFreq(),
+            "n, number of documents containing term"));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIne.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIne.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIne.java
index fb755fa..c846976 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIne.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicModelIne.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
 import static org.apache.lucene.search.similarities.SimilarityBase.log2;
 
 /**
@@ -45,6 +46,26 @@ public class BasicModelIne extends BasicModel {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tfn, double aeTimes1pTfn) {
+    double F = stats.getTotalTermFreq();
+    double N = stats.getNumberOfDocuments();
+    double ne = N * (1 - Math.pow((N - 1) / N, F));
+    Explanation explNe = Explanation.match((float) ne,
+        "ne, computed as N * (1 - Math.pow((N - 1) / N, F)) from:",
+        Explanation.match((float) F,
+            "F, total number of occurrences of term across all docs"),
+        Explanation.match((float) N,
+            "N, total number of documents with field"));
+
+    return Explanation.match(
+        (float) (score(stats, tfn, aeTimes1pTfn) * (1 + tfn) / aeTimes1pTfn),
+        getClass().getSimpleName() + ", computed as " +
+            "tfn * log2((N + 1) / (ne + 0.5)) from:",
+        Explanation.match((float) tfn, "tfn, normalized term frequency"),
+        explNe);
+  }
+
+  @Override
   public String toString() {
     return "I(ne)";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
index 35f7083..7134172 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
@@ -77,7 +77,7 @@ public class BooleanSimilarity extends Similarity {
 
       @Override
       public Explanation explain(int doc, Explanation freq) throws IOException {
-        Explanation queryBoostExpl = Explanation.match(boost, "boost");
+        Explanation queryBoostExpl = Explanation.match(boost, "boost, query boost");
         return Explanation.match(
             queryBoostExpl.getValue(),
             "score(" + getClass().getSimpleName() + ", doc=" + doc + "), computed from:",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
index ca0f4aa..6ebe9cb 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * Implements the <em>Divergence from Independence (DFI)</em> model based on Chi-square statistics
  * (i.e., standardized Chi-squared distance from independence in term frequency tf).
@@ -76,6 +78,38 @@ public class DFISimilarity extends SimilarityBase {
   }
 
   @Override
+  protected Explanation explain(
+      BasicStats stats, int doc, Explanation freq, double docLen) {
+    final double expected = (stats.getTotalTermFreq() + 1) * docLen /
+        (stats.getNumberOfFieldTokens() + 1);
+    if (freq.getValue() <= expected){
+      return Explanation.match((float) 0, "score(" +
+          getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+          freq.getValue() +"), equals to 0");
+    }
+    Explanation explExpected = Explanation.match((float) expected,
+        "expected, computed as (F + 1) * dl / (T + 1) from:",
+        Explanation.match(stats.getTotalTermFreq(),
+            "F, total number of occurrences of term across all docs"),
+        Explanation.match((float) docLen, "dl, length of field"),
+        Explanation.match(stats.getNumberOfFieldTokens(),
+            "T, total number of tokens in the field"));
+
+    final double measure = independence.score(freq.getValue(), expected);
+    Explanation explMeasure = Explanation.match((float) measure,
+        "measure, computed as independence.score(freq, expected) from:",
+        freq,
+        explExpected);
+
+    return Explanation.match(
+        (float) score(stats, freq.getValue(), docLen),
+        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+            freq.getValue() +"), computed as boost * log2(measure + 1) from:",
+        Explanation.match( (float)stats.getBoost(), "boost, query boost"),
+        explMeasure);
+  }
+
+  @Override
   public String toString() {
     return "DFI(" + independence + ")";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
index 788f30a..8c73e6e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search.similarities;
 
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.Explanation;
@@ -122,7 +123,7 @@ public class DFRSimilarity extends SimilarityBase {
   protected void explain(List<Explanation> subs,
       BasicStats stats, int doc, double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
-      subs.add(Explanation.match( (float)stats.getBoost(), "boost"));
+      subs.add(Explanation.match( (float)stats.getBoost(), "boost, query boost"));
     }
     
     Explanation normExpl = normalization.explain(stats, freq, docLen);
@@ -134,6 +135,20 @@ public class DFRSimilarity extends SimilarityBase {
   }
 
   @Override
+  protected Explanation explain(
+      BasicStats stats, int doc, Explanation freq, double docLen) {
+    List<Explanation> subs = new ArrayList<>();
+    explain(subs, stats, doc, freq.getValue(), docLen);
+
+    return Explanation.match(
+        (float) score(stats, freq.getValue(), docLen),
+        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+            freq.getValue() +"), computed as boost * " +
+            "basicModel.score(stats, tfn) * afterEffect.score(stats, tfn) from:",
+        subs);
+  }
+
+  @Override
   public String toString() {
     return "DFR " + basicModel.toString() + afterEffect.toString()
                   + normalization.toString();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
index a71614c..3e9ba0c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search.similarities;
 
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.Explanation;
@@ -113,7 +114,7 @@ public class IBSimilarity extends SimilarityBase {
   protected void explain(
       List<Explanation> subs, BasicStats stats, int doc, double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
-      subs.add(Explanation.match((float)stats.getBoost(), "boost"));
+      subs.add(Explanation.match((float)stats.getBoost(), "boost, query boost"));
     }
     Explanation normExpl = normalization.explain(stats, freq, docLen);
     Explanation lambdaExpl = lambda.explain(stats);
@@ -121,6 +122,22 @@ public class IBSimilarity extends SimilarityBase {
     subs.add(lambdaExpl);
     subs.add(distribution.explain(stats, normExpl.getValue(), lambdaExpl.getValue()));
   }
+
+  @Override
+  protected Explanation explain(
+      BasicStats stats, int doc, Explanation freq, double docLen) {
+    List<Explanation> subs = new ArrayList<>();
+    explain(subs, stats, doc, freq.getValue(), docLen);
+
+    return Explanation.match(
+        (float) score(stats, freq.getValue(), docLen),
+        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+            freq.getValue() +"), computed as boost * " +
+            "distribution.score(stats, normalization.tfn(stats, freq," +
+            " docLen), lambda.lambda(stats)) from:",
+        subs);
+  }
+
   
   /**
    * The name of IB methods follow the pattern

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
index 2a4354e..be7329b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search.similarities;
 
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
@@ -86,20 +87,43 @@ public class LMDirichletSimilarity extends LMSimilarity {
   protected void explain(List<Explanation> subs, BasicStats stats, int doc,
       double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
-      subs.add(Explanation.match((float) stats.getBoost(), "boost"));
+      subs.add(Explanation.match((float) stats.getBoost(), "query boost"));
     }
+    double p = ((LMStats)stats).getCollectionProbability();
+    Explanation explP = Explanation.match((float) p,
+        "P, probability that the current term is generated by the collection");
+    Explanation explFreq = Explanation.match((float) freq,
+        "freq, number of occurrences of term in the document");
 
     subs.add(Explanation.match(mu, "mu"));
     Explanation weightExpl = Explanation.match(
         (float)Math.log(1 + freq /
         (mu * ((LMStats)stats).getCollectionProbability())),
-        "term weight");
+        "term weight, computed as log(1 + freq /(mu * P)) from:",
+        explFreq,
+        explP);
     subs.add(weightExpl);
     subs.add(Explanation.match(
-        (float)Math.log(mu / (docLen + mu)), "document norm"));
+        (float)Math.log(mu / (docLen + mu)),
+        "document norm, computed as log(mu / (dl + mu))"));
+    subs.add(Explanation.match((float) docLen,"dl, length of field"));
     super.explain(subs, stats, doc, freq, docLen);
   }
 
+  @Override
+  protected Explanation explain(
+      BasicStats stats, int doc, Explanation freq, double docLen) {
+    List<Explanation> subs = new ArrayList<>();
+    explain(subs, stats, doc, freq.getValue(), docLen);
+
+    return Explanation.match(
+        (float) score(stats, freq.getValue(), docLen),
+        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+            freq.getValue() +"), computed as boost * " +
+            "(term weight + document norm) from:",
+        subs);
+  }
+
   /** Returns the &mu; parameter. */
   public float getMu() {
     return mu;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
index fa0ebcf..f6db238 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search.similarities;
 
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
@@ -79,9 +80,31 @@ public class LMJelinekMercerSimilarity extends LMSimilarity {
       subs.add(Explanation.match((float) stats.getBoost(), "boost"));
     }
     subs.add(Explanation.match(lambda, "lambda"));
+    double p = ((LMStats)stats).getCollectionProbability();
+    Explanation explP = Explanation.match((float) p,
+        "P, probability that the current term is generated by the collection");
+    subs.add(explP);
+    Explanation explFreq = Explanation.match((float) freq,
+        "freq, number of occurrences of term in the document");
+    subs.add(explFreq);
+    subs.add(Explanation.match((float) docLen,"dl, length of field"));
     super.explain(subs, stats, doc, freq, docLen);
   }
 
+  @Override
+  protected Explanation explain(
+      BasicStats stats, int doc, Explanation freq, double docLen) {
+    List<Explanation> subs = new ArrayList<>();
+    explain(subs, stats, doc, freq.getValue(), docLen);
+
+    return Explanation.match(
+        (float) score(stats, freq.getValue(), docLen),
+        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+            freq.getValue() +"), computed as boost * " +
+            "log(1 + ((1 - lambda) * freq / dl) /(lambda * P)) from:",
+        subs);
+  }
+
   /** Returns the &lambda; parameter. */
   public float getLambda() {
     return lambda;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaDF.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaDF.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaDF.java
index ef87c8a..7dc320d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaDF.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaDF.java
@@ -37,9 +37,12 @@ public class LambdaDF extends Lambda {
   public final Explanation explain(BasicStats stats) {
     return Explanation.match(
         lambda(stats),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match(stats.getDocFreq(), "docFreq"),
-        Explanation.match(stats.getNumberOfDocuments(), "numberOfDocuments"));
+        getClass().getSimpleName()
+            + ", computed as (n + 1) / (N + 1) from:",
+        Explanation.match(stats.getDocFreq(),
+            "n, number of documents containing term"),
+        Explanation.match(stats.getNumberOfDocuments(),
+            "N, total number of documents with field"));
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaTTF.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaTTF.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaTTF.java
index 20fe652..6dc54a3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaTTF.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LambdaTTF.java
@@ -33,15 +33,17 @@ public class LambdaTTF extends Lambda {
     return (stats.getTotalTermFreq()+1F) / (stats.getNumberOfDocuments()+1F);
   }
 
-  @Override
   public final Explanation explain(BasicStats stats) {
     return Explanation.match(
         lambda(stats),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match(stats.getTotalTermFreq(), "totalTermFreq"),
-        Explanation.match(stats.getNumberOfDocuments(), "numberOfDocuments"));
+        getClass().getSimpleName()
+            + ", computed as (F + 1) / (N + 1) from:",
+        Explanation.match(stats.getTotalTermFreq(),
+            "F, total number of occurrences of term across all documents"),
+        Explanation.match(stats.getNumberOfDocuments(),
+            "N, total number of documents with field"));
   }
-  
+
   @Override
   public String toString() {
     return "L";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/Normalization.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Normalization.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Normalization.java
index e20ca02..6bde17d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/Normalization.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Normalization.java
@@ -47,10 +47,12 @@ public abstract class Normalization {
   public Explanation explain(BasicStats stats, double tf, double len) {
     return Explanation.match(
         (float) tfn(stats, tf, len),
-        getClass().getSimpleName() + ", computed from: ",
-        Explanation.match((float) tf, "tf"),
-        Explanation.match((float) stats.getAvgFieldLength(), "avgFieldLength"),
-        Explanation.match((float) len, "len"));
+        getClass().getSimpleName() + ", computed from:",
+        Explanation.match((float) tf,
+            "tf, number of occurrences of term in the document"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgfl, average length of field across all documents"),
+        Explanation.match((float) len, "fl, field length of the document"));
   }
 
   /** Implementation used when there is no normalization. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java
index 8e5a28f..5fa4f1f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * Normalization model that assumes a uniform distribution of the term frequency.
  * <p>While this model is parameterless in the
@@ -56,6 +58,21 @@ public class NormalizationH1 extends Normalization {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tf, double len) {
+    return Explanation.match(
+        (float) tfn(stats, tf, len),
+        getClass().getSimpleName()
+            + ", computed as tf * c * (avgfl / fl) from:",
+        Explanation.match((float) tf,
+            "tf, number of occurrences of term in the document"),
+        Explanation.match(c,
+            "c, hyper-parameter"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgfl, average length of field across all documents"),
+        Explanation.match((float) len, "fl, field length of the document"));
+  }
+
+  @Override
   public String toString() {
     return "1";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH2.java b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH2.java
index 24fb74e..a30bd4b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH2.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH2.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 import static org.apache.lucene.search.similarities.SimilarityBase.log2;
 
 /**
@@ -58,6 +60,21 @@ public class NormalizationH2 extends Normalization {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tf, double len) {
+    return Explanation.match(
+        (float) tfn(stats, tf, len),
+        getClass().getSimpleName()
+            + ", computed as tf * log2(1 + c * avgfl / fl) from:",
+        Explanation.match((float) tf,
+            "tf, number of occurrences of term in the document"),
+        Explanation.match(c,
+            "c, hyper-parameter"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgfl, average length of field across all documents"),
+        Explanation.match((float) len, "fl, field length of the document"));
+  }
+
+  @Override
   public String toString() {
     return "2";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH3.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH3.java b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH3.java
index 0bbea49..64c9ec3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH3.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH3.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * Dirichlet Priors normalization
  * @lucene.experimental
@@ -48,6 +50,23 @@ public class NormalizationH3 extends Normalization {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tf, double len) {
+    return Explanation.match(
+        (float) tfn(stats, tf, len),
+        getClass().getSimpleName()
+            + ", computed as (tf + mu * ((F+1) / (T+1))) / (fl + mu) * mu from:",
+        Explanation.match((float) tf,
+            "tf, number of occurrences of term in the document"),
+        Explanation.match(mu,
+            "mu, smoothing parameter"),
+        Explanation.match((float) stats.getTotalTermFreq(),
+            "F,  total number of occurrences of term across all documents"),
+        Explanation.match((float) stats.getNumberOfFieldTokens(),
+            "T, total number of tokens of the field across all documents"),
+        Explanation.match((float) len, "fl, field length of the document"));
+  }
+
+  @Override
   public String toString() {
     return "3(" + mu + ")";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationZ.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationZ.java b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationZ.java
index dabf9c9..ca7108c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationZ.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationZ.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
+import org.apache.lucene.search.Explanation;
+
 /**
  * Pareto-Zipf Normalization
  * @lucene.experimental
@@ -49,6 +51,20 @@ public class NormalizationZ extends Normalization {
   }
 
   @Override
+  public Explanation explain(BasicStats stats, double tf, double len) {
+    return Explanation.match(
+        (float) tfn(stats, tf, len),
+        getClass().getSimpleName()
+            + ", computed as tf * Math.pow(avgfl / fl, z) from:",
+        Explanation.match((float) tf,
+            "tf, number of occurrences of term in the document"),
+        Explanation.match((float) stats.getAvgFieldLength(),
+            "avgfl, average length of field across all documents"),
+        Explanation.match((float) len, "fl, field length of the document"),
+        Explanation.match(z, "z, relates to specificity of the language"));
+  }
+
+  @Override
   public String toString() {
     return "Z(" + z + ")";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dCircleShape.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dCircleShape.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dCircleShape.java
index ccef92a..d01e2b8 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dCircleShape.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dCircleShape.java
@@ -20,13 +20,10 @@ package org.apache.lucene.spatial.spatial4j;
 import org.apache.lucene.spatial3d.geom.GeoCircle;
 import org.apache.lucene.spatial3d.geom.GeoCircleFactory;
 import org.apache.lucene.spatial3d.geom.GeoPointShapeFactory;
-import org.apache.lucene.spatial3d.geom.PlanetModel;
 import org.locationtech.spatial4j.context.SpatialContext;
 import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.locationtech.spatial4j.shape.Circle;
 import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
-import org.locationtech.spatial4j.shape.SpatialRelation;
 
 /**
  * Specialization of a {@link Geo3dShape} which represents a {@link Circle}.
@@ -67,16 +64,4 @@ public class Geo3dCircleShape extends Geo3dShape<GeoCircle> implements Circle {
     }
     return center;
   }
-
-  //TODO Improve GeoCircle to properly relate a point with WGS84 model -- LUCENE-7970
-  @Override
-  public SpatialRelation relate(Shape other) {
-    if (shape.getPlanetModel() != PlanetModel.SPHERE && other instanceof Point) {
-      if (spatialcontext.getDistCalc().distance((Point) other, getCenter()) <= getRadius()) {
-        return SpatialRelation.CONTAINS;
-      }
-      return SpatialRelation.DISJOINT;
-    }
-    return super.relate(other);
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dDistanceCalculator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dDistanceCalculator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dDistanceCalculator.java
index 5154de4..8fdb481 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dDistanceCalculator.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dDistanceCalculator.java
@@ -73,62 +73,20 @@ public class Geo3dDistanceCalculator implements DistanceCalculator {
 
   @Override
   public Point pointOnBearing(Point from, double distDEG, double bearingDEG, SpatialContext ctx, Point reuse) {
-    // Algorithm using Vincenty's formulae (https://en.wikipedia.org/wiki/Vincenty%27s_formulae)
-    // which takes into account that planets may not be spherical.
-    //Code adaptation from http://www.movable-type.co.uk/scripts/latlong-vincenty.html
     Geo3dPointShape geoFrom = (Geo3dPointShape) from;
     GeoPoint point = (GeoPoint) geoFrom.shape;
-    double lat = point.getLatitude();
-    double lon = point.getLongitude();
     double dist = DistanceUtils.DEGREES_TO_RADIANS * distDEG;
     double bearing = DistanceUtils.DEGREES_TO_RADIANS * bearingDEG;
-
-    double sinα1 = Math.sin(bearing);
-    double cosα1 = Math.cos(bearing);
-
-    double tanU1 = (1 - planetModel.flattening) * Math.tan(lat);
-    double cosU1 = 1 / Math.sqrt((1 + tanU1 * tanU1));
-    double sinU1 = tanU1 * cosU1;
-
-    double σ1 = Math.atan2(tanU1, cosα1);
-    double sinα = cosU1 * sinα1;
-    double cosSqα = 1 - sinα * sinα;
-    double uSq = cosSqα * planetModel.squareRatio;// (planetModel.ab* planetModel.ab - planetModel.c*planetModel.c) / (planetModel.c*planetModel.c);
-    double A = 1 + uSq / 16384 * (4096 + uSq * (-768 + uSq * (320 - 175 * uSq)));
-    double B = uSq / 1024 * (256 + uSq * (-128 + uSq * (74 - 47 * uSq)));
-
-    double cos2σM;
-    double sinσ;
-    double cosσ;
-    double Δσ;
-
-    double σ = dist / (planetModel.c * A);
-    double σʹ;
-    double iterations = 0;
-    do {
-      cos2σM = Math.cos(2 * σ1 + σ);
-      sinσ = Math.sin(σ);
-      cosσ = Math.cos(σ);
-      Δσ = B * sinσ * (cos2σM + B / 4 * (cosσ * (-1 + 2 * cos2σM * cos2σM) -
-          B / 6 * cos2σM * (-3 + 4 * sinσ * sinσ) * (-3 + 4 * cos2σM * cos2σM)));
-      σʹ = σ;
-      σ = dist / (planetModel.c * A) + Δσ;
-    } while (Math.abs(σ - σʹ) > 1e-12 && ++iterations < 200);
-
-    if (iterations >= 200) {
-      throw new RuntimeException("Formula failed to converge");
+    GeoPoint newPoint = planetModel.surfacePointOnBearing(point, dist, bearing);
+    double newLat = newPoint.getLatitude() * DistanceUtils.RADIANS_TO_DEGREES;
+    double newLon = newPoint.getLongitude() * DistanceUtils.RADIANS_TO_DEGREES;
+    if (reuse != null) {
+      reuse.reset(newLon, newLat);
+      return reuse;
+    }
+    else {
+      return ctx.getShapeFactory().pointXY(newLon, newLat);
     }
-
-    double x = sinU1 * sinσ - cosU1 * cosσ * cosα1;
-    double φ2 = Math.atan2(sinU1 * cosσ + cosU1 * sinσ * cosα1, (1 - planetModel.flattening) * Math.sqrt(sinα * sinα + x * x));
-    double λ = Math.atan2(sinσ * sinα1, cosU1 * cosσ - sinU1 * sinσ * cosα1);
-    double C = planetModel.flattening / 16 * cosSqα * (4 + planetModel.flattening * (4 - 3 * cosSqα));
-    double L = λ - (1 - C) * planetModel.flattening * sinα *
-        (σ + C * sinσ * (cos2σM + C * cosσ * (-1 + 2 * cos2σM * cos2σM)));
-    double λ2 = (lon + L + 3 * Math.PI) % (2 * Math.PI) - Math.PI;  // normalise to -180..+180
-
-    return ctx.getShapeFactory().pointXY(λ2 * DistanceUtils.RADIANS_TO_DEGREES,
-        φ2 * DistanceUtils.RADIANS_TO_DEGREES);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
index eedf7d6..327ac8f 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
@@ -112,11 +112,7 @@ public class Geo3dShape<T extends GeoAreaShape> implements Shape {
     if (bbox == null) {
       LatLonBounds bounds = new LatLonBounds();
       shape.getBounds(bounds);
-      double leftLon = bounds.checkNoLongitudeBound() ? -Math.PI : bounds.getLeftLongitude();
-      double rightLon = bounds.checkNoLongitudeBound() ? Math.PI : bounds.getRightLongitude();
-      double minLat = bounds.checkNoBottomLatitudeBound() ? -Math.PI * 0.5 : bounds.getMinLatitude();
-      double maxLat = bounds.checkNoTopLatitudeBound() ? Math.PI * 0.5 : bounds.getMaxLatitude();
-      GeoBBox geoBBox = GeoBBoxFactory.makeGeoBBox(shape.getPlanetModel(), maxLat, minLat, leftLon, rightLon);
+      GeoBBox geoBBox = GeoBBoxFactory.makeGeoBBox(shape.getPlanetModel(), bounds);
       bbox = new Geo3dRectangleShape(geoBBox, spatialcontext);
       this.boundingBox = bbox;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShapeFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShapeFactory.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShapeFactory.java
index a80a043..282d93b 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShapeFactory.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShapeFactory.java
@@ -55,6 +55,13 @@ public class Geo3dShapeFactory implements ShapeFactory {
   private SpatialContext context;
   private PlanetModel planetModel;
 
+  /**
+   * Default accuracy for circles when not using the unit sphere.
+   * It is equivalent to ~10m on the surface of the earth.
+   */
+  private static final double DEFAULT_CIRCLE_ACCURACY = 1e-4;
+  private double circleAccuracy = DEFAULT_CIRCLE_ACCURACY;
+
   @SuppressWarnings("unchecked")
   public Geo3dShapeFactory(SpatialContext context, SpatialContextFactory factory) {
     this.context = context;
@@ -67,6 +74,16 @@ public class Geo3dShapeFactory implements ShapeFactory {
     return context;
   }
 
+  /**
+   * Set the accuracy for circles in decimal degrees. Note that accuracy has no effect
+   * when the planet model is a sphere. In that case, circles are always fully precise.
+   *
+   * @param circleAccuracy the provided accuracy in decimal degrees.
+   */
+  public void setCircleAccuracy(double circleAccuracy) {
+    this.circleAccuracy = circleAccuracy;
+  }
+
   @Override
   public boolean isNormWrapLongitude() {
     return normWrapLongitude;
@@ -150,10 +167,23 @@ public class Geo3dShapeFactory implements ShapeFactory {
 
   @Override
   public Circle circle(double x, double y, double distance) {
-    GeoCircle circle = GeoCircleFactory.makeGeoCircle(planetModel,
-        y * DistanceUtils.DEGREES_TO_RADIANS,
-        x * DistanceUtils.DEGREES_TO_RADIANS,
-        distance * DistanceUtils.DEGREES_TO_RADIANS);
+    GeoCircle circle;
+    if (planetModel.isSphere()) {
+      circle = GeoCircleFactory.makeGeoCircle(planetModel,
+          y * DistanceUtils.DEGREES_TO_RADIANS,
+          x * DistanceUtils.DEGREES_TO_RADIANS,
+          distance * DistanceUtils.DEGREES_TO_RADIANS);
+    }
+    else {
+      //accuracy is defined as a linear distance in this class. At tiny distances, linear distance
+      //can be approximated to surface distance in radians.
+      circle = GeoCircleFactory.makeExactGeoCircle(planetModel,
+          y * DistanceUtils.DEGREES_TO_RADIANS,
+          x * DistanceUtils.DEGREES_TO_RADIANS,
+          distance * DistanceUtils.DEGREES_TO_RADIANS,
+          circleAccuracy * DistanceUtils.DEGREES_TO_RADIANS);
+
+    }
     return new Geo3dCircleShape(circle, context);
   }
 
@@ -238,8 +268,7 @@ public class Geo3dShapeFactory implements ShapeFactory {
 
   /**
    * Geo3d implementation of {@link org.locationtech.spatial4j.shape.ShapeFactory.LineStringBuilder} to generate
-   * nine Strings. Note that GeoPath needs a buffer so we set the
-   * buffer to 1e-10.
+   * line strings.
    */
   private class Geo3dLineStringBuilder extends Geo3dPointBuilder<LineStringBuilder> implements LineStringBuilder {
 
@@ -373,7 +402,7 @@ public class Geo3dShapeFactory implements ShapeFactory {
 
   /**
    * Geo3d implementation of {@link org.locationtech.spatial4j.shape.ShapeFactory.MultiShapeBuilder} to generate
-   * geometry collections
+   * geometry collections.
    *
    * @param <T> is the type of shapes.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
deleted file mode 100644
index 9873012..0000000
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
+++ /dev/null
@@ -1,264 +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.spatial.spatial4j;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.spatial3d.geom.GeoPath;
-import org.apache.lucene.spatial3d.geom.GeoPolygon;
-import org.locationtech.spatial4j.TestLog;
-import org.locationtech.spatial4j.context.SpatialContext;
-import org.locationtech.spatial4j.distance.DistanceUtils;
-import org.locationtech.spatial4j.shape.Circle;
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.RectIntersectionTestHelper;
-import org.apache.lucene.spatial3d.geom.LatLonBounds;
-import org.apache.lucene.spatial3d.geom.GeoBBox;
-import org.apache.lucene.spatial3d.geom.GeoBBoxFactory;
-import org.apache.lucene.spatial3d.geom.GeoCircle;
-import org.apache.lucene.spatial3d.geom.GeoCircleFactory;
-import org.apache.lucene.spatial3d.geom.GeoPathFactory;
-import org.apache.lucene.spatial3d.geom.GeoPoint;
-import org.apache.lucene.spatial3d.geom.GeoPolygonFactory;
-import org.apache.lucene.spatial3d.geom.GeoShape;
-import org.apache.lucene.spatial3d.geom.PlanetModel;
-import org.junit.Rule;
-import org.junit.Test;
-
-import static org.locationtech.spatial4j.distance.DistanceUtils.DEGREES_TO_RADIANS;
-
-public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTestCase {
-  protected final static double RADIANS_PER_DEGREE = Math.PI/180.0;
-
-  @Rule
-  public final TestLog testLog = TestLog.instance;
-
-  protected final PlanetModel planetModel;
-
-  public Geo3dShapeRectRelationTestCase(PlanetModel planetModel) {
-    super(SpatialContext.GEO);
-    this.planetModel = planetModel;
-  }
-
-  protected GeoBBox getBoundingBox(final GeoShape path) {
-    LatLonBounds bounds = new LatLonBounds();
-    path.getBounds(bounds);
-
-    double leftLon;
-    double rightLon;
-    if (bounds.checkNoLongitudeBound()) {
-      leftLon = -Math.PI;
-      rightLon = Math.PI;
-    } else {
-      leftLon = bounds.getLeftLongitude().doubleValue();
-      rightLon = bounds.getRightLongitude().doubleValue();
-    }
-    double minLat;
-    if (bounds.checkNoBottomLatitudeBound()) {
-      minLat = -Math.PI * 0.5;
-    } else {
-      minLat = bounds.getMinLatitude().doubleValue();
-    }
-    double maxLat;
-    if (bounds.checkNoTopLatitudeBound()) {
-      maxLat = Math.PI * 0.5;
-    } else {
-      maxLat = bounds.getMaxLatitude().doubleValue();
-    }
-    return GeoBBoxFactory.makeGeoBBox(planetModel, maxLat, minLat, leftLon, rightLon);
-  }
-
-  public abstract class Geo3dRectIntersectionTestHelper extends RectIntersectionTestHelper<Geo3dShape> {
-
-    public Geo3dRectIntersectionTestHelper(SpatialContext ctx) {
-      super(ctx);
-    }
-
-    //20 times each -- should be plenty
-
-    protected int getContainsMinimum(int laps) {
-      return 20;
-    }
-
-    protected int getIntersectsMinimum(int laps) {
-      return 20;
-    }
-
-    // producing "within" cases in Geo3D based on our random shapes doesn't happen often. It'd be nice to increase this.
-    protected int getWithinMinimum(int laps) {
-      return 2;
-    }
-
-    protected int getDisjointMinimum(int laps) {
-      return 20;
-    }
-
-    protected int getBoundingMinimum(int laps) {
-      return 20;
-    }
-  }
-
-  @Test
-  public void testGeoCircleRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final int circleRadius = 180 - random().nextInt(180);//no 0-radius
-        final Point point = nearP;
-        final GeoCircle shape = GeoCircleFactory.makeGeoCircle(planetModel, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS,
-            circleRadius * DEGREES_TO_RADIANS);
-        return new Geo3dShape(shape, ctx);
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        GeoPoint geoPoint = ((GeoCircle)shape.shape).getCenter();
-        return geoPointToSpatial4jPoint(geoPoint);
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  @Test
-  public void testGeoBBoxRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected boolean isRandomShapeRectangular() {
-        return true;
-      }
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        // (ignoring nearP)
-        Point ulhcPoint = randomPoint();
-        Point lrhcPoint = randomPoint();
-        if (ulhcPoint.getY() < lrhcPoint.getY()) {
-          //swap
-          Point temp = ulhcPoint;
-          ulhcPoint = lrhcPoint;
-          lrhcPoint = temp;
-        }
-        final GeoBBox shape = GeoBBoxFactory.makeGeoBBox(planetModel, ulhcPoint.getY() * DEGREES_TO_RADIANS,
-            lrhcPoint.getY() * DEGREES_TO_RADIANS,
-            ulhcPoint.getX() * DEGREES_TO_RADIANS,
-            lrhcPoint.getX() * DEGREES_TO_RADIANS);
-        return new Geo3dShape(shape, ctx);
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        return shape.getBoundingBox().getCenter();
-      }
-    }.testRelateWithRectangle();
-  }
-
-  @Test
-  public void testGeoPolygonRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final Point centerPoint = randomPoint();
-        final int maxDistance = random().nextInt(160) + 20;
-        final Circle pointZone = ctx.makeCircle(centerPoint, maxDistance);
-        final int vertexCount = random().nextInt(3) + 3;
-        while (true) {
-          final List<GeoPoint> geoPoints = new ArrayList<>();
-          while (geoPoints.size() < vertexCount) {
-            final Point point = randomPointIn(pointZone);
-            final GeoPoint gPt = new GeoPoint(planetModel, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS);
-            geoPoints.add(gPt);
-          }
-          try {
-            final GeoPolygon shape = GeoPolygonFactory.makeGeoPolygon(planetModel, geoPoints);
-            if (shape == null) {
-              continue;
-            }
-            return new Geo3dShape(shape, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        throw new IllegalStateException("unexpected; need to finish test code");
-      }
-
-      @Override
-      protected int getWithinMinimum(int laps) {
-        // Long/thin so lets just find 1.
-        return 1;
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  @Test
-  public void testGeoPathRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final Point centerPoint = randomPoint();
-        final int maxDistance = random().nextInt(160) + 20;
-        final Circle pointZone = ctx.makeCircle(centerPoint, maxDistance);
-        final int pointCount = random().nextInt(5) + 1;
-        final double width = (random().nextInt(89)+1) * DEGREES_TO_RADIANS;
-        final GeoPoint[] points = new GeoPoint[pointCount];
-        while (true) {
-          for (int i = 0; i < pointCount; i++) {
-            final Point nextPoint = randomPointIn(pointZone);
-            points[i] = new GeoPoint(planetModel, nextPoint.getY() * DEGREES_TO_RADIANS, nextPoint.getX() * DEGREES_TO_RADIANS);
-          }
-          
-          try {
-            final GeoPath path = GeoPathFactory.makeGeoPath(planetModel, width, points);
-            return new Geo3dShape(path, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        throw new IllegalStateException("unexpected; need to finish test code");
-      }
-
-      @Override
-      protected int getWithinMinimum(int laps) {
-        // Long/thin so lets just find 1.
-        return 1;
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  private Point geoPointToSpatial4jPoint(GeoPoint geoPoint) {
-    return ctx.makePoint(geoPoint.getLongitude() * DistanceUtils.RADIANS_TO_DEGREES,
-        geoPoint.getLatitude() * DistanceUtils.RADIANS_TO_DEGREES);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
index 20db21c..bf152b7 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
@@ -34,13 +34,13 @@ import org.locationtech.spatial4j.shape.Point;
 import org.locationtech.spatial4j.shape.Rectangle;
 import org.locationtech.spatial4j.shape.SpatialRelation;
 
-public class Geo3dShapeSphereModelRectRelationTest extends Geo3dShapeRectRelationTestCase {
+public class Geo3dShapeSphereModelRectRelationTest extends ShapeRectRelationTestCase {
+
+  PlanetModel planetModel = PlanetModel.SPHERE;
 
   public Geo3dShapeSphereModelRectRelationTest() {
-    super(PlanetModel.SPHERE);
     Geo3dSpatialContextFactory factory = new Geo3dSpatialContextFactory();
-    factory.planetModel = PlanetModel.SPHERE;
-    //factory.distCalc = new GeodesicSphereDistCalc.Haversine();
+    factory.planetModel = planetModel;
     this.ctx = factory.newSpatialContext();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
index 22d7bd4..5a7b4b5 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
@@ -30,14 +30,16 @@ import org.locationtech.spatial4j.shape.Circle;
 import org.locationtech.spatial4j.shape.Point;
 import org.locationtech.spatial4j.shape.SpatialRelation;
 
-public class Geo3dShapeWGS84ModelRectRelationTest extends Geo3dShapeRectRelationTestCase {
+public class Geo3dShapeWGS84ModelRectRelationTest extends ShapeRectRelationTestCase {
+
+  PlanetModel planetModel = PlanetModel.WGS84;
 
   public Geo3dShapeWGS84ModelRectRelationTest() {
-    super(PlanetModel.WGS84);
     Geo3dSpatialContextFactory factory = new Geo3dSpatialContextFactory();
-    factory.planetModel = PlanetModel.WGS84;
-    //factory.distCalc = new GeodesicSphereDistCalc.Haversine();
+    factory.planetModel = planetModel;
     this.ctx = factory.newSpatialContext();
+    this.maxRadius = 178;
+    ((Geo3dShapeFactory)ctx.getShapeFactory()).setCircleAccuracy(1e-6);
   }
 
   @Test


[2/5] lucene-solr:jira/solr-11702: merge master

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/UsingSolrJRefGuideExamplesTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/UsingSolrJRefGuideExamplesTest.java b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/UsingSolrJRefGuideExamplesTest.java
index a988540..b45e702 100644
--- a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/UsingSolrJRefGuideExamplesTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/UsingSolrJRefGuideExamplesTest.java
@@ -18,14 +18,17 @@
 package org.apache.solr.client.ref_guide_examples;
 
 import java.io.File;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.UUID;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrQuery.ORDER;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.beans.Field;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
@@ -54,6 +57,8 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
 
   private static final int NUM_INDEXED_DOCUMENTS = 3;
   private static final int NUM_LIVE_NODES = 1;
+  
+  private Queue<String> expectedLines = new ArrayDeque();
 
   @BeforeClass
   public static void setUpCluster() throws Exception {
@@ -69,6 +74,7 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
   @Override
   public void setUp() throws Exception {
     super.setUp();
+    expectedLines.clear();
     final SolrClient client = getSolrClient();
 
     final List<TechProduct> products = new ArrayList<TechProduct>();
@@ -84,6 +90,7 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
   @Override
   public void tearDown() throws Exception {
     super.tearDown();
+    ensureNoLeftoverOutputExpectations();
 
     final SolrClient client = getSolrClient();
     client.deleteByQuery("techproducts", "*:*");
@@ -92,45 +99,60 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
 
   @Test
   public void queryWithRawSolrParamsExample() throws Exception {
+    expectLine("Found 3 documents");
+    expectLine("id: 1; name: Fitbit Alta");
+    expectLine("id: 2; name: Sony Walkman");
+    expectLine("id: 3; name: Garmin GPS");
+    
     // tag::solrj-query-with-raw-solrparams[]
     final SolrClient client = getSolrClient();
 
     final Map<String, String> queryParamMap = new HashMap<String, String>();
     queryParamMap.put("q", "*:*");
     queryParamMap.put("fl", "id, name");
+    queryParamMap.put("sort", "id asc");
     MapSolrParams queryParams = new MapSolrParams(queryParamMap);
 
     final QueryResponse response = client.query("techproducts", queryParams);
     final SolrDocumentList documents = response.getResults();
 
-    assertEquals(NUM_INDEXED_DOCUMENTS, documents.getNumFound());
+    print("Found " + documents.getNumFound() + " documents");
     for(SolrDocument document : documents) {
-      assertTrue(document.getFieldNames().contains("id"));
-      assertTrue(document.getFieldNames().contains("name"));
+      final String id = (String) document.getFirstValue("id");
+      final String name = (String) document.getFirstValue("name");
+      
+      print("id: " + id + "; name: " + name);
     }
     // end::solrj-query-with-raw-solrparams[]
   }
 
   @Test
   public void queryWithSolrQueryExample() throws Exception {
-    final int numResultsToReturn = 1;
+    final int numResultsToReturn = 3;
+    expectLine("Found 3 documents");
+    expectLine("id: 1; name: Fitbit Alta");
+    expectLine("id: 2; name: Sony Walkman");
+    expectLine("id: 3; name: Garmin GPS");
     final SolrClient client = getSolrClient();
 
     // tag::solrj-query-with-solrquery[]
     final SolrQuery query = new SolrQuery("*:*");
     query.addField("id");
     query.addField("name");
+    query.setSort("id", ORDER.asc);
     query.setRows(numResultsToReturn);
     // end::solrj-query-with-solrquery[]
 
     final QueryResponse response = client.query("techproducts", query);
     final SolrDocumentList documents = response.getResults();
 
-    assertEquals(NUM_INDEXED_DOCUMENTS, documents.getNumFound());
+    print("Found " + documents.getNumFound() + " documents");
     assertEquals(numResultsToReturn, documents.size());
     for(SolrDocument document : documents) {
-      assertTrue(document.getFieldNames().contains("id"));
-      assertTrue(document.getFieldNames().contains("name"));
+      final String id = (String) document.getFirstValue("id");
+      final String name = (String) document.getFirstValue("name");
+      
+      print("id: "+ id + "; name: " + name);
     }
   }
 
@@ -167,26 +189,32 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
 
   @Test
   public void queryBeanValueTypeExample() throws Exception {
+    expectLine("Found 3 documents");
+    expectLine("id: 1; name: Fitbit Alta");
+    expectLine("id: 2; name: Sony Walkman");
+    expectLine("id: 3; name: Garmin GPS");
+    
     // tag::solrj-query-bean-value-type[]
     final SolrClient client = getSolrClient();
 
     final SolrQuery query = new SolrQuery("*:*");
     query.addField("id");
     query.addField("name");
+    query.setSort("id", ORDER.asc);
 
     final QueryResponse response = client.query("techproducts", query);
     final List<TechProduct> products = response.getBeans(TechProduct.class);
     // end::solrj-query-bean-value-type[]
 
-    assertEquals(NUM_INDEXED_DOCUMENTS, products.size());
+    print("Found " + products.size() + " documents");
     for (TechProduct product : products) {
-      assertFalse(product.id.isEmpty());
-      assertFalse(product.name.isEmpty());
+      print("id: " + product.id + "; name: " + product.name);
     }
   }
 
   @Test
   public void otherSolrApisExample() throws Exception {
+    expectLine("Found "+NUM_LIVE_NODES+" live nodes");
     // tag::solrj-other-apis[]
     final SolrClient client = getSolrClient();
 
@@ -196,7 +224,7 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
     final NamedList<Object> cluster = (NamedList<Object>) response.get("cluster");
     final List<String> liveNodes = (List<String>) cluster.get("live_nodes");
 
-    assertEquals(NUM_LIVE_NODES, liveNodes.size());
+    print("Found " + liveNodes.size() + " live nodes");
     // end::solrj-other-apis[]
   }
 
@@ -231,5 +259,28 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
     public TechProduct() {}
   }
   // end::solrj-techproduct-value-type[]
-
+  
+  private void expectLine(String expectedLine) {
+    expectedLines.add(expectedLine);
+  }
+  
+  private void print(String actualOutput) {
+    final String nextExpectedLine = expectedLines.poll();
+    assertNotNull("No more output expected, but was asked to print: " + actualOutput, nextExpectedLine);
+    
+    final String unexpectedOutputMessage = "Expected line containing " + nextExpectedLine + ", but printed line was: "
+        + actualOutput;
+    assertTrue(unexpectedOutputMessage, actualOutput.contains(nextExpectedLine));
+  }
+  
+  private void ensureNoLeftoverOutputExpectations() {
+    if (expectedLines.isEmpty()) return;
+    
+    final StringBuilder builder = new StringBuilder();
+    builder.append("Leftover output was expected but not printed:");
+    for (String expectedLine : expectedLines) {
+      builder.append("\n\t" + expectedLine);
+    }
+    fail(builder.toString());
+  }
 }


[3/5] lucene-solr:jira/solr-11702: merge master

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/ShapeRectRelationTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/ShapeRectRelationTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/ShapeRectRelationTestCase.java
new file mode 100644
index 0000000..7ec2a2b
--- /dev/null
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/ShapeRectRelationTestCase.java
@@ -0,0 +1,201 @@
+/*
+ * 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.spatial.spatial4j;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.locationtech.spatial4j.TestLog;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.RectIntersectionTestHelper;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.ShapeFactory;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.DEGREES_TO_RADIANS;
+
+public abstract class ShapeRectRelationTestCase extends RandomizedShapeTestCase {
+  protected final static double RADIANS_PER_DEGREE = Math.PI/180.0;
+
+  @Rule
+  public final TestLog testLog = TestLog.instance;
+
+  protected int maxRadius = 180;
+
+  public ShapeRectRelationTestCase() {
+    super(SpatialContext.GEO);
+  }
+
+  public abstract class AbstractRectIntersectionTestHelper extends RectIntersectionTestHelper<Shape> {
+
+    public AbstractRectIntersectionTestHelper(SpatialContext ctx) {
+      super(ctx);
+    }
+
+    //20 times each -- should be plenty
+
+    protected int getContainsMinimum(int laps) {
+      return 20;
+    }
+
+    protected int getIntersectsMinimum(int laps) {
+      return 20;
+    }
+
+    // producing "within" cases in Geo3D based on our random shapes doesn't happen often. It'd be nice to increase this.
+    protected int getWithinMinimum(int laps) {
+      return 2;
+    }
+
+    protected int getDisjointMinimum(int laps) {
+      return 20;
+    }
+
+    protected int getBoundingMinimum(int laps) {
+      return 20;
+    }
+  }
+
+  @Test
+  public void testGeoCircleRect() {
+    new AbstractRectIntersectionTestHelper(ctx) {
+
+      @Override
+      protected Shape generateRandomShape(Point nearP) {
+        final int circleRadius = maxRadius - random().nextInt(maxRadius);//no 0-radius
+        return ctx.getShapeFactory().circle(nearP, circleRadius);
+      }
+
+      @Override
+      protected Point randomPointInEmptyShape(Shape shape) {
+        return shape.getCenter();
+      }
+
+    }.testRelateWithRectangle();
+  }
+
+  @Test
+  public void testGeoBBoxRect() {
+    new AbstractRectIntersectionTestHelper(ctx) {
+
+      @Override
+      protected boolean isRandomShapeRectangular() {
+        return true;
+      }
+
+      @Override
+      protected Shape generateRandomShape(Point nearP) {
+        Point upperRight = randomPoint();
+        Point lowerLeft = randomPoint();
+        if (upperRight.getY() < lowerLeft.getY()) {
+          //swap
+          Point temp = upperRight;
+          upperRight = lowerLeft;
+          lowerLeft = temp;
+        }
+        return ctx.getShapeFactory().rect(lowerLeft, upperRight);
+      }
+
+      @Override
+      protected Point randomPointInEmptyShape(Shape shape) {
+        return shape.getCenter();
+      }
+    }.testRelateWithRectangle();
+  }
+
+  @Test
+  public void testGeoPolygonRect() {
+    new AbstractRectIntersectionTestHelper(ctx) {
+
+      @Override
+      protected Shape generateRandomShape(Point nearP) {
+        final Point centerPoint = randomPoint();
+        final int maxDistance = random().nextInt(maxRadius -20) + 20;
+        final Circle pointZone = ctx.getShapeFactory().circle(centerPoint, maxDistance);
+        final int vertexCount = random().nextInt(3) + 3;
+        while (true) {
+          ShapeFactory.PolygonBuilder builder = ctx.getShapeFactory().polygon();
+          for (int i = 0; i < vertexCount; i++) {
+            final Point point = randomPointIn(pointZone);
+            builder.pointXY(point.getX(), point.getY());
+          }
+          try {
+            return builder.build();
+          } catch (IllegalArgumentException e) {
+            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
+            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
+            continue;
+          }
+        }
+      }
+
+      @Override
+      protected Point randomPointInEmptyShape(Shape shape) {
+        throw new IllegalStateException("unexpected; need to finish test code");
+      }
+
+      @Override
+      protected int getWithinMinimum(int laps) {
+        // Long/thin so lets just find 1.
+        return 1;
+      }
+
+    }.testRelateWithRectangle();
+  }
+
+  @Test
+  public void testGeoPathRect() {
+    new AbstractRectIntersectionTestHelper(ctx) {
+
+      @Override
+      protected Shape generateRandomShape(Point nearP) {
+        final Point centerPoint = randomPoint();
+        final int maxDistance = random().nextInt(maxRadius -20) + 20;
+        final Circle pointZone = ctx.getShapeFactory().circle(centerPoint, maxDistance);
+        final int pointCount = random().nextInt(5) + 1;
+        final double width = (random().nextInt(89)+1) * DEGREES_TO_RADIANS;
+        final ShapeFactory.LineStringBuilder builder = ctx.getShapeFactory().lineString();
+        while (true) {
+          for (int i = 0; i < pointCount; i++) {
+            final Point nextPoint = randomPointIn(pointZone);
+            builder.pointXY(nextPoint.getX(), nextPoint.getY());
+          }
+          builder.buffer(width);
+          try {
+            return builder.build();
+          } catch (IllegalArgumentException e) {
+            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
+            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
+            continue;
+          }
+        }
+      }
+
+      @Override
+      protected Point randomPointInEmptyShape(Shape shape) {
+        throw new IllegalStateException("unexpected; need to finish test code");
+      }
+
+      @Override
+      protected int getWithinMinimum(int laps) {
+        // Long/thin so lets just find 1.
+        return 1;
+      }
+
+    }.testRelateWithRectangle();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoCircleFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoCircleFactory.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoCircleFactory.java
index 8feecce..f32f366 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoCircleFactory.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoCircleFactory.java
@@ -26,27 +26,40 @@ public class GeoCircleFactory {
   }
 
   /**
-   * Create a GeoCircle of the right kind given the specified bounds.
+   * Create a GeoCircle from a center and a cutoff angle. The resulting shape is a circle in spherical
+   * planets, otherwise is an ellipse. It is the most efficient shape to represent a circle on a sphere.
+   *
    * @param planetModel is the planet model.
    * @param latitude is the center latitude.
    * @param longitude is the center longitude.
-   * @param radius is the radius angle.
+   * @param cutoffAngle is the cutoff angle.
    * @return a GeoCircle corresponding to what was specified.
    */
-  public static GeoCircle makeGeoCircle(final PlanetModel planetModel, final double latitude, final double longitude, final double radius) {
-    if (radius < Vector.MINIMUM_ANGULAR_RESOLUTION) {
+  public static GeoCircle makeGeoCircle(final PlanetModel planetModel, final double latitude, final double longitude, final double cutoffAngle) {
+    if (cutoffAngle < Vector.MINIMUM_ANGULAR_RESOLUTION) {
       return new GeoDegeneratePoint(planetModel, latitude, longitude);
     }
-    return new GeoStandardCircle(planetModel, latitude, longitude, radius);
+    return new GeoStandardCircle(planetModel, latitude, longitude, cutoffAngle);
   }
 
-  /** Create an exact GeoCircle given specified bounds and desired accuracy.
+  /**
+   * Create an GeoCircle from a center, a radius and a desired accuracy. It is the most accurate shape to represent
+   * a circle in non-spherical planets.
+   * <p>
+   * The accuracy of the circle is defined as the maximum linear distance between any point on the
+   * surface circle and planes that describe the circle. Therefore, with planet model WSG84, since the
+   * radius of earth is 6,371,000 meters, an accuracy of 1e-6 corresponds to 6.3 meters.
+   * For an accuracy of 1.0 meters, the accuracy value would be 1.6e-7. The maximum accuracy possible is 1e-12.
+   * <p>
+   * Note that this method may thrown an IllegalArgumentException if the circle being specified cannot be
+   * represented by plane approximation given the planet model provided.
+   *
    * @param planetModel is the planet model.
    * @param latitude is the center latitude.
    * @param longitude is the center longitude.
    * @param radius is the radius surface distance.
-   * @param accuracy is the maximum linear distance between the circle approximation and the real circle, as computed using
-   *  the Vincenty formula.
+   * @param accuracy is the maximum linear distance between the circle approximation and the real circle,
+   *                 as computed using the Vincenty formula.
    * @return a GeoCircle corresponding to what was specified.
    */
   public static GeoCircle makeExactGeoCircle(final PlanetModel planetModel, final double latitude, final double longitude, final double radius, final double accuracy) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoExactCircle.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoExactCircle.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoExactCircle.java
index 876b24f..845dd6a 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoExactCircle.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoExactCircle.java
@@ -24,44 +24,47 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 /**
- * Circular area with a center and radius.
+ * Circular area with a center and a radius that represents the surface distance to the center.
+ * The circle is divided in sectors where the circle edge is approximated using Vincenty formulae.
+ * The higher is the precision the more sectors are needed to describe the shape and therefore a penalty
+ * in performance.
  *
  * @lucene.experimental
  */
 class GeoExactCircle extends GeoBaseCircle {
   /** Center of circle */
   protected final GeoPoint center;
-  /** Cutoff angle of circle (not quite the same thing as radius) */
-  protected final double cutoffAngle;
+  /** Radius of circle */
+  protected final double radius;
   /** Actual accuracy */
   protected final double actualAccuracy;
-  /** A point that is on the world and on the circle plane */
+  /** A point that is on the edge of the circle */
   protected final GeoPoint[] edgePoints;
-  /** Slices of the circle. */
+  /** Slices of the circle */
   protected final List<CircleSlice> circleSlices;
 
   /** Constructor.
    *@param planetModel is the planet model.
    *@param lat is the center latitude.
    *@param lon is the center longitude.
-   *@param cutoffAngle is the surface radius for the circle.
-   *@param accuracy is the allowed error value (linear distance).
+   *@param radius is the surface radius for the circle.
+   *@param accuracy is the allowed error value (linear distance). Maximum accuracy is 1e-12.
    */
-  public GeoExactCircle(final PlanetModel planetModel, final double lat, final double lon, final double cutoffAngle, final double accuracy) {
+  public GeoExactCircle(final PlanetModel planetModel, final double lat, final double lon, final double radius, final double accuracy) {
     super(planetModel);
     if (lat < -Math.PI * 0.5 || lat > Math.PI * 0.5)
       throw new IllegalArgumentException("Latitude out of bounds");
     if (lon < -Math.PI || lon > Math.PI)
       throw new IllegalArgumentException("Longitude out of bounds");
-    if (cutoffAngle < 0.0)
-      throw new IllegalArgumentException("Cutoff angle out of bounds");
-    if (cutoffAngle < Vector.MINIMUM_RESOLUTION)
-      throw new IllegalArgumentException("Cutoff angle cannot be effectively zero");
-    if (planetModel.minimumPoleDistance - cutoffAngle  < Vector.MINIMUM_RESOLUTION)
-      throw new IllegalArgumentException("Cutoff angle out of bounds. It cannot be bigger than " +  planetModel.minimumPoleDistance + " for this planet model");
+    if (radius < 0.0)
+      throw new IllegalArgumentException("Radius out of bounds");
+    if (radius < Vector.MINIMUM_RESOLUTION)
+      throw new IllegalArgumentException("Radius cannot be effectively zero");
+    if (planetModel.minimumPoleDistance - radius < Vector.MINIMUM_RESOLUTION)
+      throw new IllegalArgumentException("Radius out of bounds. It cannot be bigger than " +  planetModel.minimumPoleDistance + " for this planet model");
 
     this.center = new GeoPoint(planetModel, lat, lon);
-    this.cutoffAngle = cutoffAngle;
+    this.radius = radius;
 
     if (accuracy < Vector.MINIMUM_RESOLUTION) {
       actualAccuracy = Vector.MINIMUM_RESOLUTION;
@@ -72,10 +75,10 @@ class GeoExactCircle extends GeoBaseCircle {
     // We construct approximation planes until we have a low enough error estimate
     final List<ApproximationSlice> slices = new ArrayList<>(100);
     // Construct four cardinal points, and then we'll build the first two planes
-    final GeoPoint northPoint = planetModel.surfacePointOnBearing(center, cutoffAngle, 0.0);
-    final GeoPoint southPoint = planetModel.surfacePointOnBearing(center, cutoffAngle, Math.PI);
-    final GeoPoint eastPoint = planetModel.surfacePointOnBearing(center, cutoffAngle, Math.PI * 0.5);
-    final GeoPoint westPoint = planetModel.surfacePointOnBearing(center, cutoffAngle, Math.PI * 1.5);
+    final GeoPoint northPoint = planetModel.surfacePointOnBearing(center, radius, 0.0);
+    final GeoPoint southPoint = planetModel.surfacePointOnBearing(center, radius, Math.PI);
+    final GeoPoint eastPoint = planetModel.surfacePointOnBearing(center, radius, Math.PI * 0.5);
+    final GeoPoint westPoint = planetModel.surfacePointOnBearing(center, radius, Math.PI * 1.5);
 
     final GeoPoint edgePoint;
     if (planetModel.c > planetModel.ab) {
@@ -101,9 +104,9 @@ class GeoExactCircle extends GeoBaseCircle {
       // To do this, we need to look at the part of the circle that will have the greatest error.
       // We will need to compute bearing points for these.
       final double interpPoint1Bearing = (thisSlice.point1Bearing + thisSlice.middlePointBearing) * 0.5;
-      final GeoPoint interpPoint1 = planetModel.surfacePointOnBearing(center, cutoffAngle, interpPoint1Bearing);
+      final GeoPoint interpPoint1 = planetModel.surfacePointOnBearing(center, radius, interpPoint1Bearing);
       final double interpPoint2Bearing = (thisSlice.point2Bearing + thisSlice.middlePointBearing) * 0.5;
-      final GeoPoint interpPoint2 = planetModel.surfacePointOnBearing(center, cutoffAngle, interpPoint2Bearing);
+      final GeoPoint interpPoint2 = planetModel.surfacePointOnBearing(center, radius, interpPoint2Bearing);
       
       // Is this point on the plane? (that is, is the approximation good enough?)
       if (!thisSlice.mustSplit && Math.abs(thisSlice.plane.evaluate(interpPoint1)) < actualAccuracy && Math.abs(thisSlice.plane.evaluate(interpPoint2)) < actualAccuracy) {
@@ -145,13 +148,13 @@ class GeoExactCircle extends GeoBaseCircle {
   public void write(final OutputStream outputStream) throws IOException {
     SerializableObject.writeDouble(outputStream, center.getLatitude());
     SerializableObject.writeDouble(outputStream, center.getLongitude());
-    SerializableObject.writeDouble(outputStream, cutoffAngle);
+    SerializableObject.writeDouble(outputStream, radius);
     SerializableObject.writeDouble(outputStream, actualAccuracy);
   }
 
   @Override
   public double getRadius() {
-    return cutoffAngle;
+    return radius;
   }
 
   @Override
@@ -235,14 +238,14 @@ class GeoExactCircle extends GeoBaseCircle {
     if (!(o instanceof GeoExactCircle))
       return false;
     GeoExactCircle other = (GeoExactCircle) o;
-    return super.equals(other) && other.center.equals(center) && other.cutoffAngle == cutoffAngle && other.actualAccuracy == actualAccuracy;
+    return super.equals(other) && other.center.equals(center) && other.radius == radius && other.actualAccuracy == actualAccuracy;
   }
 
   @Override
   public int hashCode() {
     int result = super.hashCode();
     result = 31 * result + center.hashCode();
-    long temp = Double.doubleToLongBits(cutoffAngle);
+    long temp = Double.doubleToLongBits(radius);
     result = 31 * result + (int) (temp ^ (temp >>> 32));
     temp = Double.doubleToLongBits(actualAccuracy);
     result = 31 * result + (int) (temp ^ (temp >>> 32));    
@@ -251,7 +254,7 @@ class GeoExactCircle extends GeoBaseCircle {
 
   @Override
   public String toString() {
-    return "GeoExactCircle: {planetmodel=" + planetModel+", center=" + center + ", radius=" + cutoffAngle + "(" + cutoffAngle * 180.0 / Math.PI + "), accuracy=" + actualAccuracy + "}";
+    return "GeoExactCircle: {planetmodel=" + planetModel+", center=" + center + ", radius=" + radius + "(" + radius * 180.0 / Math.PI + "), accuracy=" + actualAccuracy + "}";
   }
   
   /** A temporary description of a section of circle.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardCircle.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardCircle.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardCircle.java
index acba9f5..7ac45932 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardCircle.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardCircle.java
@@ -21,7 +21,9 @@ import java.io.OutputStream;
 import java.io.IOException;
 
 /**
- * Circular area with a center and radius.
+ * Circular area with a center and cutoff angle that represents the latitude and longitude distance
+ * from the center where the planet will be cut. The resulting area is a circle for spherical
+ * planets and an ellipse otherwise.
  *
  * @lucene.experimental
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/PlanetModel.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/PlanetModel.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/PlanetModel.java
index 2aabfc1..55b730d 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/PlanetModel.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/PlanetModel.java
@@ -122,6 +122,13 @@ public class PlanetModel implements SerializableObject {
     SerializableObject.writeDouble(outputStream, c);
   }
   
+  /** Does this planet model describe a sphere?
+   *@return true if so.
+   */
+  public boolean isSphere() {
+    return this.ab == this.c;
+  }
+  
   /** Find the minimum magnitude of all points on the ellipsoid.
    * @return the minimum magnitude for the planet.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1218171..b6e729f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -29,8 +29,12 @@ Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.10
 Jetty 9.3.20.v20170531
 
+Upgrade Notes
+----------------------
 
-(No Changes)
+* LUCENE-7996: The 'func' query parser now returns scores that are equal to 0
+  when a negative value is produced. This change is due to the fact that
+  Lucene now requires scores to be positive. (Adrien Grand)
 
 ==================  7.3.0 ==================
 
@@ -45,9 +49,15 @@ Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.10
 Jetty 9.3.20.v20170531
 
+Optimizations
+----------------------
 
-(No Changes)
-
+* SOLR-11711: Fixed distributed processing of facet.field/facet.pivot sub requests to prevent requesting
+  unneccessary and excessive '0' count terms from each shard (Houston Putman via hossman)
+  
+Other Changes
+----------------------
+* SOLR-11575: Improve ref-guide solrj snippets via mock 'print()' method (Jason Gerlowski via hossman)
 
 ==================  7.2.0 ==================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index 2fb483c..f9814c2 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -729,13 +729,13 @@ function stop_solr() {
 
   DIR="$1"
   SOLR_PORT="$2"
-  STOP_PORT="${STOP_PORT:-$(expr $SOLR_PORT - 1000)}"
+  THIS_STOP_PORT="${STOP_PORT:-$(expr $SOLR_PORT - 1000)}"
   STOP_KEY="$3"
   SOLR_PID="$4"
 
   if [ "$SOLR_PID" != "" ]; then
     echo -e "Sending stop command to Solr running on port $SOLR_PORT ... waiting up to $SOLR_STOP_WAIT seconds to allow Jetty process $SOLR_PID to stop gracefully."
-    "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS -jar "$DIR/start.jar" "STOP.PORT=$STOP_PORT" "STOP.KEY=$STOP_KEY" --stop || true
+    "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS -jar "$DIR/start.jar" "STOP.PORT=$THIS_STOP_PORT" "STOP.KEY=$STOP_KEY" --stop || true
       (loops=0
       while true
       do

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
index e1a6bc4..f0f5109 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
@@ -571,21 +571,8 @@ public class FacetComponent extends SearchComponent {
           // set the initial limit higher to increase accuracy
           dff.initialLimit = doOverRequestMath(dff.initialLimit, dff.overrequestRatio, 
                                                dff.overrequestCount);
-          
-          // If option FACET_DISTRIB_MCO is turned on then we will use 1 as the initial 
-          // minCount (unless the user explicitly set it to something less than 1). If 
-          // option FACET_DISTRIB_MCO is turned off then we will use 0 as the initial 
-          // minCount regardless of what the user might have provided (prior to the
-          // addition of the FACET_DISTRIB_MCO option the default logic was to use 0).
-          // As described in issues SOLR-8559 and SOLR-8988 the use of 1 provides a 
-          // significant performance boost.
-          dff.initialMincount = dff.mco ? Math.min(dff.minCount, 1) : 0;
-                                   
-        } else {
-          // if limit==-1, then no need to artificially lower mincount to 0 if
-          // it's 1
-          dff.initialMincount = Math.min(dff.minCount, 1);
         }
+        dff.initialMincount = Math.min(dff.minCount, 1);
       } else {
         // we're sorting by index order.
         // if minCount==0, we should always be able to get accurate results w/o
@@ -682,10 +669,8 @@ public class FacetComponent extends SearchComponent {
     } else if ( FacetParams.FACET_SORT_COUNT.equals(sort) ) {
       if ( 0 < requestedLimit ) {
         shardLimit = doOverRequestMath(shardLimit, overRequestRatio, overRequestCount);
-        shardMinCount = 0; 
-      } else {
-        shardMinCount = Math.min(requestedMinCount, 1);
       }
+      shardMinCount = Math.min(requestedMinCount, 1);
     } 
     sreq.params.set(paramStart + FacetParams.FACET_LIMIT, shardLimit);
     sreq.params.set(paramStart + FacetParams.FACET_PIVOT_MINCOUNT, shardMinCount);
@@ -1437,7 +1422,6 @@ public class FacetComponent extends SearchComponent {
     
     public int initialLimit; // how many terms requested in first phase
     public int initialMincount; // mincount param sent to each shard
-    public boolean mco;
     public double overrequestRatio;
     public int overrequestCount;
     public boolean needRefinements;
@@ -1456,9 +1440,6 @@ public class FacetComponent extends SearchComponent {
         = params.getFieldDouble(field, FacetParams.FACET_OVERREQUEST_RATIO, 1.5);
       this.overrequestCount 
         = params.getFieldInt(field, FacetParams.FACET_OVERREQUEST_COUNT, 10);
-      
-      this.mco 
-      = params.getFieldBool(field, FacetParams.FACET_DISTRIB_MCO, false);
     }
     
     void add(int shardNum, NamedList shardCounts, int numRequested) {
@@ -1496,10 +1477,10 @@ public class FacetComponent extends SearchComponent {
         }
       }
       
-      // the largest possible missing term is initialMincount if we received
+      // the largest possible missing term is (initialMincount - 1) if we received
       // less than the number requested.
       if (numRequested < 0 || numRequested != 0 && numReceived < numRequested) {
-        last = initialMincount;
+        last = Math.max(0, initialMincount - 1);
       }
       
       missingMaxPossible += last;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
index fb9c86a..fa977da 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudPivotFacet.java
@@ -53,7 +53,6 @@ import static org.apache.solr.common.params.FacetParams.FACET_OVERREQUEST_RATIO;
 import static org.apache.solr.common.params.FacetParams.FACET_PIVOT;
 import static org.apache.solr.common.params.FacetParams.FACET_PIVOT_MINCOUNT;
 import static org.apache.solr.common.params.FacetParams.FACET_SORT;
-import static org.apache.solr.common.params.FacetParams.FACET_DISTRIB_MCO;
 
 /**
  * <p>
@@ -85,8 +84,6 @@ public class TestCloudPivotFacet extends AbstractFullDistribZkTestBase {
   // param used by test purely for tracing & validation
   private static String TRACE_MIN = "_test_min";
   // param used by test purely for tracing & validation
-  private static String TRACE_DISTRIB_MIN = "_test_distrib_min";
-  // param used by test purely for tracing & validation
   private static String TRACE_MISS = "_test_miss";
   // param used by test purely for tracing & validation
   private static String TRACE_SORT = "_test_sort";
@@ -200,12 +197,6 @@ public class TestCloudPivotFacet extends AbstractFullDistribZkTestBase {
       }
       
       if (random().nextBoolean()) {
-        pivotP.add(FACET_DISTRIB_MCO, "true");
-        // trace param for validation
-        baseP.add(TRACE_DISTRIB_MIN, "true");
-      }
-
-      if (random().nextBoolean()) {
         String missing = ""+random().nextBoolean();
         pivotP.add(FACET_MISSING, missing);
         // trace param for validation

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
index f18f551..996ae2e 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotLongTailTest.java
@@ -19,93 +19,57 @@ package org.apache.solr.handler.component;
 import java.util.List;
 
 import org.apache.solr.BaseDistributedSearchTestCase;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.PivotField;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.search.facet.DistributedFacetSimpleRefinementLongTailTest;
+
 import org.junit.Test;
 
 /**
+ * <p>
  * test demonstrating how overrequesting helps finds top-terms in the "long tail" 
  * of shards that don't have even distributions of terms (something that can be common
  * in cases of custom sharding -- even if you don't know that there is a corrolation 
  * between the property you are sharding on and the property you are faceting on).
- *
- * NOTE: This test ignores the control collection (in single node mode, there is no 
+ * <p>
+ * <b>NOTE:</b> This test ignores the control collection (in single node mode, there is no 
  * need for the overrequesting, all the data is local -- so comparisons with it wouldn't 
  * be valid in the cases we are testing here)
+ * </p>
+ * <p>
+ * <b>NOTE:</b> uses the same indexed documents as {@link DistributedFacetSimpleRefinementLongTailTest} -- 
+ * however the behavior of <code>refine:simple</code> is "simpler" then the refinement logic used by 
+ * <code>facet.pivot</code> so the assertions in this test vary from that test.
+ * </p>
  */
 public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTestCase {
   
-  private int docNumber = 0;
+  private String STAT_FIELD = null; // will be randomized single value vs multivalued
 
   public DistributedFacetPivotLongTailTest() {
     // we need DVs on point fields to compute stats & facets
     if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+
+    STAT_FIELD = random().nextBoolean() ? "stat_i1" : "stat_i";
   }
   
-  public int getDocNum() {
-    docNumber++;
-    return docNumber;
-  }
-
   @Test
   @ShardsFixed(num = 3)
   public void test() throws Exception {
-
-    final SolrClient shard0 = clients.get(0);
-    final SolrClient shard1 = clients.get(1);
-    final SolrClient shard2 = clients.get(2);
-    
-    // the 5 top foo_s terms have 100 docs each on every shard
-    for (int i = 0; i < 100; i++) {
-      for (int j = 0; j < 5; j++) {
-        shard0.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", j * 13 - i));
-        shard1.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", j * 3 + i));
-        shard2.add(sdoc("id", getDocNum(), "foo_s", "aaa"+j, "stat_i", i * 7 + j));
-      }
-    }
-
-    // 20 foo_s terms that come in "second" with 50 docs each 
-    // on both shard0 & shard1 ("bbb_")
-    for (int i = 0; i < 50; i++) {
-      for (int j = 0; j < 20; j++) {
-        shard0.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j, "stat_i", 0));
-        shard1.add(sdoc("id", getDocNum(), "foo_s", "bbb"+j, "stat_i", 1));
-      }
-      // distracting term appears on only on shard2 50 times
-      shard2.add(sdoc("id", getDocNum(), "foo_s", "junkA"));
-    }
-    // put "bbb0" on shard2 exactly once to sanity check refinement
-    shard2.add(sdoc("id", getDocNum(), "foo_s", "bbb0", "stat_i", -2));
-
-    // long 'tail' foo_s term appears in 45 docs on every shard
-    // foo_s:tail is the only term with bar_s sub-pivot terms
-    for (int i = 0; i < 45; i++) {
-
-      // for sub-pivot, shard0 & shard1 have 6 docs each for "tailB"
-      // but the top 5 terms are ccc(0-4) -- 7 on each shard
-      // (4 docs each have junk terms)
-      String sub_term = (i < 35) ? "ccc"+(i % 5) : ((i < 41) ? "tailB" : "junkA");
-      shard0.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term, "stat_i", i));
-      shard1.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term, "stat_i", i));
-
-      // shard2's top 5 sub-pivot terms are junk only it has with 8 docs each
-      // and 5 docs that use "tailB"
-      // NOTE: none of these get stat_i ! !
-      sub_term = (i < 40) ? "junkB"+(i % 5) : "tailB";
-      shard2.add(sdoc("id", getDocNum(), "foo_s", "tail", "bar_s", sub_term));
-    }
-
-    // really long tail uncommon foo_s terms on shard2
-    for (int i = 0; i < 30; i++) {
-      shard2.add(sdoc("id", getDocNum(), "foo_s", "zzz"+i));
-    }
-
+    DistributedFacetSimpleRefinementLongTailTest.buildIndexes(clients, STAT_FIELD);
     commit();
 
+    sanityCheckIndividualShards();
+    checkRefinementAndOverrequesting();
+    doTestDeepPivotStats();
+  }
+  
+  private void sanityCheckIndividualShards() throws Exception {
+    assertEquals("This test assumes exactly 3 shards/clients", 3, clients.size());
+    
     SolrParams req = params( "q", "*:*", 
                              "distrib", "false",
                              "facet", "true", 
@@ -116,10 +80,11 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
 
     PivotField pivot = null;
     List<PivotField> pivots = null;
-    List<PivotField>[] shardPivots = new List[3];
-    shardPivots[0] = shard0.query( req ).getFacetPivot().get("foo_s,bar_s");
-    shardPivots[1] = shard1.query( req ).getFacetPivot().get("foo_s,bar_s");
-    shardPivots[2] = shard2.query( req ).getFacetPivot().get("foo_s,bar_s");
+    
+    List<PivotField>[] shardPivots = new List[clients.size()];
+    for (int i = 0; i < clients.size(); i++) {
+      shardPivots[i] = clients.get(i).query( req ).getFacetPivot().get("foo_s,bar_s");
+    }
 
     // top 5 same on all shards
     for (int i = 0; i < 3; i++) {
@@ -143,11 +108,9 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
     assertEquals(50, shardPivots[2].get(5).getCount());
     assertEquals("tail", shardPivots[2].get(6).getValue());
     assertEquals(45, shardPivots[2].get(6).getCount());
-    assertEquals("bbb0", shardPivots[2].get(7).getValue());
-    assertEquals(1, shardPivots[2].get(7).getCount());
-    for (int j = 8; j < 10; j++) {
+    for (int j = 7; j < 10; j++) {
       pivot = shardPivots[2].get(j);
-      assertTrue(pivot.toString(), pivot.getValue().toString().startsWith("zzz"));
+      assertTrue(pivot.toString(), pivot.getValue().toString().startsWith("ZZZ"));
       assertEquals(pivot.toString(), 1, pivot.getCount());
     }
     // check sub-shardPivots on "tail" from shard2
@@ -161,9 +124,12 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
     pivot = pivots.get(5);
     assertEquals("tailB", pivot.getValue());
     assertEquals(5, pivot.getCount());
+  }
 
+  private void checkRefinementAndOverrequesting() throws Exception {
     // if we disable overrequesting, we don't find the long tail
-
+    List<PivotField> pivots = null;
+    PivotField pivot = null;
     pivots = queryServer( params( "q", "*:*",
                                   "shards", getShardsString(),
                                   FacetParams.FACET_OVERREQUEST_COUNT, "0",
@@ -172,7 +138,7 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
                                   "facet.limit", "6",
                                   "facet.pivot", "{!stats=sxy}foo_s,bar_s",
                                   "stats", "true",
-                                  "stats.field", "{!tag=sxy}stat_i")
+                                  "stats.field", "{!tag=sxy}" + STAT_FIELD)
                           ).getFacetPivot().get("foo_s,bar_s");
     assertEquals(6, pivots.size());
     for (int i = 0; i < 5; i++) {
@@ -185,8 +151,8 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
       assertTrue(pivot.toString(), pivot.getValue().equals("bbb0"));
       assertEquals(pivot.toString(), 101, pivot.getCount());
       // basic check of refined stats
-      FieldStatsInfo bbb0Stats = pivot.getFieldStatsInfo().get("stat_i");
-      assertEquals("stat_i", bbb0Stats.getName());
+      FieldStatsInfo bbb0Stats = pivot.getFieldStatsInfo().get(STAT_FIELD);
+      assertEquals(STAT_FIELD, bbb0Stats.getName());
       assertEquals(-2.0, bbb0Stats.getMin());
       assertEquals(1.0, bbb0Stats.getMax());
       assertEquals(101, (long) bbb0Stats.getCount());
@@ -295,11 +261,10 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
       assertTrue(pivot.toString(), pivot.getValue().toString().startsWith("ccc"));
       assertEquals(pivot.toString(), 14, pivot.getCount());
     }
-    
-    doTestDeepPivotStats();
+
   }
 
-  public void doTestDeepPivotStats() throws Exception {
+  private void doTestDeepPivotStats() throws Exception {
     // Deep checking of some Facet stats - no refinement involved here
 
     List<PivotField> pivots = 
@@ -309,7 +274,7 @@ public class DistributedFacetPivotLongTailTest extends BaseDistributedSearchTest
             "rows" , "0",
             "facet.pivot","{!stats=s1}foo_s,bar_s",
             "stats", "true",
-            "stats.field", "{!key=avg_price tag=s1}stat_i").getFacetPivot().get("foo_s,bar_s");
+            "stats.field", "{!key=avg_price tag=s1}" + STAT_FIELD).getFacetPivot().get("foo_s,bar_s");
     PivotField aaa0PivotField = pivots.get(0);
     assertEquals("aaa0", aaa0PivotField.getValue());
     assertEquals(300, aaa0PivotField.getCount());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/core/src/test/org/apache/solr/search/facet/DistributedFacetSimpleRefinementLongTailTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/DistributedFacetSimpleRefinementLongTailTest.java b/solr/core/src/test/org/apache/solr/search/facet/DistributedFacetSimpleRefinementLongTailTest.java
new file mode 100644
index 0000000..0612755
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/facet/DistributedFacetSimpleRefinementLongTailTest.java
@@ -0,0 +1,428 @@
+/*
+ * 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.search.facet;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.junit.Test;
+
+/**
+ * A test the demonstrates some of the expected behavior fo "long tail" terms when using <code>refine:simple</code>
+ * <p>
+ * <b>NOTE:</b> This test ignores the control collection (in single node mode, there is no 
+ * need for the overrequesting, all the data is local -- so comparisons with it wouldn't 
+ * be valid in the cases we are testing here)
+ * </p>
+ * <p>
+ * <b>NOTE:</b> This test is heavily inspired by (and uses the same indexed documents) as 
+ * {@link org.apache.solr.handler.component.DistributedFacetPivotLongTailTest} -- however the behavior of 
+ * <code>refine:simple</code> is "simpler" then the refinement logic used by 
+ * <code>facet.pivot</code> so the assertions in this test vary from that test.
+ * </p>
+ */
+public class DistributedFacetSimpleRefinementLongTailTest extends BaseDistributedSearchTestCase {
+
+  // TODO: SOLR-11695: need "num_values" and "missing"...
+  // TODO: add hll & variance - update all assertions to test their values (right after any mention of 'stddev')
+  private static List<String> ALL_STATS = Arrays.asList("min", "max", "sum", "stddev", "avg", "sumsq", "unique");
+                                                        
+  private String STAT_FIELD = "stat_i1";
+  private String ALL_STATS_JSON = "";
+
+  public DistributedFacetSimpleRefinementLongTailTest() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+
+    // TODO: randomizing STAT_FIELD to be multiValued=true blocked by SOLR-11706
+    // STAT_FIELD = random().nextBoolean() ? "stat_i1" : "stat_i";
+
+    for (String stat : ALL_STATS) {
+      ALL_STATS_JSON += stat + ":'" + stat + "(" + STAT_FIELD + ")',";
+    }
+  }
+  
+  @Test
+  @ShardsFixed(num = 3)
+  public void test() throws Exception {
+    buildIndexes(clients, STAT_FIELD);
+    commit();
+    
+    sanityCheckIndividualShards();
+    checkRefinementAndOverrequesting();
+    checkSubFacetStats();
+
+  }
+
+  public static void buildIndexes(final List<SolrClient> clients, final String statField) throws Exception {
+
+    assertEquals("This indexing code assumes exactly 3 shards/clients", 3, clients.size());
+    
+    final AtomicInteger docNum = new AtomicInteger();
+    final SolrClient shard0 = clients.get(0);
+    final SolrClient shard1 = clients.get(1);
+    final SolrClient shard2 = clients.get(2);
+    
+    // the 5 top foo_s terms have 100 docs each on every shard
+    for (int i = 0; i < 100; i++) {
+      for (int j = 0; j < 5; j++) {
+        shard0.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "aaa"+j, statField, j * 13 - i));
+        shard1.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "aaa"+j, statField, j * 3 + i));
+        shard2.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "aaa"+j, statField, i * 7 + j));
+      }
+    }
+
+    // 20 foo_s terms that come in "second" with 50 docs each 
+    // on both shard0 & shard1 ("bbb_")
+    for (int i = 0; i < 50; i++) {
+      for (int j = 0; j < 20; j++) {
+        shard0.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "bbb"+j, statField, 0));
+        shard1.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "bbb"+j, statField, 1));
+      }
+      // distracting term appears on only on shard2 50 times
+      shard2.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "junkA"));
+    }
+    // put "bbb0" on shard2 exactly once to sanity check refinement
+    shard2.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "bbb0", statField, -2));
+
+    // long 'tail' foo_s term appears in 45 docs on every shard
+    // foo_s:tail is the only term with bar_s sub-pivot terms
+    for (int i = 0; i < 45; i++) {
+
+      // for sub-pivot, shard0 & shard1 have 6 docs each for "tailB"
+      // but the top 5 terms are ccc(0-4) -- 7 on each shard
+      // (4 docs each have junk terms)
+      String sub_term = (i < 35) ? "ccc"+(i % 5) : ((i < 41) ? "tailB" : "junkA");
+      shard0.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "tail", "bar_s", sub_term, statField, i));
+      shard1.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "tail", "bar_s", sub_term, statField, i));
+
+      // shard2's top 5 sub-pivot terms are junk only it has with 8 docs each
+      // and 5 docs that use "tailB"
+      // NOTE: none of these get statField ! !
+      sub_term = (i < 40) ? "junkB"+(i % 5) : "tailB";
+      shard2.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "tail", "bar_s", sub_term));
+    }
+
+    // really long tail uncommon foo_s terms on shard2
+    for (int i = 0; i < 30; i++) {
+      // NOTE: using "Z" here so these sort before bbb0 when they tie for '1' instance each on shard2
+      shard2.add(sdoc("id", docNum.incrementAndGet(), "foo_s", "ZZZ"+i));
+    }
+
+  }
+
+  private void sanityCheckIndividualShards() throws Exception {
+    // sanity check that our expectations about each shard (non-distrib) are correct
+
+    SolrParams req = params( "q", "*:*", "distrib", "false", "json.facet",
+                             " { foo:{ type:terms, limit:10, field:foo_s, facet:{ bar:{ type:terms, limit:10, field:bar_s }}}}");
+
+    List<NamedList>[] shardFooBuckets = new List[clients.size()];
+    for (int i = 0; i < clients.size(); i++) {
+      shardFooBuckets[i] = (List<NamedList>)
+        ((NamedList<NamedList>)clients.get(i).query( req ).getResponse().get("facets")).get("foo").get("buckets");
+    }
+    
+    // top 5 same on all shards
+    for (int i = 0; i < 3; i++) {
+      assertEquals(10, shardFooBuckets[i].size());
+      for (int j = 0; j < 5; j++) {
+        NamedList bucket = shardFooBuckets[i].get(j);
+        assertEquals(bucket.toString(), "aaa"+j, bucket.get("val"));
+        assertEquals(bucket.toString(), 100, bucket.get("count"));
+      }
+    }
+    // top 6-10 same on shard0 & shard1
+    for (int i = 0; i < 2; i++) {
+      for (int j = 5; j < 10; j++) {
+        NamedList bucket = shardFooBuckets[i].get(j);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("bbb"));
+        assertEquals(bucket.toString(), 50, bucket.get("count"));
+      }
+    }
+
+    // 6-10 on shard2
+    assertEquals("junkA", shardFooBuckets[2].get(5).get("val"));
+    assertEquals(50, shardFooBuckets[2].get(5).get("count"));
+    assertEquals("tail", shardFooBuckets[2].get(6).get("val"));
+    assertEquals(45, shardFooBuckets[2].get(6).get("count"));
+    for (int j = 7; j < 10; j++) {
+      NamedList bucket = shardFooBuckets[2].get(j);
+      assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("ZZZ"));
+      assertEquals(bucket.toString(), 1, bucket.get("count"));
+    }
+    
+    // check 'bar' sub buckets on "tail" from shard2
+    { List<NamedList> bar_buckets = (List<NamedList>)  ((NamedList<NamedList>) shardFooBuckets[2].get(6).get("bar")).get("buckets");
+      assertEquals(6, bar_buckets.size());
+      for (int j = 0; j < 5; j++) {
+        NamedList bucket = bar_buckets.get(j);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("junkB"));
+        assertEquals(bucket.toString(), 8, bucket.get("count"));
+      }
+      NamedList bucket = bar_buckets.get(5);
+      assertEquals("tailB", bucket.get("val"));
+      assertEquals(5, bucket.get("count"));
+    }
+  }
+
+  private void checkRefinementAndOverrequesting() throws Exception {
+    // // distributed queries // //
+    
+    { // w/o refinement, the default overrequest isn't enough to find the long 'tail' *OR* the correct count for 'bbb0'...
+      List<NamedList> foo_buckets = (List<NamedList>)
+        ((NamedList<NamedList>)
+         queryServer( params( "q", "*:*", "shards", getShardsString(), "json.facet",
+                              "{ foo: { type:terms, refine:none, limit:6, field:foo_s } }"
+                              ) ).getResponse().get("facets")).get("foo").get("buckets");
+      assertEquals(6, foo_buckets.size());
+      for (int i = 0; i < 5; i++) {
+        NamedList bucket = foo_buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("aaa"));
+        assertEquals(bucket.toString(), 300L, bucket.get("count"));
+      }
+      
+      // this will be short the "+1" fo the doc added to shard2...
+      NamedList bucket = foo_buckets.get(5);
+      assertTrue(bucket.toString(), bucket.get("val").equals("bbb0")); // 'tail' is missed
+      assertEquals(bucket.toString(), 100L, bucket.get("count")); // will not include the "+1" for the doc added to shard2
+    }
+    
+    // even if we enable refinement, we still won't find the long 'tail' ...
+    // regardless of wether we use either the default overrequest, or disable overrequesting...
+    for (String over : Arrays.asList( "", "overrequest:0,")) { 
+      List<NamedList> foo_buckets = (List<NamedList>)
+        ((NamedList<NamedList>)
+         queryServer( params( "q", "*:*", "shards", getShardsString(), "json.facet",
+                              "{ foo: { type:terms, refine:simple, limit:6, "+ over +" field:foo_s, facet:{ " + ALL_STATS_JSON + 
+                              "  bar: { type:terms, refine:simple, limit:6, "+ over +" field:bar_s, facet:{"+ALL_STATS_JSON+"}}}}}"
+                              ) ).getResponse().get("facets")).get("foo").get("buckets");
+      assertEquals(6, foo_buckets.size());
+      for (int i = 0; i < 5; i++) {
+        NamedList bucket = foo_buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("aaa"));
+        assertEquals(bucket.toString(), 300L, bucket.get("count"));
+      }
+      // ...but it should have correctly asked shard2 to refine bbb0
+      NamedList bucket = foo_buckets.get(5);
+      assertTrue(bucket.toString(), bucket.get("val").equals("bbb0"));
+      assertEquals(bucket.toString(), 101L, bucket.get("count"));
+      // ...and the status under bbb0 should be correct to include the refinement
+      assertEquals(ALL_STATS.size() + 3, bucket.size()); // val,count,facet
+      assertEquals(-2L, bucket.get("min"));                                         // this min only exists on shard2
+      assertEquals(1L, bucket.get("max"));
+      // assertEquals(101L, bucket.get("num_values")); // TODO: SOLR-11695
+      // assertEquals(0L, bucket.get("missing")); // TODO: SOLR-11695
+      assertEquals(48.0D, bucket.get("sum"));
+      assertEquals(0.475247524752475D, (double) bucket.get("avg"), 0.1E-7);
+      assertEquals(54.0D, (double) bucket.get("sumsq"), 0.1E-7);
+      // assertEquals(0.55846323792D, bucket.getStddev(), 0.1E-7); // TODO: SOLR-11725
+      assertEquals(0.55569169111D, (double) bucket.get("stddev"), 0.1E-7); // json.facet is using the "uncorrected stddev"
+    }
+
+
+    // with a limit==6, we have to "overrequest >= 20" in order to ensure that 'tail' is included in the top 6
+    // this is because of how the "simple" refinement process works: the "top buckets" are determined based
+    // on the info available in the first pass request.
+    //
+    // Even though 'tail' is returned in the top6 for shard2, the cummulative total for 'bbb0' from shard0 and shard1 is
+    // high enough that the simple facet refinement ignores 'tail' because it assumes 'bbb0's final total will be greater.
+    //
+    // Meanwhile, for the sub-facet on 'bar', a limit==6 means we should correctly find 'tailB' as the top sub-term of 'tail',
+    // regardless of how much overrequest is used (or even if we don't have any refinement) since it's always in the top6...
+    for (String bar_opts : Arrays.asList( "refine:none,",
+                                          "refine:simple,",
+                                          "refine:none,   overrequest:0,",
+                                          "refine:simple, overrequest:0," )) {
+
+
+      List<NamedList> buckets = (List<NamedList>)
+        ((NamedList<NamedList>)
+         queryServer( params( "q", "*:*", "shards", getShardsString(), "json.facet",
+                              "{ foo: { type:terms, limit:6, overrequest:20, refine:simple, field:foo_s, facet:{ " +
+                              "  bar: { type:terms, limit:6, " + bar_opts + " field:bar_s }}}}"
+                              ) ).getResponse().get("facets")).get("foo").get("buckets");
+      
+      assertEquals(6, buckets.size());
+      for (int i = 0; i < 5; i++) {
+        NamedList bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("aaa"));
+        assertEquals(bucket.toString(), 300L, bucket.get("count"));
+      }
+    
+      NamedList bucket = buckets.get(5);
+      assertEquals(bucket.toString(), "tail", bucket.get("val"));
+      assertEquals(bucket.toString(), 135L, bucket.get("count"));
+      // check the sub buckets
+      buckets = ((NamedList<NamedList<List<NamedList>>>) bucket).get("bar").get("buckets");
+      assertEquals(6, buckets.size());
+      bucket = buckets.get(0);
+      assertEquals(bucket.toString(), "tailB", bucket.get("val"));
+      assertEquals(bucket.toString(), 17L, bucket.get("count"));
+      for (int i = 1; i < 6; i++) { // ccc(0-4)
+        bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("ccc"));
+        assertEquals(bucket.toString(), 14L, bucket.get("count"));
+      }
+    }
+    
+    // if we lower the limit on the sub-bucket to '5', overrequesting of at least 1 should still ensure 
+    // that we get the correct top5 including "tailB" -- even w/o refinement
+    for (String bar_opts : Arrays.asList( "refine:none,",
+                                          "refine:simple,",
+                                          "refine:none,   overrequest:1,",
+                                          "refine:simple, overrequest:1," )) {
+      
+      List<NamedList> buckets = (List<NamedList>)
+        ((NamedList<NamedList>)
+         queryServer( params( "q", "*:*", "shards", getShardsString(), "json.facet",
+                              "{ foo: { type:terms, limit:6, overrequest:20, refine:simple, field:foo_s, facet:{ " +
+                              "  bar: { type:terms, limit:5, " + bar_opts + " field:bar_s }}}}"
+                              ) ).getResponse().get("facets")).get("foo").get("buckets");
+      
+      assertEquals(6, buckets.size());
+      for (int i = 0; i < 5; i++) {
+        NamedList bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("aaa"));
+        assertEquals(bucket.toString(), 300L, bucket.get("count"));
+      }
+      NamedList bucket = buckets.get(5);
+      assertEquals(bucket.toString(), "tail", bucket.get("val"));
+      assertEquals(bucket.toString(), 135L, bucket.get("count"));
+      // check the sub buckets
+      buckets = ((NamedList<NamedList<List<NamedList>>>) bucket).get("bar").get("buckets");
+      assertEquals(5, buckets.size());
+      bucket = buckets.get(0);
+      assertEquals(bucket.toString(), "tailB", bucket.get("val"));
+      assertEquals(bucket.toString(), 17L, bucket.get("count"));
+      for (int i = 1; i < 5; i++) { // ccc(0-3)
+        bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("ccc"));
+        assertEquals(bucket.toString(), 14L, bucket.get("count"));
+      }
+    }
+    
+    // however: with a lower sub-facet limit==5, and overrequesting disabled, 
+    // we're going to miss out on tailB even if we have refinement
+    for (String bar_opts : Arrays.asList( "refine:none,   overrequest:0,",
+                                          "refine:simple, overrequest:0," )) {
+      
+      List<NamedList> buckets = (List<NamedList>)
+        ((NamedList<NamedList>)
+         queryServer( params( "q", "*:*", "shards", getShardsString(), "json.facet",
+                              "{ foo: { type:terms, limit:6, overrequest:20, refine:simple, field:foo_s, facet:{ " +
+                              "  bar: { type:terms, limit:5, " + bar_opts + " field:bar_s }}}}"
+                              ) ).getResponse().get("facets")).get("foo").get("buckets");
+
+      assertEquals(6, buckets.size());
+      for (int i = 0; i < 5; i++) {
+        NamedList bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("aaa"));
+        assertEquals(bucket.toString(), 300L, bucket.get("count"));
+      }
+      NamedList bucket = buckets.get(5);
+      assertEquals(bucket.toString(), "tail", bucket.get("val"));
+      assertEquals(bucket.toString(), 135L, bucket.get("count"));
+      // check the sub buckets
+      buckets = ((NamedList<NamedList<List<NamedList>>>) bucket).get("bar").get("buckets");
+      assertEquals(5, buckets.size());
+      for (int i = 0; i < 5; i++) { // ccc(0-4)
+        bucket = buckets.get(i);
+        assertTrue(bucket.toString(), bucket.get("val").toString().startsWith("ccc"));
+        assertEquals(bucket.toString(), 14L, bucket.get("count"));
+      }
+    }
+    
+  }
+
+  private void checkSubFacetStats() throws Exception { 
+    // Deep checking of some Facet stats
+    
+    // the assertions only care about the first 5 results of each facet, but to get the long tail more are needed
+    // from the sub-shards.  results should be the same regardless of: "high limit" vs "low limit + high overrequest"
+    checkSubFacetStats("refine:simple, limit: 100,");
+    checkSubFacetStats("refine:simple, overrequest: 100,");
+
+    // and the results shouldn't change if we explicitly disable refinement
+    checkSubFacetStats("refine:none, limit: 100,");
+    checkSubFacetStats("refine:none, overrequest: 100,");
+
+  }
+  
+  private void checkSubFacetStats(String extraJson) throws Exception {
+    String commonJson = "type: terms, " + extraJson;
+    NamedList<NamedList> all_facets = (NamedList) queryServer
+      ( params( "q", "*:*", "shards", getShardsString(), "rows" , "0", "json.facet",
+                "{ foo : { " + commonJson + " field: foo_s, facet: { " +
+                ALL_STATS_JSON + " bar: { " + commonJson + " field: bar_s, facet: { " + ALL_STATS_JSON + "} } } } }"
+      ) ).getResponse().get("facets");
+    
+    assertNotNull(all_facets);
+
+    List<NamedList> foo_buckets = (List) ((NamedList)all_facets.get("foo")).get("buckets");
+
+    NamedList aaa0_Bucket = foo_buckets.get(0);
+    assertEquals(ALL_STATS.size() + 3, aaa0_Bucket.size()); // val,count,facet
+    assertEquals("aaa0", aaa0_Bucket.get("val"));
+    assertEquals(300L, aaa0_Bucket.get("count"));
+    assertEquals(-99L, aaa0_Bucket.get("min"));
+    assertEquals(693L, aaa0_Bucket.get("max"));
+    // assertEquals(300L, aaa0_Bucket.get("num_values")); // TODO: SOLR-11695
+    // assertEquals(0L, aaa0_Bucket.get("missing")); // TODO: SOLR-11695
+    assertEquals(34650.0D, aaa0_Bucket.get("sum"));
+    assertEquals(115.5D, (double) aaa0_Bucket.get("avg"), 0.1E-7);
+    assertEquals(1.674585E7D, (double) aaa0_Bucket.get("sumsq"), 0.1E-7);
+    // assertEquals(206.4493184076D, (double) aaa0_Bucket.get("stddev"), 0.1E-7); // TODO: SOLR-11725
+    assertEquals(206.1049489944D, (double) aaa0_Bucket.get("stddev"), 0.1E-7); // json.facet is using the "uncorrected stddev"
+
+    NamedList tail_Bucket = foo_buckets.get(5);
+    assertEquals(ALL_STATS.size() + 3, tail_Bucket.size()); // val,count,facet
+    assertEquals("tail", tail_Bucket.get("val"));
+    assertEquals(135L, tail_Bucket.get("count"));
+    assertEquals(0L, tail_Bucket.get("min"));
+    assertEquals(44L, tail_Bucket.get("max"));
+    // assertEquals(90L, tail_Bucket.get("num_values")); // TODO: SOLR-11695
+    // assertEquals(45L, tail_Bucket.get("missing")); // TODO: SOLR-11695
+    assertEquals(1980.0D, tail_Bucket.get("sum"));
+    assertEquals(22.0D, (double) tail_Bucket.get("avg"), 0.1E-7);
+    assertEquals(58740.0D, (double) tail_Bucket.get("sumsq"), 0.1E-7);
+    // assertEquals(13.0599310011D, (double) tail_Bucket.get("stddev"), 0.1E-7); // TODO: SOLR-11725
+    assertEquals(12.9871731592D, (double) tail_Bucket.get("stddev"), 0.1E-7); // json.facet is using the "uncorrected stddev"
+
+    List<NamedList> tail_bar_buckets = (List) ((NamedList)tail_Bucket.get("bar")).get("buckets");
+   
+    NamedList tailB_Bucket = tail_bar_buckets.get(0);
+    assertEquals(ALL_STATS.size() + 2, tailB_Bucket.size()); // val,count ... NO SUB FACETS
+    assertEquals("tailB", tailB_Bucket.get("val"));
+    assertEquals(17L, tailB_Bucket.get("count"));
+    assertEquals(35L, tailB_Bucket.get("min"));
+    assertEquals(40L, tailB_Bucket.get("max"));
+    // assertEquals(12L, tailB_Bucket.get("num_values")); // TODO: SOLR-11695
+    // assertEquals(5L, tailB_Bucket.get("missing")); // TODO: SOLR-11695
+    assertEquals(450.0D, tailB_Bucket.get("sum"));
+    assertEquals(37.5D, (double) tailB_Bucket.get("avg"), 0.1E-7);
+    assertEquals(16910.0D, (double) tailB_Bucket.get("sumsq"), 0.1E-7);
+    // assertEquals(1.78376517D, (double) tailB_Bucket.get("stddev"), 0.1E-7); // TODO: SOLR-11725
+    assertEquals(1.70782513D, (double) tailB_Bucket.get("stddev"), 0.1E-7); // json.facet is using the "uncorrected stddev"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solr-ref-guide/src/collections-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index 784e2cf..a0c8038 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -109,6 +109,9 @@ Details of the snitch provider. See the section <<rule-based-replica-placement.a
 
 `policy`:: Name of the collection-level policy. See <<solrcloud-autoscaling-policy-preferences.adoc#collection-specific-policy, Defining Collection-Specific Policies >> for details.
 
+`waitForFinalState`::
+If `true`, the request will complete only when all affected replicas become active. The default is `false`, which means that the API will return the status of the single action, which may be before the new replica is online and active.
+
 === CREATE Response
 
 The response will include the status of the request and the new core names. If the status is anything other than "success", an error message will explain why the request failed.
@@ -267,6 +270,9 @@ For example, suppose `split.key=A!` hashes to the range `12-15` and belongs to s
 `property._name_=_value_`::
 Set core property _name_ to _value_. See the section <<defining-core-properties.adoc#defining-core-properties,Defining core.properties>> for details on supported properties and values.
 
+`waitForFinalState`::
+If `true`, the request will complete only when all affected replicas become active. The default is `false`, which means that the API will return the status of the single action, which may be before the new replica is online and active.
+
 `async`::
 Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>
 
@@ -372,6 +378,9 @@ The format is a comma-separated list of node_names, such as `localhost:8983_solr
 `property._name_=_value_`::
 Set core property _name_ to _value_. See the section <<defining-core-properties.adoc#defining-core-properties,Defining core.properties>> for details on supported properties and values.
 
+`waitForFinalState`::
+If `true`, the request will complete only when all affected replicas become active. The default is `false`, which means that the API will return the status of the single action, which may be before the new replica is online and active.
+
 `async`::
 Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
 
@@ -749,6 +758,9 @@ See the section <<shards-and-indexing-data-in-solrcloud.adoc#types-of-replicas,T
 `property._name_=_value_`::
 Set core property _name_ to _value_. See <<defining-core-properties.adoc#defining-core-properties,Defining core.properties>> for details about supported properties and values.
 
+`waitForFinalState`::
+If `true`, the request will complete only when all affected replicas become active. The default is `false`, which means that the API will return the status of the single action, which may be before the new replica is online and active.
+
 `async`::
 Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>
 
@@ -1885,6 +1897,7 @@ Time in seconds to wait until new replicas are created, and until leader replica
 ====
 This operation does not hold necessary locks on the replicas that belong to on the source node. So don't perform other collection operations in this period.
 ====
+
 [[movereplica]]
 == MOVEREPLICA: Move a Replica to a New Node
 
@@ -1913,14 +1926,17 @@ The name of the destination node. This parameter is required.
 Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
 
 [[utilizenode]]
-== UTILIZENODE: Utilize a new node
+== UTILIZENODE: Utilize a New Node
+
+This command can be used to move some replicas from the existing nodes to either a new node or a less loaded node to reduce the load on the existing node.
 
-This command can be used to move some replicas from the existing nodes to a new node or lightly loaded node and reduce the load on them. This uses your autoscaling policies and preferences to identify which replica needs to be moved. It tries to fix any policy violations first and then it tries to move some load off of the most loaded nodes according to the preferences.
+This uses your autoscaling policies and preferences to identify which replica needs to be moved. It tries to fix any policy violations first and then it tries to move some load off of the most loaded nodes according to the preferences.
 
 `/admin/collections?action=UTILIZENODE&node=nodeName`
+
 === UTILIZENODE Parameters
 
-`node`:: The name of the node that needs to be utilized. This parameter is required
+`node`:: The name of the node that needs to be utilized. This parameter is required.
 
 
 == Asynchronous Calls

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solr-ref-guide/src/installing-solr.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/installing-solr.adoc b/solr/solr-ref-guide/src/installing-solr.adoc
index d537420..9e7b23a 100644
--- a/solr/solr-ref-guide/src/installing-solr.adoc
+++ b/solr/solr-ref-guide/src/installing-solr.adoc
@@ -65,7 +65,7 @@ Once extracted, you are now ready to run Solr using the instructions provided in
 
 After installing Solr, you'll see the following directories and files within them:
 
-bin::
+bin/::
 This directory includes several important scripts that will make using Solr easier.
 
 solr and solr.cmd::: This is <<solr-control-script-reference.adoc#solr-control-script-reference,Solr's Control Script>>, also known as `bin/solr` (*nix) / `bin/solr.cmd` (Windows). This script is the preferred tool to start and stop Solr. You can also create collections or cores, configure authentication, and work with configuration files when running in SolrCloud mode.
@@ -78,22 +78,22 @@ These are property files for *nix and Windows systems, respectively. System-leve
 install_solr_services.sh:::
 This script is used on *nix systems to install Solr as a service. It is described in more detail in the section <<taking-solr-to-production.adoc#taking-solr-to-production,Taking Solr to Production>>.
 
-contrib::
+contrib/::
 Solr's `contrib` directory includes add-on plugins for specialized features of Solr.
 
-dist::
+dist/::
 The `dist` directory contains the main Solr .jar files.
 
-docs::
+docs/::
 The `docs` directory includes a link to online Javadocs for Solr.
 
-example::
+example/::
 The `example` directory includes several types of examples that demonstrate various Solr capabilities. See the section <<Solr Examples>> below for more details on what is in this directory.
 
-licenses::
+licenses/::
 The `licenses` directory includes all of the licenses for 3rd party libraries used by Solr.
 
-server::
+server/::
 This directory is where the heart of the Solr application resides. A README in this directory provides a detailed overview, but here are some highlights:
 * Solr's Admin UI (`server/solr-webapp`)
 * Jetty libraries (`server/lib`)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solr-upgrade-notes.adoc b/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
index 30d27bf..05f7ee2 100644
--- a/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
+++ b/solr/solr-ref-guide/src/solr-upgrade-notes.adoc
@@ -25,7 +25,32 @@ When planning your Solr upgrade, consider the customizations you have made to yo
 
 Detailed steps for upgrading a Solr cluster can be found in the section <<upgrading-a-solr-cluster.adoc#upgrading-a-solr-cluster,Upgrading a Solr Cluster>>.
 
-== Upgrading from Solr 7.0
+== Upgrading from 7.x Releases
+
+=== Solr 7.2
+
+See the https://wiki.apache.org/solr/ReleaseNote72[7.2 Release Notes] for an overview of the main new features in Solr 7.2.
+
+Users should be aware of the following major changes from v7.1:
+
+* Starting a query string with <<local-parameters-in-queries.adoc#local-parameters-in-queries,local parameters>> `{!myparser ...}` is used to switch from one query parser to another, and is intended for use by Solr system developers, not end users doing searches. To reduce negative side-effects of unintended hack-ability, Solr now limits the cases when local parameters will be parsed to only contexts in which the default parser is "<<other-parsers.adoc#lucene-query-parser,lucene>>" or "<<other-parsers.adoc#function-query-parser,func>>".
++
+So, if `defType=edismax` then `q={!myparser ...}` won't work. In that example, put the desired query parser into the `defType` parameter.
++
+Another example is if `deftype=edismax` then `hl.q={!myparser ...}` won't work for the same reason. In this example, either put the desired query parser into the `hl.qparser` parameter or set `hl.qparser=lucene`. Most users won't run into these cases but some will need to change.
++
+If you must have full backwards compatibility, use `luceneMatchVersion=7.1.0` or an earlier version.
+
+* The eDisMax parser by default no longer allows subqueries that specify a Solr parser using either local parameters, or the older `\_query_` magic field trick.
++
+For example, `{!prefix f=myfield v=enterp}` or `\_query_:"{!prefix f=myfield v=enterp}"` are not supported by default any longer. If you want to allow power-users to do this, set `uf=*,\_query_` or some other value that includes `\_query_`.
++
+If you need full backwards compatibility for the time being, use `luceneMatchVersion=7.1.0` or something earlier.
+
+=== Solr 7.1
+
+See the https://wiki.apache.org/solr/ReleaseNote71[7.1 Release Notes] for an overview of the main new features of Solr 7.1.
+
 Users should be aware of the following major changes from v7.0:
 
 * The feature to automatically add replicas if a replica goes down, previously available only when storing indexes in HDFS, has been ported to the autoscaling framework. Due to this, `autoAddReplicas` is now available to all users even if their indexes are on local disks.
@@ -39,7 +64,7 @@ More information about the changes to this feature can be found in the section <
 
 * Shard and cluster metric reporter configuration now require a class attribute.
 ** If a reporter configures the `group="shard"` attribute then please also configure the `class="org.apache.solr.metrics.reporters.solr.SolrShardReporter"` attribute.
-** If a reporter configures the `group="cluster"` attribute then please also configure the   `class="org.apache.solr.metrics.reporters.solr.SolrClusterReporter"` attribute.
+** If a reporter configures the `group="cluster"` attribute then please also configure the  `class="org.apache.solr.metrics.reporters.solr.SolrClusterReporter"` attribute.
 +
 See the section <<metrics-reporting.adoc#shard-and-cluster-reporters,Shard and Cluster Reporters>> for more information.
 
@@ -48,12 +73,12 @@ See the section <<metrics-reporting.adoc#shard-and-cluster-reporters,Shard and C
 * The `RunExecutableListener` was removed for security reasons. If you want to listen to events caused by updates, commits, or optimize, write your own listener as native Java class as part of a Solr plugin.
 * In the XML query parser (`defType=xmlparser` or `{!xmlparser ... }`) the resolving of external entities is now disallowed by default.
 
-== Upgrading from Any 6.x Release
+== Upgrading to 7.0 from Any 6.x Release
 
 The upgrade from Solr 6.x to Solr 7 introduces several *major* changes that you should be aware of before upgrading. Please do a thorough review of the section <<major-changes-in-solr-7.adoc#major-changes-in-solr-7,Major Changes in Solr 7>> before starting your upgrade.
 
-== Upgrading from Older Versions of Solr
+== Upgrading from pre-6.x Versions of Solr
 
 Users upgrading from versions of Solr prior to 6.x are strongly encouraged to consult {solr-javadocs}/changes/Changes.html[`CHANGES.txt`] for the details of _all_ changes since the version they are upgrading from.
 
-A summary of the significant changes between Solr 5.x and Solr 6.0 can be found in the <<major-changes-from-solr-5-to-solr-6.adoc#major-changes-from-solr-5-to-solr-6,Major Changes from Solr 5 to Solr 6>> section.
+A summary of the significant changes between Solr 5.x and Solr 6.0 can be found in the section <<major-changes-from-solr-5-to-solr-6.adoc#major-changes-from-solr-5-to-solr-6,Major Changes from Solr 5 to Solr 6>>.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
index 9d53552..51ee31f 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
@@ -142,10 +142,13 @@ However, since the first node in the first example had more than 1 replica for a
 
 In the above example the node with port 8983 has two replicas for `shard1` in violation of our policy.
 
-== Suggestions API ==
-Suggestions are operations recommended by the system according to the policies and preferences the user has set. Note that the suggestions are made only if there are `violations` to the policies and the collection admin operation would use the preferences to identify the target node.
+== Suggestions API
+Suggestions are operations recommended by the system according to the policies and preferences the user has set.
+
+Suggestions are made only if there are `violations` to active policies. The `operation` section of the response uses the defined preferences to identify the target node.
+
+The API is available at `/admin/autocaling/suggestion`. Here is an example output from a suggestion request:
 
-The API is available at `/admin/autocaling/suggestion`
 [source,json]
 ----
 {
@@ -191,7 +194,7 @@ The API is available at `/admin/autocaling/suggestion`
   "WARNING":"This response format is experimental.  It is likely to change in the future."}
 ----
 
-The operation is an actual API call that can be invoked to remedy the current violation
+The suggested `operation` is an API call that can be invoked to remedy the current violation.
 
 == History API
 
@@ -213,14 +216,31 @@ further insight into e.g., exact operations that were computed and/or executed.
 Specifically, the following query parameters can be used (they are turned into
 filter queries, so an implicit AND is applied):
 
-* `trigger` - trigger name
-* `eventType` - event type / trigger type (e.g., `nodeAdded`)
-* `collection` - collection name involved in event processing
-* `stage` - event processing stage
-* `action` - trigger action
-* `node` - node name that the event refers to
-* `beforeAction` - beforeAction stage
-* `afterAction` - afterAction stage
+`trigger`::
+The name of the trigger.
+
+`eventType`::
+The event type or trigger type (e.g., `nodeAdded`).
+
+`collection`::
+The name of the collection involved in event processing.
+
+`stage`::
+An event processing stage.
+
+`action`::
+A trigger action.
+
+`node`::
+A node name that the event refers to.
+
+`beforeAction`::
+A `beforeAction` stage.
+
+`afterAction`::
+An `afterAction` stage.
+
+// TODO someday add an input example also
 
 .Example output
 [source,json]
@@ -559,20 +579,27 @@ The `remove-listener` command can be used to remove an existing listener. It acc
 }
 ----
 
-=== Change AutoScaling Properties
+=== Change Autoscaling Properties
 
 The `set-properties` command can be used to change the default properties used by the Autoscaling framework.
 
-The following well known properties can be specified in the payload:
+The following properties can be specified in the payload:
+
+`triggerScheduleDelaySeconds`::
+This is the delay in seconds between two executions of a trigger. Every trigger is scheduled using Java's ScheduledThreadPoolExecutor with this delay. The default is `1` second.
+
+`triggerCooldownPeriodSeconds`::
+Solr pauses all other triggers for this cool down period after a trigger fires so that the system can stabilize before running triggers again. The default is `5` seconds.
+
+`triggerCorePoolSize`::
+The core pool size of the `ScheduledThreadPoolExecutor` used to schedule triggers. The default is `4` threads.
 
-* `triggerScheduleDelaySeconds` (defaults to 1 second): This is the delay in seconds between two executions of a trigger. Every trigger is scheduled using Java's ScheduledThreadPoolExecutor with this delay.
-* `triggerCooldownPeriodSeconds` (defaults to 5 seconds): Solr pauses all other triggers for this cool down period after a trigger fires so that the system can stabilize before running triggers again.
-* `triggerCorePoolSize` (defaults to 4 threads): The core pool size of the `ScheduledThreadPoolExecutor` used to schedule triggers.
-* `actionThrottlePeriodSeconds` (defaults to 5 seconds): This is the minimum throttling delay between executing actions for triggers. It is guaranteed that actions for two trigger events are executed after this delay period.
+`actionThrottlePeriodSeconds`::
+This is the minimum throttling delay between executing actions for triggers. It is guaranteed that actions for two trigger events are executed after this delay period. The default is `5` seconds.
 
-The command allows setting arbitrary properties in addition to the above well-known properties. Such arbitrary properties can be useful in custom `TriggerAction` instances.
+The command allows setting arbitrary properties in addition to the above properties. Such arbitrary properties can be useful in custom `TriggerAction` instances.
 
-.Change default triggerScheduleDelaySeconds
+.Change default `triggerScheduleDelaySeconds`
 [source.json]
 ----
 {
@@ -582,11 +609,11 @@ The command allows setting arbitrary properties in addition to the above well-kn
 }
 ----
 
-The set-properties command replaces older values if present. So using set-properties to set the same value twice will overwrite the old value.
+The `set-properties` command replaces older values if present. So using `set-properties` to set the same value twice will overwrite the old value.
 If a property is not specified then it retains the last set value or the default, if no change was made.
 A changed value can be unset by using a null value.
 
-.Revert changed value of triggerScheduleDelaySeconds to default value
+.Revert changed value of `triggerScheduleDelaySeconds` to default
 [source.json]
 ----
 {
@@ -596,4 +623,4 @@ A changed value can be unset by using a null value.
 }
 ----
 
-The changed values of these properties, if any, can be read using the Autoscaling Read API in the `properties` section.
+The changed values of these properties, if any, can be read using the Autoscaling <<Read API>> in the `properties` section.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e80554a8/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java b/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
index 52e5f4a..b61df07 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
@@ -135,8 +135,12 @@ public interface FacetParams {
    * In SOLR-8599 and SOLR-8988, significant performance increase has been seen when enabling this optimization.
    *
    * Note: enabling this flag has no effect when the conditions above are not met. For those other cases the default behavior is sufficient.
+   *
+   * @deprecated
+   * This option is no longer used nor will if affect any queries as the fix has been built in. (SOLR-11711)
+   * This will be removed entirely in 8.0.0
    */
-
+  @Deprecated
   public static final String FACET_DISTRIB_MCO = FACET_DISTRIB + ".mco";
   
   /**


[5/5] lucene-solr:jira/solr-11702: SOLR-11702: Improve error handling for ZkShardTerms

Posted by da...@apache.org.
SOLR-11702: Improve error handling for ZkShardTerms


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

Branch: refs/heads/jira/solr-11702
Commit: f881a62a65f8230f97167142a32180e8c52ef7b9
Parents: e80554a
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu Dec 14 14:56:19 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu Dec 14 14:56:19 2017 +0700

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f881a62a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index 0a4cb9e..15e8c61 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -181,7 +181,7 @@ public class ZkShardTerms implements AutoCloseable{
     } catch (NoSuchElementException e) {
       throw e;
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error save shard term for collection:" + collection, e);
     }
     return false;
   }
@@ -228,8 +228,11 @@ public class ZkShardTerms implements AutoCloseable{
       version = data.getVersion();
       terms = (Map<String, Long>) Utils.fromJSON(data.getData());
       onTermUpdates();
-    } catch (Exception e) {
-      e.printStackTrace();
+    } catch (InterruptedException e) {
+      Thread.interrupted();
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error updating shard term for collection:" + collection, e);
+    } catch (IOException | KeeperException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error updating shard term for collection:" + collection, e);
     }
   }