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

[01/25] lucene-solr:jira/solr-8593: SOLR-9915: PeerSync alreadyInSync check is not backwards compatible and results in full replication during a rolling restart

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-8593 5f6548765 -> 3793eb5ec


SOLR-9915: PeerSync alreadyInSync check is not backwards compatible and results in full replication during a rolling restart


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

Branch: refs/heads/jira/solr-8593
Commit: 5b1f6b2ba48f8afc6c822c097d0500eb2ed66815
Parents: f330678
Author: Noble Paul <no...@apache.org>
Authored: Wed Jan 4 04:38:51 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Wed Jan 4 04:38:51 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                        | 4 ++++
 solr/core/src/java/org/apache/solr/update/PeerSync.java | 8 +++++++-
 2 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b1f6b2b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index afcd295..8cba8a9 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -370,6 +370,10 @@ Other Changes
 * SOLR-9899: StandardDirectoryFactory should use optimizations for all FilterDirectorys not just NRTCachingDirectory.
   (Mark Miller)
 
+* SOLR-9915: PeerSync alreadyInSync check is not backwards compatible and results in full replication during a rolling restart
+  (Tim Owen via noble)
+
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b1f6b2b/solr/core/src/java/org/apache/solr/update/PeerSync.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index 1f61a56..12ba7ae 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -335,9 +335,15 @@ public class PeerSync  {
     for (;;) {
       ShardResponse srsp = shardHandler.takeCompletedOrError();
       if (srsp == null) break;
+
+      Object replicaFingerprint = srsp.getSolrResponse().getResponse().get("fingerprint");
+      if (replicaFingerprint == null) {
+        log.warn("Replica did not return a fingerprint - possibly an older Solr version");
+        continue;
+      }
       
       try {
-        IndexFingerprint otherFingerprint = IndexFingerprint.fromObject(srsp.getSolrResponse().getResponse().get("fingerprint"));
+        IndexFingerprint otherFingerprint = IndexFingerprint.fromObject(replicaFingerprint);
         IndexFingerprint ourFingerprint = IndexFingerprint.getFingerprint(core, Long.MAX_VALUE);
         if(IndexFingerprint.compare(otherFingerprint, ourFingerprint) == 0) {
           log.info("We are already in sync. No need to do a PeerSync ");


[05/25] lucene-solr:jira/solr-8593: SOLR-8530: Updated CHANGES.txt

Posted by kr...@apache.org.
SOLR-8530: Updated CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: db7d2ff1629e7ae45a405eebdcdde1c68664d01f
Parents: 390a01b
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 17:09:21 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:09:21 2017 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/db7d2ff1/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8cba8a9..3281158 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -214,6 +214,8 @@ New Features
 
 * SOLR-9854: Collect metrics for index merges and index store IO (ab)
 
+* SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[12/25] lucene-solr:jira/solr-8593: SOLR-9919: Update CHANGES.txt

Posted by kr...@apache.org.
SOLR-9919: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: 194f5167f496ec1ffcabe4a0d9966ef88f590b63
Parents: b8f4adf
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Jan 4 10:35:23 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Jan 4 10:35:23 2017 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/194f5167/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ff1fdc7..cb41602 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -325,6 +325,8 @@ Bug Fixes
 
 * SOLR-9154: Fix DirectSolrSpellChecker to work when added through the Config API. (Anshum Gupta)
 
+* SOLR-9919: random Streaming Expression is not registered in /stream or /graph handler (Joel Bernstein)
+
 Other Changes
 ----------------------
 


[21/25] lucene-solr:jira/solr-8593: SOLR-9911: Remove http group from example in change log

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


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

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

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


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


[08/25] lucene-solr:jira/solr-8593: SOLR-9911: Add a way to filter metrics by prefix in the MetricsHandler API

Posted by kr...@apache.org.
SOLR-9911: Add a way to filter metrics by prefix in the MetricsHandler API


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

Branch: refs/heads/jira/solr-8593
Commit: 0452cb8c969f156dc7086d16b2ca0b7814fa322e
Parents: 12d8492
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed Jan 4 20:17:45 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Wed Jan 4 20:17:45 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                | 10 ++++--
 .../solr/handler/admin/MetricsHandler.java      | 16 +++++++--
 .../org/apache/solr/util/stats/MetricUtils.java | 13 ++++++--
 .../solr/handler/admin/MetricsHandlerTest.java  | 35 +++++++++++++++++++-
 4 files changed, 65 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0452cb8c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1ad6359..82fd7ae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -186,9 +186,13 @@ New Features
 
 * SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)
 
-* SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API. API supports two
-  optional parameters 'group' (all,jvm,jetty,http,node,core) and 'type' (all,counter,timer,gauge,histogram) both
-  of which are multi-valued. Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
+* SOLR-9812: SOLR-9911: Added a new /admin/metrics API to return all metrics collected by Solr via API.
+  API supports three optional parameters:
+  * 'group' (all,jvm,jetty,http,node,core),
+  * 'type' (all,counter,timer,gauge,histogram) both of which are multi-valued
+  * 'prefix' that filters the return metrics
+  Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
+  Example: http://localhost:8983/solr/admin/metrics?group=jvm&prefix=buffers
   (shalin)
 
 * SOLR-9884: Add version to segments handler output (Steven Bower via Erick Erickson)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0452cb8c/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
index 78b2045..428a72b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -71,6 +71,7 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
       throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized");
     }
 
+    MetricFilter mustMatchFilter = parseMustMatchFilter(req);
     List<MetricType> metricTypes = parseMetricTypes(req);
     List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
     List<Group> requestedGroups = parseGroups(req);
@@ -86,16 +87,27 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
             coreRegistryName = core.getCoreMetricManager().getRegistryName();
           }
           MetricRegistry registry = metricManager.registry(coreRegistryName);
-          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
+          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter));
         });
       } else {
         MetricRegistry registry = metricManager.registry(registryName);
-        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
+        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter));
       }
     }
     rsp.getValues().add("metrics", response);
   }
 
+  private MetricFilter parseMustMatchFilter(SolrQueryRequest req) {
+    String prefix = req.getParams().get("prefix");
+    MetricFilter mustMatchFilter;
+    if (prefix != null) {
+      mustMatchFilter = new SolrMetricManager.PrefixFilter(prefix.trim());
+    } else  {
+      mustMatchFilter = MetricFilter.ALL;
+    }
+    return mustMatchFilter;
+  }
+
   private List<Group> parseGroups(SolrQueryRequest req) {
     String[] groupStr = req.getParams().getParams("group");
     List<String> groups = Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0452cb8c/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
index 4a83c86..0d386ae 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -72,14 +72,21 @@ public class MetricUtils {
    * are converted to NamedList which match at least one of the given MetricFilter instances.
    *
    * @param registry      the {@link MetricRegistry} to be converted to NamedList
-   * @param metricFilters a list of {@link MetricFilter} instances
+   * @param shouldMatchFilters a list of {@link MetricFilter} instances.
+   *                           A metric must match <em>any one</em> of the filters from this list to be
+   *                           included in the output
+   * @param mustMatchFilter a {@link MetricFilter}.
+   *                        A metric <em>must</em> match this filter to be included in the output.
    * @return a {@link NamedList}
    */
-  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> metricFilters) {
+  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> shouldMatchFilters, MetricFilter mustMatchFilter) {
     NamedList response = new NamedList();
     Map<String, Metric> metrics = registry.getMetrics();
     SortedSet<String> names = registry.getNames();
-    names.stream().filter(s -> metricFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s)))).forEach(n -> {
+    names.stream()
+        .filter(s -> shouldMatchFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s))))
+        .filter(s -> mustMatchFilter.matches(s, metrics.get(s)))
+        .forEach(n -> {
       Metric metric = metrics.get(n);
       if (metric instanceof Counter) {
         Counter counter = (Counter) metric;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0452cb8c/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index e15778d..67bf0e3 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -91,7 +91,40 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
     assertEquals(1, values.size());
-    assertNotNull(values.get("solr.node"));
+    values = (NamedList) values.get("solr.node");
+    assertNotNull(values);
     assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "prefix", "cores"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(5, values.size());
+    assertEquals(0, ((NamedList)values.get("solr.jvm")).size());
+    assertEquals(0, ((NamedList)values.get("solr.http")).size());
+    assertEquals(0, ((NamedList)values.get("solr.jetty")).size());
+    assertEquals(0, ((NamedList)values.get("solr.core.collection1")).size());
+    assertEquals(3, ((NamedList)values.get("solr.node")).size());
+    assertNotNull(values.get("solr.node"));
+    values = (NamedList) values.get("solr.node");
+    assertNotNull(values.get("cores.lazy")); // this is a gauge node
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "prefix", "cores"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertEquals(0, ((NamedList)values.get("solr.jvm")).size());
+    assertNull(values.get("solr.node"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "timer", "prefix", "cores"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertEquals(0, ((NamedList)values.get("solr.node")).size());
   }
 }


[02/25] lucene-solr:jira/solr-8593: SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions

Posted by kr...@apache.org.
SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions


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

Branch: refs/heads/jira/solr-8593
Commit: 1da283ef2c673b2effac834da1de1cb94c0118bb
Parents: 5b1f6b2
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 13:09:49 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 14:53:03 2017 -0500

----------------------------------------------------------------------
 .../solr/client/solrj/io/ops/AndOperation.java  | 101 ++++++++++
 .../client/solrj/io/ops/BooleanOperation.java   |  24 +++
 .../client/solrj/io/ops/EqualsOperation.java    |  70 +++++++
 .../io/ops/GreaterThanEqualToOperation.java     |  70 +++++++
 .../solrj/io/ops/GreaterThanOperation.java      |  70 +++++++
 .../solr/client/solrj/io/ops/LeafOperation.java |  59 ++++++
 .../solrj/io/ops/LessThanEqualToOperation.java  |  70 +++++++
 .../client/solrj/io/ops/LessThanOperation.java  |  70 +++++++
 .../solr/client/solrj/io/ops/NotOperation.java  |  87 +++++++++
 .../solr/client/solrj/io/ops/OrOperation.java   |  71 +++++++
 .../client/solrj/io/stream/HavingStream.java    | 190 +++++++++++++++++++
 11 files changed, 882 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
new file mode 100644
index 0000000..f095f63
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
@@ -0,0 +1,101 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class AndOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected BooleanOperation leftOperand;
+  protected BooleanOperation rightOperand;
+
+  public void operate(Tuple tuple) {
+    leftOperand.operate(tuple);
+    rightOperand.operate(tuple);
+  }
+
+  public AndOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
+    this.leftOperand = leftOperand;
+    this.rightOperand = rightOperand;
+  }
+
+  public AndOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+      List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+      if(operationExpressions != null && operationExpressions.size() == 2) {
+        StreamExpression left = operationExpressions.get(0);
+        StreamOperation leftOp = factory.constructOperation(left);
+        if(leftOp instanceof BooleanOperation) {
+          leftOperand = (BooleanOperation) leftOp;
+        } else {
+          throw new IOException("The And/Or Operation requires a BooleanOperation.");
+        }
+
+        StreamExpression right = operationExpressions.get(1);
+        StreamOperation rightOp = factory.constructOperation(right);
+        if(rightOp instanceof BooleanOperation) {
+          rightOperand = (BooleanOperation) rightOp;
+        } else {
+          throw new IOException("The And/Or Operation requires a BooleanOperation.");
+        }
+      } else {
+        throw new IOException("The And/Or Operation requires a BooleanOperations.");
+      }
+  }
+
+  public boolean evaluate() {
+    return leftOperand.evaluate() && rightOperand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(leftOperand instanceof Expressible) {
+      expression.addParameter(leftOperand.toExpression(factory));
+    } else {
+      throw new IOException("This left operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    if(rightOperand instanceof Expressible) {
+      expression.addParameter(rightOperand.toExpression(factory));
+    } else {
+      throw new IOException("This the right operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
new file mode 100644
index 0000000..609e4e1
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
@@ -0,0 +1,24 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import org.apache.solr.client.solrj.io.Tuple;
+
+
+public interface BooleanOperation extends StreamOperation {
+  public abstract boolean evaluate();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
new file mode 100644
index 0000000..1958551
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class EqualsOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public EqualsOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public EqualsOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() == val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
new file mode 100644
index 0000000..87c8364
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class GreaterThanEqualToOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public GreaterThanEqualToOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public GreaterThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() >= val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
new file mode 100644
index 0000000..664438a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class GreaterThanOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public GreaterThanOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public GreaterThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() > val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
new file mode 100644
index 0000000..b6ad897
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public abstract class LeafOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected String field;
+  protected Double val;
+  protected Tuple tuple;
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LeafOperation(String field, double val) {
+    this.field = field;
+    this.val = val;
+  }
+
+  public LeafOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    this.field = factory.getValueOperand(expression, 0);
+    this.val = Double.parseDouble(factory.getValueOperand(expression, 1));
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
new file mode 100644
index 0000000..2da3274
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LessThanEqualToOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LessThanEqualToOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public LessThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return true;
+    }
+
+    return d.doubleValue() <= val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
new file mode 100644
index 0000000..c1cec95
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LessThanOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LessThanOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public LessThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return true;
+    }
+    
+    return d.doubleValue() < val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
new file mode 100644
index 0000000..0e40b72
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
@@ -0,0 +1,87 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+
+public class NotOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected BooleanOperation operand;
+
+  public void operate(Tuple tuple) {
+    operand.operate(tuple);
+  }
+
+  public NotOperation(BooleanOperation operand) {
+    this.operand = operand;
+  }
+
+  public NotOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+    if(operationExpressions != null && operationExpressions.size() == 1) {
+      StreamExpression op = operationExpressions.get(0);
+      StreamOperation streamOp = factory.constructOperation(op);
+      if(streamOp instanceof BooleanOperation) {
+        operand = (BooleanOperation) streamOp;
+      } else {
+        throw new IOException("The NotOperation requires a BooleanOperation.");
+      }
+
+    } else {
+      throw new IOException("The NotOperation requires a BooleanOperations.");
+    }
+  }
+
+  public boolean evaluate() {
+    return !operand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(operand instanceof Expressible) {
+      expression.addParameter(operand.toExpression(factory));
+    } else {
+      throw new IOException("The operand of the NotOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
new file mode 100644
index 0000000..faac5cd
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
@@ -0,0 +1,71 @@
+/*
+ * 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.client.solrj.io.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class OrOperation extends AndOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public OrOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
+    super(leftOperand, rightOperand);
+  }
+
+  public OrOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    return leftOperand.evaluate() || rightOperand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(leftOperand instanceof Expressible) {
+      expression.addParameter(leftOperand.toExpression(factory));
+    } else {
+      throw new IOException("This left operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    if(rightOperand instanceof Expressible) {
+      expression.addParameter(rightOperand.toExpression(factory));
+    } else {
+      throw new IOException("This the right operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da283ef/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
new file mode 100644
index 0000000..36ca113
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -0,0 +1,190 @@
+/*
+ * 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.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.FieldComparator;
+import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.eq.FieldEqualitor;
+import org.apache.solr.client.solrj.io.eq.MultipleFieldEqualitor;
+import org.apache.solr.client.solrj.io.eq.StreamEqualitor;
+import org.apache.solr.client.solrj.io.ops.BooleanOperation;
+import org.apache.solr.client.solrj.io.ops.StreamOperation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ *  Iterates over a TupleStream and buffers Tuples that are equal based on a comparator.
+ *  This allows tuples to be grouped by common field(s).
+ *
+ *  The read() method emits one tuple per group. The fields of the emitted Tuple reflect the first tuple
+ *  encountered in the group.
+ *
+ *  Use the Tuple.getMaps() method to return all the Tuples in the group. This method returns
+ *  a list of maps (including the group head), which hold the data for each Tuple in the group.
+ *
+ *  Note: The ReducerStream requires that it's underlying stream be sorted and partitioned by the same
+ *  fields as it's comparator.
+ *
+ **/
+
+public class HavingStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  private TupleStream stream;
+  private BooleanOperation op;
+
+  private transient Tuple currentGroupHead;
+
+  public HavingStream(TupleStream stream, BooleanOperation op) throws IOException {
+    init(stream, op);
+  }
+
+
+  public HavingStream(StreamExpression expression, StreamFactory factory) throws IOException{
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+
+    // validate expression contains only what we want.
+    if(expression.getParameters().size() != streamExpressions.size() + 1){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
+    }
+
+    if(1 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+
+    BooleanOperation booleanOperation = null;
+    if(operationExpressions != null && operationExpressions.size() == 1) {
+      StreamExpression ex = operationExpressions.get(0);
+      StreamOperation operation = factory.constructOperation(ex);
+      if(operation instanceof BooleanOperation) {
+        booleanOperation = (BooleanOperation) operation;
+      } else {
+        throw new IOException("The HavingStream requires a BooleanOperation. A StreamOperation was provided.");
+      }
+    } else {
+      throw new IOException("The HavingStream requires a BooleanOperation.");
+    }
+
+    init(factory.constructStream(streamExpressions.get(0)), booleanOperation);
+  }
+
+  private void init(TupleStream stream, BooleanOperation op) throws IOException{
+    this.stream = stream;
+    this.op = op;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    // stream
+    if(includeStreams){
+      expression.addParameter(((Expressible) stream).toExpression(factory));
+    }
+    else{
+      expression.addParameter("<stream>");
+    }
+
+    if(op instanceof Expressible) {
+      expression.addParameter(op.toExpression(factory));
+    } else {
+      throw new IOException("This ReducerStream contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            stream.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString())
+        .withHelpers(new Explanation[]{
+            op.toExplanation(factory)
+        });
+  }
+
+  public void setStreamContext(StreamContext context) {
+    this.stream.setStreamContext(context);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    stream.open();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+    while(true) {
+      Tuple tuple = stream.read();
+      if(tuple.EOF) {
+        return tuple;
+      }
+
+      op.operate(tuple);
+
+      if(op.evaluate()) {
+        return tuple;
+      }
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+}
\ No newline at end of file


[03/25] lucene-solr:jira/solr-8593: SOLR-8530: Add tests from the HavingStream

Posted by kr...@apache.org.
SOLR-8530: Add tests from the HavingStream


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

Branch: refs/heads/jira/solr-8593
Commit: 5bbd4d6765d69d245131d049a2551c0534c1180d
Parents: 1da283e
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 16 11:26:55 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 15:05:28 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  19 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 201 +++++++++++++++++++
 2 files changed, 219 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5bbd4d67/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 661704f..e0f1186 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -31,9 +31,17 @@ import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
 import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
+import org.apache.solr.client.solrj.io.ops.AndOperation;
 import org.apache.solr.client.solrj.io.ops.ConcatOperation;
 import org.apache.solr.client.solrj.io.ops.DistinctOperation;
+import org.apache.solr.client.solrj.io.ops.EqualsOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanOperation;
+import org.apache.solr.client.solrj.io.ops.NotOperation;
+import org.apache.solr.client.solrj.io.ops.OrOperation;
 import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.*;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -154,7 +162,16 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       
       // stream reduction operations
       .withFunctionName("group", GroupOperation.class)
-      .withFunctionName("distinct", DistinctOperation.class);
+      .withFunctionName("distinct", DistinctOperation.class)
+      .withFunctionName("having", HavingStream.class)
+      .withFunctionName("and", AndOperation.class)
+      .withFunctionName("or", OrOperation.class)
+      .withFunctionName("not", NotOperation.class)
+      .withFunctionName("gt", GreaterThanOperation.class)
+      .withFunctionName("lt", LessThanOperation.class)
+      .withFunctionName("eq", EqualsOperation.class)
+      .withFunctionName("lteq", LessThanEqualToOperation.class)
+      .withFunctionName("gteq", GreaterThanEqualToOperation.class);
 
      // This pulls all the overrides and additions from the config
      List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5bbd4d67/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 1316af4..fd088f1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -33,8 +33,16 @@ import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
 import org.apache.solr.client.solrj.io.comp.FieldComparator;
+import org.apache.solr.client.solrj.io.ops.AndOperation;
 import org.apache.solr.client.solrj.io.ops.ConcatOperation;
+import org.apache.solr.client.solrj.io.ops.EqualsOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanOperation;
+import org.apache.solr.client.solrj.io.ops.NotOperation;
+import org.apache.solr.client.solrj.io.ops.OrOperation;
 import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
@@ -803,6 +811,199 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
+
+  @Test
+  public void testHavingStream() throws Exception {
+
+    SolrClientCache solrClientCache = new SolrClientCache();
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("having", HavingStream.class)
+        .withFunctionName("and", AndOperation.class)
+        .withFunctionName("or", OrOperation.class)
+        .withFunctionName("not", NotOperation.class)
+        .withFunctionName("gt", GreaterThanOperation.class)
+        .withFunctionName("lt", LessThanOperation.class)
+        .withFunctionName("eq", EqualsOperation.class)
+        .withFunctionName("lteq", LessThanEqualToOperation.class)
+        .withFunctionName("gteq", GreaterThanEqualToOperation.class);
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), eq(a_i, 9))");
+    StreamContext context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    Tuple t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(eq(a_i, 9),lt(a_i, 10)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), or(eq(a_i, 9),eq(a_i, 8)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 2);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(eq(a_i, 9),not(eq(a_i, 9))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 0);
+
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(lteq(a_i, 9), gteq(a_i, 8)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    System.out.println("####Tuples:"+tuples.size());
+    assert(tuples.size() == 2);
+
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+    solrClientCache.close();
+  }
+
+
+  @Test
+  public void testParallelHavingStream() throws Exception {
+
+    SolrClientCache solrClientCache = new SolrClientCache();
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("having", HavingStream.class)
+        .withFunctionName("and", AndOperation.class)
+        .withFunctionName("or", OrOperation.class)
+        .withFunctionName("not", NotOperation.class)
+        .withFunctionName("gt", GreaterThanOperation.class)
+        .withFunctionName("lt", LessThanOperation.class)
+        .withFunctionName("eq", EqualsOperation.class)
+        .withFunctionName("lteq", LessThanEqualToOperation.class)
+        .withFunctionName("gteq", GreaterThanEqualToOperation.class)
+        .withFunctionName("parallel", ParallelStream.class);
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), eq(a_i, 9)))");
+    StreamContext context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    Tuple t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(eq(a_i, 9),lt(a_i, 10))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\",having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), or(eq(a_i, 9),eq(a_i, 8))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 2);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(eq(a_i, 9),not(eq(a_i, 9)))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 0);
+
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\",having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(lteq(a_i, 9), gteq(a_i, 8))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    System.out.println("####Tuples:"+tuples.size());
+    assert(tuples.size() == 2);
+
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+    solrClientCache.close();
+  }
+
   @Test
   public void testFetchStream() throws Exception {
 


[23/25] lucene-solr:jira/solr-8593: SOLR-8530: Add support for aggregate HAVING comparisons without single quotes

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


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

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

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


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

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

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

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

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

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

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

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


[17/25] lucene-solr:jira/solr-8593: SOLR-9917: fix NPE in distrib percentiles when no values for field in bucket

Posted by kr...@apache.org.
SOLR-9917: fix NPE in distrib percentiles when no values for field in bucket


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

Branch: refs/heads/jira/solr-8593
Commit: 7ef6a8184682e046a6a8ce9166978c320d285a1a
Parents: f9e3554
Author: yonik <yo...@apache.org>
Authored: Wed Jan 4 23:53:07 2017 -0500
Committer: yonik <yo...@apache.org>
Committed: Wed Jan 4 23:53:07 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../apache/solr/search/facet/PercentileAgg.java |  7 ++--
 .../org/apache/solr/search/facet/SlotAcc.java   |  5 ++-
 .../solr/search/facet/TestJsonFacets.java       | 44 ++++++++++++++------
 .../java/org/apache/solr/SolrTestCaseHS.java    |  4 ++
 5 files changed, 46 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ef6a818/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e3a4b4b..f4dd133 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -329,6 +329,9 @@ Bug Fixes
 
 * SOLR-7495: Support Facet.field on a non-DocValued, single-value, int field (Varun Thacker, Scott Stults)
 
+* SOLR-9917: JSON Facet API percentile function caused a NullPointerException in distributed mode when
+  there were no values in a bucket from a shard. (yonik)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ef6a818/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java b/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
index 7f063b2..ea46a91 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/PercentileAgg.java
@@ -149,7 +149,7 @@ public class PercentileAgg extends SimpleAggValueSource {
       }
       if (sortvals != null && percentiles.size()==1) {
         // we've already calculated everything we need
-        return sortvals[slotNum];
+        return digests[slotNum] != null ? sortvals[slotNum] : null;
       }
       return getValueFromDigest( digests[slotNum] );
     }
@@ -192,6 +192,7 @@ public class PercentileAgg extends SimpleAggValueSource {
     @Override
     public void merge(Object facetResult, Context mcontext) {
       byte[] arr = (byte[])facetResult;
+      if (arr == null) return; // an explicit null can mean no values in the field
       AVLTreeDigest subDigest = AVLTreeDigest.fromBytes(ByteBuffer.wrap(arr));
       if (digest == null) {
         digest = subDigest;
@@ -202,7 +203,7 @@ public class PercentileAgg extends SimpleAggValueSource {
 
     @Override
     public Object getMergedResult() {
-      if (percentiles.size() == 1) return getSortVal();
+      if (percentiles.size() == 1 && digest != null) return getSortVal();
       return getValueFromDigest(digest);
     }
 
@@ -213,7 +214,7 @@ public class PercentileAgg extends SimpleAggValueSource {
 
     private Double getSortVal() {
       if (sortVal == null) {
-        sortVal = digest.quantile( percentiles.get(0) * 0.01 );
+        sortVal = digest==null ? Double.NEGATIVE_INFINITY : digest.quantile( percentiles.get(0) * 0.01 );
       }
       return sortVal;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ef6a818/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
index 4f4851b..3da3541 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
@@ -90,7 +90,10 @@ public abstract class SlotAcc implements Closeable {
 
   public void setValues(SimpleOrderedMap<Object> bucket, int slotNum) throws IOException {
     if (key == null) return;
-    bucket.add(key, getValue(slotNum));
+    Object val = getValue(slotNum);
+    if (val != null) {
+      bucket.add(key, val);
+    }
   }
 
   public abstract void reset();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ef6a818/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index 32f9dfa..d7e1cc0 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -381,6 +381,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
 
   public static void doStatsTemplated(Client client, ModifiableSolrParams p) throws Exception {
     p.set("Z_num_i", "Z_" + p.get("num_i") );
+    p.set("sparse_num_d", "sparse_" + p.get("num_d") );
     if (p.get("num_is") == null) p.add("num_is","num_is");
     if (p.get("num_fs") == null) p.add("num_fs","num_fs");
 
@@ -398,21 +399,34 @@ public class TestJsonFacets extends SolrTestCaseHS {
     String super_s = m.expand("${super_s}");
     String sparse_s = m.expand("${sparse_s}");
     String multi_ss = m.expand("${multi_ss}");
+    String sparse_num_d = m.expand("${sparse_num_d}");
+
 
     client.deleteByQuery("*:*", null);
 
+    Client iclient = client;
+
+    /*** This code was not needed yet, but may be needed if we want to force empty shard results more often.
+    // create a new indexing client that doesn't use one shard to better test for empty or non-existent results
+    if (!client.local()) {
+      List<SolrClient> shards = client.getClientProvider().all();
+      iclient = new Client(shards.subList(0, shards.size()-1), client.getClientProvider().getSeed());
+     }
+     ***/
+
     SolrInputDocument doc =
-               sdoc("id", "1", cat_s, "A", where_s, "NY", num_d, "4", num_i, "2",   num_is,"2",num_is,"-5", num_fs,"2",num_fs,"-5",   super_s, "zodiac", date, "2001-01-01T01:01:01Z", val_b, "true", sparse_s, "one");
-    client.add(doc, null);
-    client.add(doc, null);
-    client.add(doc, null);  // a couple of deleted docs
-    client.add(sdoc("id", "2", cat_s, "B", where_s, "NJ", num_d, "-9", num_i, "-5", num_is,"3",num_is,"-1", num_fs,"3",num_fs,"-1.5", super_s,"superman", date,"2002-02-02T02:02:02Z", val_b, "false"         , multi_ss,"a", multi_ss,"b" , Z_num_i, "0"), null);
-    client.add(sdoc("id", "3"), null);
-    client.commit();
-    client.add(sdoc("id", "4", cat_s, "A", where_s, "NJ", num_d, "2", num_i, "3",   num_is,"0",num_is,"3", num_fs,"0", num_fs,"3",   super_s,"spiderman", date,"2003-03-03T03:03:03Z"                         , multi_ss, "b", Z_num_i, ""+Integer.MIN_VALUE), null);
-    client.add(sdoc("id", "5", cat_s, "B", where_s, "NJ", num_d, "11", num_i, "7",  num_is,"0",            num_fs,"0",               super_s,"batman"   , date,"2001-02-03T01:02:03Z"          ,sparse_s,"two", multi_ss, "a"), null);
-    client.commit();
-    client.add(sdoc("id", "6", cat_s, "B", where_s, "NY", num_d, "-5", num_i, "-5", num_is,"-1",           num_fs,"-1.5",            super_s,"hulk"     , date,"2002-03-01T03:02:01Z"                         , multi_ss, "b", multi_ss, "a", Z_num_i, ""+Integer.MAX_VALUE), null);
+               sdoc("id", "1", cat_s, "A", where_s, "NY", num_d, "4", sparse_num_d, "6", num_i, "2",   num_is,"2",num_is,"-5", num_fs,"2",num_fs,"-5",   super_s, "zodiac", date, "2001-01-01T01:01:01Z", val_b, "true", sparse_s, "one");
+    iclient.add(doc, null);
+    iclient.add(doc, null);
+    iclient.add(doc, null);  // a couple of deleted docs
+    iclient.add(sdoc("id", "2", cat_s, "B", where_s, "NJ", num_d, "-9",                  num_i, "-5", num_is,"3",num_is,"-1", num_fs,"3",num_fs,"-1.5", super_s,"superman", date,"2002-02-02T02:02:02Z", val_b, "false"         , multi_ss,"a", multi_ss,"b" , Z_num_i, "0"), null);
+    iclient.add(sdoc("id", "3"), null);
+    iclient.commit();
+    iclient.add(sdoc("id", "4", cat_s, "A", where_s, "NJ", num_d, "2", sparse_num_d,"-4",num_i, "3",   num_is,"0",num_is,"3", num_fs,"0", num_fs,"3",   super_s,"spiderman", date,"2003-03-03T03:03:03Z"                         , multi_ss, "b", Z_num_i, ""+Integer.MIN_VALUE), null);
+    iclient.add(sdoc("id", "5", cat_s, "B", where_s, "NJ", num_d, "11",                  num_i, "7",  num_is,"0",            num_fs,"0",               super_s,"batman"   , date,"2001-02-03T01:02:03Z"          ,sparse_s,"two", multi_ss, "a"), null);
+    iclient.commit();
+    iclient.add(sdoc("id", "6", cat_s, "B", where_s, "NY", num_d, "-5",                  num_i, "-5", num_is,"-1",           num_fs,"-1.5",            super_s,"hulk"     , date,"2002-03-01T03:02:01Z"                         , multi_ss, "b", multi_ss, "a", Z_num_i, ""+Integer.MAX_VALUE), null);
+    iclient.commit();
     client.commit();
 
     // test for presence of debugging info
@@ -542,11 +556,15 @@ public class TestJsonFacets extends SolrTestCaseHS {
     // test sorting by single percentile
     client.testJQ(params(p, "q", "*:*"
             , "json.facet", "{f1:{terms:{${terms} field:'${cat_s}', sort:'n1 desc', facet:{n1:'percentile(${num_d},50)'}  }}" +
-                " , f2:{terms:{${terms} field:'${cat_s}', sort:'n1 asc', facet:{n1:'percentile(${num_d},50)'}  }} }"
+                " , f2:{terms:{${terms} field:'${cat_s}', sort:'n1 asc', facet:{n1:'percentile(${num_d},50)'}  }} " +
+                " , f3:{terms:{${terms} field:'${cat_s}', sort:'n1 desc', facet:{n1:'percentile(${sparse_num_d},50)'}  }} " +
+            "}"
         )
         , "facets=={ 'count':6, " +
             "  f1:{  'buckets':[{ val:'A', count:2, n1:3.0 }, { val:'B', count:3, n1:-5.0}]}" +
-            ", f2:{  'buckets':[{ val:'B', count:3, n1:-5.0}, { val:'A', count:2, n1:3.0 }]} }"
+            ", f2:{  'buckets':[{ val:'B', count:3, n1:-5.0}, { val:'A', count:2, n1:3.0 }]}" +
+            ", f3:{  'buckets':[{ val:'A', count:2, n1:1.0}, { val:'B', count:3}]}" +
+            "}"
     );
 
     // test sorting by multiple percentiles (sort is by first)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ef6a818/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
index a27fbf2..2da0c84 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
@@ -364,6 +364,10 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
     public List<SolrClient> all() {
       return clients;
     }
+
+    public int getSeed() {
+      return hashSeed;
+    }
   }
 
 


[18/25] lucene-solr:jira/solr-8593: SOLR-9923: Remove solr.http metric group and merge its metrics to solr.node group

Posted by kr...@apache.org.
SOLR-9923: Remove solr.http metric group and merge its metrics to solr.node group


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

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

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


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

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

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

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

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

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

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


[09/25] lucene-solr:jira/solr-8593: SOLR-9911: Fix typo in CHANGES.txt

Posted by kr...@apache.org.
SOLR-9911: Fix typo in CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: f87efac521457bf9ddbc784e64faebe49d910803
Parents: 0452cb8
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed Jan 4 20:18:18 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Wed Jan 4 20:18:18 2017 +0530

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f87efac5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 82fd7ae..a499cc8 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -190,7 +190,7 @@ New Features
   API supports three optional parameters:
   * 'group' (all,jvm,jetty,http,node,core),
   * 'type' (all,counter,timer,gauge,histogram) both of which are multi-valued
-  * 'prefix' that filters the return metrics
+  * 'prefix' that filters the returned metrics
   Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
   Example: http://localhost:8983/solr/admin/metrics?group=jvm&prefix=buffers
   (shalin)


[10/25] lucene-solr:jira/solr-8593: SOLR-7466: reverse-aware leading wildcards in complexphrase query parser

Posted by kr...@apache.org.
SOLR-7466: reverse-aware leading wildcards in complexphrase query parser


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

Branch: refs/heads/jira/solr-8593
Commit: d3f83bb948fd44e66099ef9537363ecef5bdb0f3
Parents: f87efac
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Fri Dec 30 00:01:20 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Wed Jan 4 18:11:02 2017 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   5 +
 .../apache/solr/parser/SolrQueryParserBase.java |  26 ++---
 .../solr/search/ComplexPhraseQParserPlugin.java |  70 +++++++++++-
 .../TestComplexPhraseLeadingWildcard.java       | 113 +++++++++++++++++++
 4 files changed, 196 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3f83bb9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a499cc8..ff1fdc7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -119,6 +119,8 @@ Upgrade Notes
   risk in overlapping commits.  Nonetheless users should continue to avoid excessive committing. Users are
   advised to remove any pre-existing maxWarmingSearchers entries from their solrconfig.xml files.
 
+* SOLR-7466: complexphrase query parser now supports leading wildcards, beware of its' possible heaviness. 
+  Users are encouraged to use ReversedWildcardFilter in index time analysis.    
 
 New Features
 ----------------------
@@ -220,6 +222,9 @@ New Features
 
 * SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions (Joel Bernstein)
 
+* SOLR-7466: Enable leading wildcard in complexphrase query parser, optimize it with  ReversedWildcardFilterFactory 
+  when it's provided (Mikhail Khludnev)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3f83bb9/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
index f54e9e9..84ffcb9 100644
--- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
+++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
@@ -63,6 +63,8 @@ import org.apache.solr.search.SyntaxError;
  */
 public abstract class SolrQueryParserBase extends QueryBuilder {
 
+  protected static final String REVERSE_WILDCARD_LOWER_BOUND = new String(new char[]{ReverseStringFilter.START_OF_HEADING_MARKER + 1});
+
   public static final int TERMS_QUERY_THRESHOLD = 16;   // @lucene.internal Set to a low value temporarily for better test coverage
 
   static final int CONJ_NONE   = 0;
@@ -889,28 +891,24 @@ public abstract class SolrQueryParserBase extends QueryBuilder {
     return newFieldQuery(getAnalyzer(), field, queryText, quoted);
   }
 
+ protected boolean isRangeShouldBeProtectedFromReverse(String field, String part1){
+   checkNullField(field);
+   SchemaField sf = schema.getField(field);
 
+   return part1 == null && getReversedWildcardFilterFactory(sf.getType())!=null;
+ }
 
   // called from parser
   protected Query getRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) throws SyntaxError {
+    boolean reverse = isRangeShouldBeProtectedFromReverse(field, part1);
+    return getRangeQueryImpl(field, reverse ? REVERSE_WILDCARD_LOWER_BOUND : part1, part2, startInclusive || reverse, endInclusive);
+  }
+
+  protected Query getRangeQueryImpl(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) throws SyntaxError {
     checkNullField(field);
     SchemaField sf = schema.getField(field);
-
-    if (part1 == null) {
-      ReversedWildcardFilterFactory factory = getReversedWildcardFilterFactory(sf.getType());
-      if (factory != null) {
-        // There will be reversed tokens starting with u0001 that we want to exclude, so
-        // lets start at u0002 inclusive instead.
-        char[] buf = new char[1];
-        buf[0] = ReverseStringFilter.START_OF_HEADING_MARKER + 1;
-        part1 = new String(buf);
-        startInclusive = true;
-      }
-    }
-
     return sf.getType().getRangeQuery(parser, sf, part1, part2, startInclusive, endInclusive);
   }
-
   // called from parser
   protected Query getPrefixQuery(String field, String termStr) throws SyntaxError {
     checkNullField(field);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3f83bb9/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java
index 2904de9..22702dc 100644
--- a/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java
@@ -18,6 +18,7 @@ package org.apache.solr.search;
 
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.queryparser.complexPhrase.ComplexPhraseQueryParser;
+import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -60,6 +61,32 @@ public class ComplexPhraseQParserPlugin extends QParserPlugin {
    */
   class ComplexPhraseQParser extends QParser {
 
+    final class SolrQueryParserDelegate extends SolrQueryParser {
+      private SolrQueryParserDelegate(QParser parser, String defaultField) {
+        super(parser, defaultField);
+      }
+
+      @Override
+      protected org.apache.lucene.search.Query getWildcardQuery(String field, String termStr) throws SyntaxError {
+        return super.getWildcardQuery(field, termStr);
+      }
+      
+      @Override
+      protected org.apache.lucene.search.Query getRangeQuery(String field, String part1, String part2,
+          boolean startInclusive, boolean endInclusive) throws SyntaxError {
+        return super.getRangeQuery(field, part1, part2, startInclusive, endInclusive);
+      }
+      
+      @Override
+      protected boolean isRangeShouldBeProtectedFromReverse(String field, String part1) {
+        return super.isRangeShouldBeProtectedFromReverse(field, part1);
+      }
+
+      public String getLowerBoundForReverse() {
+        return REVERSE_WILDCARD_LOWER_BOUND;
+      }
+    }
+
     ComplexPhraseQueryParser lparser;
 
     boolean inOrder = true;
@@ -87,11 +114,46 @@ public class ComplexPhraseQParserPlugin extends QParserPlugin {
         defaultField = getReq().getSchema().getDefaultSearchFieldName();
       }
 
-      lparser = new ComplexPhraseQueryParser(defaultField, getReq().getSchema().getQueryAnalyzer());
-
-      if (localParams != null)
+      SolrQueryParserDelegate reverseAwareParser = new SolrQueryParserDelegate(this, defaultField);
+      
+      lparser = new ComplexPhraseQueryParser(defaultField, getReq().getSchema().getQueryAnalyzer())
+          {
+              protected Query newWildcardQuery(org.apache.lucene.index.Term t) {
+                try {
+                  org.apache.lucene.search.Query wildcardQuery = reverseAwareParser.getWildcardQuery(t.field(), t.text());
+                  setRewriteMethod(wildcardQuery);
+                  return wildcardQuery;
+                } catch (SyntaxError e) {
+                  throw new RuntimeException(e);
+                }
+              }
+
+              private Query setRewriteMethod(org.apache.lucene.search.Query query) {
+                if (query instanceof MultiTermQuery) {
+                  ((MultiTermQuery) query).setRewriteMethod(
+                      org.apache.lucene.search.MultiTermQuery.SCORING_BOOLEAN_REWRITE);
+                }
+                return query;
+              }
+              
+              protected Query newRangeQuery(String field, String part1, String part2, boolean startInclusive,
+                  boolean endInclusive) {
+                boolean reverse = reverseAwareParser.isRangeShouldBeProtectedFromReverse(field, part1);
+                return super.newRangeQuery(field, 
+                                            reverse ? reverseAwareParser.getLowerBoundForReverse() : part1, 
+                                            part2,
+                                            startInclusive || reverse, 
+                                            endInclusive);
+              }
+          }
+          ;
+
+      lparser.setAllowLeadingWildcard(true);
+          
+      if (localParams != null) {
         inOrder = localParams.getBool("inOrder", inOrder);
-
+      }
+      
       lparser.setInOrder(inOrder);
 
       QueryParser.Operator defaultOperator = QueryParsing.getQueryParserDefaultOperator(getReq().getSchema(), getParam(QueryParsing.OP));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d3f83bb9/solr/core/src/test/org/apache/solr/search/TestComplexPhraseLeadingWildcard.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestComplexPhraseLeadingWildcard.java b/solr/core/src/test/org/apache/solr/search/TestComplexPhraseLeadingWildcard.java
new file mode 100644
index 0000000..6c48cc3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestComplexPhraseLeadingWildcard.java
@@ -0,0 +1,113 @@
+/*
+ * 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;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestComplexPhraseLeadingWildcard extends SolrTestCaseJ4 { 
+
+  private static final String noReverseText = "three";
+  private static final String withOriginal = "one";
+  private static final String withoutOriginal = "two";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml","schema-reversed.xml");
+    assertU(doc123(1, "one ever"));
+    assertU(doc123(2, "once forever"));
+                      
+    assertU(doc123(7, "once slope forever"));
+    assertU(doc123(8, "once again slope forever"));
+    assertU(doc123(9, "forever once"));
+    assertU(commit());
+  }
+  
+  @Test
+  public void testReverseWithOriginal() throws Exception {
+    checkField(withOriginal);
+    
+  }
+
+  // prefix query won't match without original tokens
+  @Test
+  public void testReverseWithoutOriginal() throws Exception {
+    assertQ( "prefix query doesn't work without original term",
+        req("q","{!complexphrase inOrder=true}\"on* for*\"",
+            "df",withoutOriginal),
+        expect());
+    
+    assertQ("postfix query works fine even without original",
+        req("q","{!complexphrase inOrder=true}\"*nce *ver\"",
+            "df",withoutOriginal),
+        expect("2"));
+  }
+  
+  @Test
+  public void testWithoutReverse() throws Exception {
+    checkField(noReverseText);
+  }
+
+  private void checkField(String field) {
+    assertQ(
+        req("q","{!complexphrase inOrder=true}\"on* *ver\"",
+            "df",field,
+            "indent","on",
+            "debugQuery", "true"),
+        expect("1","2"));
+    
+    assertQ(
+        req("q","{!complexphrase inOrder=true}\"ON* *VER\"",
+            "df",field), 
+        expect("1","2"));
+    
+    assertQ(
+        req("q","{!complexphrase inOrder=true}\"ON* *ver\"",
+            "df",field), 
+        expect("1","2"));
+    
+    assertQ(
+        req("q","{!complexphrase inOrder=true}\"on* *ver\"~1",
+            "df",field),
+        expect("1","2","7"));
+    
+    assertQ("range works if reverse doesn't mess",
+        req("q","{!complexphrase inOrder=true}\"on* [* TO a]\"",
+            "df",field),
+        expect());
+
+    assertQ("range works if reverse doesn't mess",
+        req("q","{!complexphrase inOrder=true}\"[on TO onZ] for*\"",
+            "df",field),
+        expect("2"));
+  } 
+  
+  private static String doc123(int id, String text){
+    return adoc("id",""+id, withOriginal, text, withoutOriginal, text, noReverseText, text);
+  }
+  
+  private static String [] expect(String ...ids) {
+    String[] xpathes = new String[ids.length+1];
+    xpathes[0]= "//result[@numFound=" +ids.length+ "]";
+    int i=1;
+    for(String id : ids) {
+      xpathes[i++] = "//doc/int[@name='id' and text()='"+id+"']";
+    }
+    return xpathes;
+  }
+}


[07/25] lucene-solr:jira/solr-8593: SOLR-9921 Separate SolrMetricProducer from SolrInfoMBean.

Posted by kr...@apache.org.
SOLR-9921 Separate SolrMetricProducer from SolrInfoMBean.


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

Branch: refs/heads/jira/solr-8593
Commit: 12d849221727d3294429397a456adebf583defbc
Parents: ccdbb6a
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Wed Jan 4 15:31:31 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Wed Jan 4 15:31:31 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../apache/solr/handler/RequestHandlerBase.java |  5 +--
 .../solr/handler/admin/CoreAdminHandler.java    |  6 +--
 .../component/HttpShardHandlerFactory.java      | 46 ++-----------------
 .../solr/metrics/SolrCoreMetricManager.java     |  9 +---
 .../apache/solr/metrics/SolrMetricProducer.java | 12 ++---
 .../apache/solr/update/UpdateShardHandler.java  | 14 +++---
 .../stats/InstrumentedHttpRequestExecutor.java  | 42 +----------------
 ...entedPoolingHttpClientConnectionManager.java | 43 +-----------------
 .../solr/metrics/SolrCoreMetricManagerTest.java |  7 ++-
 .../solr/metrics/SolrMetricTestUtils.java       | 47 +++-----------------
 11 files changed, 28 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3281158..1ad6359 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -178,8 +178,8 @@ New Features
 
 * SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder (Hrishikesh Gadre, Ishan Chattopadhyaya)
  
-* SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
-  for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
+* SOLR-4735, SOLR-9921: Improve metrics reporting. This uses the dropwizard metric library, adding an internal
+  API for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
   reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)
 
 * SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 85597dc..1fd6c30 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -18,8 +18,6 @@ package org.apache.solr.handler;
 
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
-import java.util.Arrays;
-import java.util.Collection;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
@@ -133,14 +131,13 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   }
 
   @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
     numErrors = manager.meter(registryName, "errors", getCategory().toString(), scope);
     numServerErrors = manager.meter(registryName, "serverErrors", getCategory().toString(), scope);
     numClientErrors = manager.meter(registryName, "clientErrors", getCategory().toString(), scope);
     numTimeouts = manager.meter(registryName, "timeouts", getCategory().toString(), scope);
     requests = manager.counter(registryName, "requests", getCategory().toString(), scope);
     requestTimes = manager.timer(registryName, "requestTimes", getCategory().toString(), scope);
-    return Arrays.asList("errors", "serverErrors", "clientErrors", "timeouts", "requestTimes", "requests");
   }
 
   public static SolrParams getSolrParamsFromNamedList(NamedList args, String key) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index 458b7a5..05a1671 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -18,7 +18,6 @@ package org.apache.solr.handler.admin;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -115,11 +114,10 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   }
 
   @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
-    Collection<String> metrics = super.initializeMetrics(manager, registryName, scope);
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    super.initializeMetrics(manager, registryName, scope);
     parallelExecutor = MetricUtils.instrumentedExecutorService(parallelExecutor, manager.registry(registryName),
         SolrMetricManager.mkName("parallelCoreAdminExecutor", getCategory().name(),scope, "threadPool"));
-    return metrics;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index d190ce0..14af88a 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -48,9 +48,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
@@ -365,48 +362,11 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   }
 
   @Override
-  public String getName() {
-    return this.getClass().getName();
-  }
-
-  @Override
-  public String getVersion() {
-    return getClass().getPackage().getSpecificationVersion();
-  }
-
-  @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    List<String> metricNames = new ArrayList<>(4);
-    metricNames.addAll(clientConnectionManager.initializeMetrics(manager, registry, scope));
-    metricNames.addAll(httpRequestExecutor.initializeMetrics(manager, registry, scope));
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+    clientConnectionManager.initializeMetrics(manager, registry, scope);
+    httpRequestExecutor.initializeMetrics(manager, registry, scope);
     commExecutor = MetricUtils.instrumentedExecutorService(commExecutor,
         manager.registry(registry),
         SolrMetricManager.mkName("httpShardExecutor", scope, "threadPool"));
-    return metricNames;
-  }
-
-  @Override
-  public String getDescription() {
-    return "Metrics tracked by HttpShardHandlerFactory for distributed query requests";
-  }
-
-  @Override
-  public Category getCategory() {
-    return Category.OTHER;
-  }
-
-  @Override
-  public String getSource() {
-    return null;
-  }
-
-  @Override
-  public URL[] getDocs() {
-    return new URL[0];
-  }
-
-  @Override
-  public NamedList getStatistics() {
-    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index 0e5403b..e7ca9c4 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -18,7 +18,6 @@ package org.apache.solr.metrics;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Collection;
 import java.lang.invoke.MethodHandles;
 
 import org.apache.solr.core.NodeConfig;
@@ -88,15 +87,11 @@ public class SolrCoreMetricManager implements Closeable {
    * @param producer  producer of metrics to be registered
    */
   public void registerMetricProducer(String scope, SolrMetricProducer producer) {
-    if (scope == null || producer == null || producer.getCategory() == null) {
+    if (scope == null || producer == null) {
       throw new IllegalArgumentException("registerMetricProducer() called with illegal arguments: " +
           "scope = " + scope + ", producer = " + producer);
     }
-    Collection<String> registered = producer.initializeMetrics(metricManager, getRegistryName(), scope);
-    if (registered == null || registered.isEmpty()) {
-      throw new IllegalArgumentException("registerMetricProducer() did not register any metrics " +
-      "for scope = " + scope + ", producer = " + producer);
-    }
+    producer.initializeMetrics(metricManager, getRegistryName(), scope);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
index 472931f..73abab1 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -16,15 +16,10 @@
  */
 package org.apache.solr.metrics;
 
-import java.util.Collection;
-
-import org.apache.solr.core.SolrInfoMBean;
-
 /**
- * Extension of {@link SolrInfoMBean} for use by objects that
- * expose metrics through {@link SolrCoreMetricManager}.
+ * Used by objects that expose metrics through {@link SolrCoreMetricManager}.
  */
-public interface SolrMetricProducer extends SolrInfoMBean {
+public interface SolrMetricProducer {
 
   /**
    * Initializes metrics specific to this producer
@@ -32,7 +27,6 @@ public interface SolrMetricProducer extends SolrInfoMBean {
    * @param registry registry name where metrics are registered
    * @param scope scope of the metrics (eg. handler name) to separate metrics of
    *              instances of the same component executing in different contexts
-   * @return registered (or existing) unqualified names of metrics specific to this producer.
    */
-  Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope);
+  void initializeMetrics(SolrMetricManager manager, String registry, String scope);
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index 9d230bc..cc13f3e 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -18,9 +18,6 @@ package org.apache.solr.update;
 
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
 import java.util.concurrent.ExecutorService;
 
 import com.codahale.metrics.InstrumentedExecutorService;
@@ -34,6 +31,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
@@ -41,7 +39,7 @@ import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class UpdateShardHandler implements SolrMetricProducer {
+public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
   
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -99,17 +97,15 @@ public class UpdateShardHandler implements SolrMetricProducer {
   }
 
   @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    List<String> metricNames = new ArrayList<>(4);
-    metricNames.addAll(clientConnectionManager.initializeMetrics(manager, registry, scope));
-    metricNames.addAll(httpRequestExecutor.initializeMetrics(manager, registry, scope));
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+    clientConnectionManager.initializeMetrics(manager, registry, scope);
+    httpRequestExecutor.initializeMetrics(manager, registry, scope);
     updateExecutor = new InstrumentedExecutorService(updateExecutor,
         manager.registry(registry),
         SolrMetricManager.mkName("updateExecutor", scope, "threadPool"));
     recoveryExecutor = new InstrumentedExecutorService(recoveryExecutor,
         manager.registry(registry),
         SolrMetricManager.mkName("recoveryExecutor", scope, "threadPool"));
-    return metricNames;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index 0426780..6f3f759 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -19,9 +19,6 @@ package org.apache.solr.util.stats;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.Locale;
 
 import com.codahale.metrics.MetricRegistry;
@@ -35,7 +32,6 @@ import org.apache.http.client.methods.HttpRequestWrapper;
 import org.apache.http.client.utils.URIBuilder;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.protocol.HttpRequestExecutor;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 
@@ -71,45 +67,9 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   }
 
   @Override
-  public String getName() {
-    return this.getClass().getName();
-  }
-
-  @Override
-  public String getVersion() {
-    return getClass().getPackage().getSpecificationVersion();
-  }
-
-  @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
     this.metricsRegistry = manager.registry(registry);
     this.scope = scope;
-    return Collections.emptyList(); // we do not know the names of the metrics yet
-  }
-
-  @Override
-  public String getDescription() {
-    return null;
-  }
-
-  @Override
-  public Category getCategory() {
-    return Category.OTHER;
-  }
-
-  @Override
-  public String getSource() {
-    return null;
-  }
-
-  @Override
-  public URL[] getDocs() {
-    return null;
-  }
-
-  @Override
-  public NamedList getStatistics() {
-    return null;
   }
 
   private String getNameFor(HttpRequest request) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java
index 08b68cb..6c992fa 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java
@@ -17,16 +17,11 @@
 
 package org.apache.solr.util.stats;
 
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collection;
-
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.http.config.Registry;
 import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 
@@ -51,17 +46,7 @@ public class InstrumentedPoolingHttpClientConnectionManager extends PoolingHttpC
   }
 
   @Override
-  public String getName() {
-    return this.getClass().getName();
-  }
-
-  @Override
-  public String getVersion() {
-    return getClass().getPackage().getSpecificationVersion();
-  }
-
-  @Override
-  public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
     this.metricsRegistry = manager.registry(registry);
     metricsRegistry.register(SolrMetricManager.mkName("availableConnections", scope),
         (Gauge<Integer>) () -> {
@@ -83,31 +68,5 @@ public class InstrumentedPoolingHttpClientConnectionManager extends PoolingHttpC
           // this acquires a lock on the connection pool; remove if contention sucks
           return getTotalStats().getPending();
         });
-    return Arrays.asList("availableConnections", "leasedConnections", "maxConnections", "pendingConnections");
-  }
-
-  @Override
-  public String getDescription() {
-    return "";
-  }
-
-  @Override
-  public Category getCategory() {
-    return Category.OTHER;
-  }
-
-  @Override
-  public String getSource() {
-    return null;
-  }
-
-  @Override
-  public URL[] getDocs() {
-    return null;
-  }
-
-  @Override
-  public NamedList getStatistics() {
-    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
index 65ffb93..1df6021 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -68,11 +68,10 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
       coreMetricManager.registerMetricProducer(scope, producer);
       assertNotNull(scope);
       assertNotNull(category);
-      assertNotNull(metrics);
       assertRegistered(scope, metrics, coreMetricManager);
     } catch (final IllegalArgumentException e) {
-      assertTrue("expected at least one null but got: scope="+scope+" category="+category+" metrics="+metrics,
-          (scope == null || category == null || metrics == null));
+      assertTrue("expected at least one null but got: scope="+scope+", category="+category,
+          (scope == null || category == null));
       assertRegistered(scope, new HashMap<>(), coreMetricManager);
     }
   }
@@ -130,7 +129,7 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
   }
 
   private void assertRegistered(String scope, Map<String, Counter> newMetrics, SolrCoreMetricManager coreMetricManager) {
-    if (scope == null) {
+    if (scope == null || newMetrics == null) {
       return;
     }
     String filter = "." + scope + ".";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12d84922/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
index 44ae9db..6bd6500 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
@@ -16,17 +16,13 @@
  */
 package org.apache.solr.metrics;
 
-import java.net.URL;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
 import com.codahale.metrics.Counter;
 import org.apache.lucene.util.TestUtil;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrInfoMBean;
 
 public final class SolrMetricTestUtils {
@@ -82,49 +78,16 @@ public final class SolrMetricTestUtils {
   public static SolrMetricProducer getProducerOf(SolrMetricManager metricManager, SolrInfoMBean.Category category, String scope, Map<String, Counter> metrics) {
     return new SolrMetricProducer() {
       @Override
-      public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+      public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+        if (category == null) {
+          throw new IllegalArgumentException("null category");
+        }
         if (metrics == null || metrics.isEmpty()) {
-          return Collections.emptyList();
+          return;
         }
         for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
           manager.counter(registry, entry.getKey(), category.toString(), scope);
         }
-        return metrics.keySet();
-      }
-
-      @Override
-      public String getName() {
-        return scope;
-      }
-
-      @Override
-      public String getVersion() {
-        return "0.0";
-      }
-
-      @Override
-      public String getDescription() {
-        return "foo";
-      }
-
-      @Override
-      public Category getCategory() {
-        return category;
-      }
-
-      @Override
-      public String getSource() {
-        return null;
-      }
-
-      @Override
-      public URL[] getDocs() {
-        return new URL[0];
-      }
-
-      @Override
-      public NamedList getStatistics() {
-        return null;
       }
 
       @Override


[25/25] lucene-solr:jira/solr-8593: Merge branch 'apache-https-master' into jira/solr-8593

Posted by kr...@apache.org.
Merge branch 'apache-https-master' into jira/solr-8593


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

Branch: refs/heads/jira/solr-8593
Commit: 3793eb5ec637f110e1be2acb94169be16ef549cc
Parents: 5f65487 cd4f908
Author: Kevin Risden <kr...@apache.org>
Authored: Fri Jan 6 15:47:02 2017 -0600
Committer: Kevin Risden <kr...@apache.org>
Committed: Fri Jan 6 15:47:02 2017 -0600

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  40 +-
 solr/contrib/ltr/README.md                      | 390 +------------------
 solr/contrib/ltr/example/README.md              |  60 +++
 solr/contrib/ltr/example/config.json            |  13 +-
 solr/contrib/ltr/example/exampleFeatures.json   |  26 ++
 solr/contrib/ltr/example/libsvm_formatter.py    |  12 +-
 .../ltr/example/techproducts-features.json      |  26 --
 .../contrib/ltr/example/techproducts-model.json |  18 -
 .../ltr/example/train_and_upload_demo_model.py  | 143 ++++---
 .../ltr/store/rest/ManagedFeatureStore.java     |   3 -
 .../solr/ltr/store/rest/ManagedModelStore.java  |   6 +-
 .../multipleadditivetreesmodel.json             |   2 +-
 .../org/apache/solr/ltr/TestRerankBase.java     |   9 +-
 .../model/TestMultipleAdditiveTreesModel.java   |   9 +-
 .../solr/ltr/store/rest/TestModelManager.java   |  47 ++-
 .../store/rest/TestModelManagerPersistence.java |   4 +-
 .../apache/solr/cloud/rule/ReplicaAssigner.java |  31 +-
 .../java/org/apache/solr/cloud/rule/Rule.java   |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |   5 +-
 .../solr/core/MetricsDirectoryFactory.java      |   6 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  20 +
 .../org/apache/solr/core/SolrInfoMBean.java     |   4 +-
 .../org/apache/solr/handler/GraphHandler.java   |   3 +-
 .../apache/solr/handler/ReplicationHandler.java |  20 +-
 .../apache/solr/handler/RequestHandlerBase.java |   5 +-
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../solr/handler/admin/CoreAdminHandler.java    |   6 +-
 .../solr/handler/admin/MetricsHandler.java      |  16 +-
 .../component/HttpShardHandlerFactory.java      |  50 +--
 .../org/apache/solr/handler/sql/SolrSchema.java |   8 +-
 .../solr/metrics/SolrCoreMetricManager.java     |   9 +-
 .../apache/solr/metrics/SolrMetricProducer.java |  12 +-
 .../apache/solr/parser/SolrQueryParserBase.java |  26 +-
 .../request/PerSegmentSingleValuedFaceting.java |   9 +-
 .../org/apache/solr/request/SimpleFacets.java   |  37 +-
 .../solr/search/ComplexPhraseQParserPlugin.java |  70 +++-
 .../org/apache/solr/search/facet/HLLAgg.java    |  13 +-
 .../apache/solr/search/facet/PercentileAgg.java |   7 +-
 .../org/apache/solr/search/facet/SlotAcc.java   |   5 +-
 .../apache/solr/search/facet/UniqueSlotAcc.java |   2 +-
 .../java/org/apache/solr/update/PeerSync.java   |   8 +-
 .../apache/solr/update/UpdateShardHandler.java  |  21 +-
 .../stats/InstrumentedHttpRequestExecutor.java  |  48 +--
 ...entedPoolingHttpClientConnectionManager.java |  43 +-
 .../org/apache/solr/util/stats/MetricUtils.java |  13 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |   2 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |  36 +-
 .../solr/metrics/SolrCoreMetricManagerTest.java |   7 +-
 .../solr/metrics/SolrMetricTestUtils.java       |  47 +--
 .../apache/solr/request/SimpleFacetsTest.java   |  12 +-
 .../TestComplexPhraseLeadingWildcard.java       | 113 ++++++
 .../solr/search/facet/TestJsonFacets.java       |  71 +++-
 .../client/solrj/io/ops/BooleanOperation.java   |   6 +-
 .../solr/client/solrj/io/ops/LeafOperation.java |   8 +
 .../client/solrj/io/stream/HavingStream.java    |  21 +-
 .../solrj/io/stream/expr/StreamFactory.java     |   2 +
 .../solrj/io/stream/StreamExpressionTest.java   |  36 +-
 .../java/org/apache/solr/SolrTestCaseHS.java    |   4 +
 58 files changed, 779 insertions(+), 896 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3793eb5e/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
index 221ddf8,0000000..aecfd42
mode 100644,000000..100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
@@@ -1,120 -1,0 +1,124 @@@
 +/*
 + * 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.handler.sql;
 +
 +import com.google.common.collect.ImmutableMap;
 +import org.apache.calcite.rel.type.*;
 +import org.apache.calcite.schema.Table;
 +import org.apache.calcite.schema.impl.AbstractSchema;
 +import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 +import org.apache.solr.client.solrj.SolrServerException;
 +import org.apache.solr.client.solrj.impl.CloudSolrClient;
 +import org.apache.solr.client.solrj.request.LukeRequest;
 +import org.apache.solr.client.solrj.response.LukeResponse;
++import org.apache.solr.common.cloud.Aliases;
 +import org.apache.solr.common.cloud.ClusterState;
 +import org.apache.solr.common.cloud.ZkStateReader;
 +import org.apache.solr.common.luke.FieldFlag;
 +
 +import java.io.IOException;
 +import java.util.EnumSet;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +
 +class SolrSchema extends AbstractSchema {
 +  final Properties properties;
 +
 +  SolrSchema(Properties properties) {
 +    super();
 +    this.properties = properties;
 +  }
 +
 +  @Override
 +  protected Map<String, Table> getTableMap() {
 +    String zk = this.properties.getProperty("zk");
 +    try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zk).build()) {
 +      cloudSolrClient.connect();
 +      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
 +      ClusterState clusterState = zkStateReader.getClusterState();
 +
 +      final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
 +
 +      for (String collection : clusterState.getCollectionsMap().keySet()) {
 +        builder.put(collection, new SolrTable(this, collection));
 +      }
 +
-       for (Map.Entry<String, String> alias : zkStateReader.getAliases().getCollectionAliasMap().entrySet()) {
-         builder.put(alias.getKey(), new SolrTable(this, alias.getValue()));
++      Aliases aliases = zkStateReader.getAliases();
++      if(aliases.collectionAliasSize() > 0) {
++        for (Map.Entry<String, String> alias : aliases.getCollectionAliasMap().entrySet()) {
++          builder.put(alias.getKey(), new SolrTable(this, alias.getValue()));
++        }
 +      }
 +
 +      return builder.build();
 +    } catch (IOException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  private Map<String, LukeResponse.FieldInfo> getFieldInfo(String collection) {
 +    String zk = this.properties.getProperty("zk");
 +    try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zk).build()) {
 +      cloudSolrClient.connect();
 +      LukeRequest lukeRequest = new LukeRequest();
 +      lukeRequest.setNumTerms(0);
 +      LukeResponse lukeResponse = lukeRequest.process(cloudSolrClient, collection);
 +      return lukeResponse.getFieldInfo();
 +    } catch (SolrServerException | IOException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  RelProtoDataType getRelDataType(String collection) {
 +    // Temporary type factory, just for the duration of this method. Allowable
 +    // because we're creating a proto-type, not a type; before being used, the
 +    // proto-type will be copied into a real type factory.
 +    final RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
 +    final RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
 +    Map<String, LukeResponse.FieldInfo> luceneFieldInfoMap = getFieldInfo(collection);
 +    for(Map.Entry<String, LukeResponse.FieldInfo> entry : luceneFieldInfoMap.entrySet()) {
 +      LukeResponse.FieldInfo luceneFieldInfo = entry.getValue();
 +
 +      RelDataType type;
 +      switch (luceneFieldInfo.getType()) {
 +        case "string":
 +          type = typeFactory.createJavaType(String.class);
 +          break;
 +        case "int":
 +        case "long":
 +          type = typeFactory.createJavaType(Long.class);
 +          break;
 +        case "float":
 +        case "double":
 +          type = typeFactory.createJavaType(Double.class);
 +          break;
 +        default:
 +          type = typeFactory.createJavaType(String.class);
 +      }
 +
 +      EnumSet<FieldFlag> flags = luceneFieldInfo.getFlags();
 +      if(flags != null && flags.contains(FieldFlag.MULTI_VALUED)) {
 +        type = typeFactory.createArrayType(type, -1);
 +      }
 +
 +      fieldInfo.add(entry.getKey(), type).nullable(true);
 +    }
 +
 +    return RelDataTypeImpl.proto(fieldInfo.build());
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3793eb5e/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --cc solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index fd088f1,5b806a8..e84cb69
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@@ -904,6 -914,17 +914,16 @@@ public class StreamExpressionTest exten
      t = tuples.get(1);
      assertTrue(t.getString("id").equals("9"));
  
 -
+     stream = factory.constructStream("having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")), and(eq(sum(a_i), 9),eq(sum(a_i), 9)))");
+     context = new StreamContext();
+     context.setSolrClientCache(solrClientCache);
+     stream.setStreamContext(context);
+     tuples = getTuples(stream);
+ 
+     assert(tuples.size() == 1);
+     t = tuples.get(0);
+     assertTrue(t.getDouble("a_f") == 10.0D);
+ 
      solrClientCache.close();
    }
  
@@@ -1001,6 -1023,19 +1022,17 @@@
      t = tuples.get(1);
      assertTrue(t.getString("id").equals("9"));
  
+     stream = factory.constructStream("parallel("+COLLECTIONORALIAS+", workers=2, sort=\"a_f asc\", having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=a_f)), and(eq(sum(a_i), 9),eq(sum(a_i),9))))");
+     context = new StreamContext();
+     context.setSolrClientCache(solrClientCache);
+     stream.setStreamContext(context);
+     tuples = getTuples(stream);
+ 
+     assert(tuples.size() == 1);
+ 
+     t = tuples.get(0);
+     assertTrue(t.getDouble("a_f") == 10.0D);
+ 
 -
 -
      solrClientCache.close();
    }
  


[14/25] lucene-solr:jira/solr-8593: SOLR-8542: README and solr/contrib/ltr/example changes

Posted by kr...@apache.org.
SOLR-8542: README and solr/contrib/ltr/example changes

details:
* reduced README in favour of equivalent Solr Ref Guide content and (new) example/README
* solr/contrib/ltr/example improvements and fixes

also:
* stop supporting '*' in Managed(Feature|Model)Store.doDeleteChild


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

Branch: refs/heads/jira/solr-8593
Commit: eb2a8ba2eec0841f03bbcf7807e602f7164a606e
Parents: 2f62fac
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Dec 28 16:40:47 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Jan 4 17:12:38 2017 +0000

----------------------------------------------------------------------
 solr/contrib/ltr/README.md                      | 390 +------------------
 solr/contrib/ltr/example/README.md              |  60 +++
 solr/contrib/ltr/example/config.json            |  13 +-
 solr/contrib/ltr/example/exampleFeatures.json   |  26 ++
 solr/contrib/ltr/example/libsvm_formatter.py    |  12 +-
 .../ltr/example/techproducts-features.json      |  26 --
 .../contrib/ltr/example/techproducts-model.json |  18 -
 .../ltr/example/train_and_upload_demo_model.py  | 143 ++++---
 .../ltr/store/rest/ManagedFeatureStore.java     |   3 -
 .../solr/ltr/store/rest/ManagedModelStore.java  |   6 +-
 .../multipleadditivetreesmodel.json             |   2 +-
 .../org/apache/solr/ltr/TestRerankBase.java     |   9 +-
 .../model/TestMultipleAdditiveTreesModel.java   |   9 +-
 .../solr/ltr/store/rest/TestModelManager.java   |  47 ++-
 .../store/rest/TestModelManagerPersistence.java |   4 +-
 15 files changed, 237 insertions(+), 531 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/README.md b/solr/contrib/ltr/README.md
index 83fb279..a997d16 100644
--- a/solr/contrib/ltr/README.md
+++ b/solr/contrib/ltr/README.md
@@ -1,390 +1,24 @@
-Apache Solr Learning to Rank
+Welcome to Apache Solr Learning to Rank!
 ========
 
-This is the main [learning to rank integrated into solr](http://www.slideshare.net/lucidworks/learning-to-rank-in-solr-presented-by-michael-nilsson-diego-ceccarelli-bloomberg-lp)
-repository.
-[Read up on learning to rank](https://en.wikipedia.org/wiki/Learning_to_rank)
-
 Apache Solr Learning to Rank (LTR) provides a way for you to extract features
 directly inside Solr for use in training a machine learned model.  You can then
 deploy that model to Solr and use it to rerank your top X search results.
 
-# Test the plugin with solr/example/techproducts in a few easy steps!
-
-Solr provides some simple example of indices. In order to test the plugin with
-the techproducts example please follow these steps.
-If you want to install the plugin on your instance of Solr, please refer
-to the [Solr Ref Guide](https://cwiki.apache.org/confluence/display/solr/Result+Reranking).
-
-1. Compile solr and the examples
-
-    `cd solr`
-    `ant dist`
-    `ant server`
-
-2. Run the example to setup the index, enabling the ltr plugin 
-
-   `./bin/solr -e techproducts -Dsolr.ltr.enabled=true`
-
-3. Deploy features and a model
-
-      `curl -XPUT 'http://localhost:8983/solr/techproducts/schema/feature-store'  --data-binary "@./contrib/ltr/example/techproducts-features.json"  -H 'Content-type:application/json'`
-
-      `curl -XPUT 'http://localhost:8983/solr/techproducts/schema/model-store'  --data-binary "@./contrib/ltr/example/techproducts-model.json"  -H 'Content-type:application/json'`
-
-4. Have fun !
-
-     * Access to the default feature store
-
-       http://localhost:8983/solr/techproducts/schema/feature-store/\_DEFAULT\_
-     * Access to the model store
-
-       http://localhost:8983/solr/techproducts/schema/model-store
-     * Perform a reranking query using the model, and retrieve the features
-
-       http://localhost:8983/solr/techproducts/query?indent=on&q=test&wt=json&rq={!ltr%20model=linear%20reRankDocs=25%20efi.user_query=%27test%27}&fl=[features],price,score,name
-
-
-BONUS: Train an actual machine learning model
-
-1. Download and install [liblinear](https://www.csie.ntu.edu.tw/~cjlin/liblinear/)
-
-2. Change `contrib/ltr/example/config.json` "trainingLibraryLocation" to point to the train directory where you installed liblinear.
-
-3. Extract features, train a reranking model, and deploy it to Solr.
-
-  `cd  contrib/ltr/example`
-
-  `python  train_and_upload_demo_model.py -c config.json`
-
-   This script deploys your features from `config.json` "featuresFile" to Solr.  Then it takes the relevance judged query
-   document pairs of "userQueriesFile" and merges it with the features extracted from Solr into a training
-   file.  That file is used to train a linear model, which is then deployed to Solr for you to rerank results.
-
-4. Search and rerank the results using the trained model
-
-   http://localhost:8983/solr/techproducts/query?indent=on&q=test&wt=json&rq={!ltr%20model=ExampleModel%20reRankDocs=25%20efi.user_query=%27test%27}&fl=price,score,name
-
-# Changes to solrconfig.xml
-```xml
-<config>
-  ...
-
-  <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" />
-
-  <!--  Transformer that will encode the document features in the response.
-  For each document the transformer will add the features as an extra field
-  in the response. The name of the field will be the the name of the
-  transformer enclosed between brackets (in this case [features]).
-  In order to get the feature vector you will have to
-  specify that you want the field (e.g., fl="*,[features])  -->
-
-  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
-    <str name="fvCacheName">QUERY_DOC_FV</str>
-  </transformer>
-
-  <query>
-    ...
-
-    <!-- Cache for storing and fetching feature vectors -->
-    <cache name="QUERY_DOC_FV"
-      class="solr.search.LRUCache"
-      size="4096"
-      initialSize="2048"
-      autowarmCount="4096"
-      regenerator="solr.search.NoOpRegenerator" />
-  </query>
-
-</config>
-
-```
-
-# Defining Features
-In the learning to rank plugin, you can define features in a feature space
-using standard Solr queries. As an example:
-
-###### features.json
-```json
-[
-{ "name": "isBook",
-  "class": "org.apache.solr.ltr.feature.SolrFeature",
-  "params":{ "fq": ["{!terms f=category}book"] }
-},
-{
-  "name":  "documentRecency",
-  "class": "org.apache.solr.ltr.feature.SolrFeature",
-  "params": {
-      "q": "{!func}recip( ms(NOW,publish_date), 3.16e-11, 1, 1)"
-  }
-},
-{
-  "name":"originalScore",
-  "class":"org.apache.solr.ltr.feature.OriginalScoreFeature",
-  "params":{}
-},
-{
-  "name" : "userTextTitleMatch",
-  "class" : "org.apache.solr.ltr.feature.SolrFeature",
-  "params" : { "q" : "{!field f=title}${user_text}" }
-},
- {
-   "name" : "userFromMobile",
-   "class" : "org.apache.solr.ltr.feature.ValueFeature",
-   "params" : { "value" : "${userFromMobile}", "required":true }
- }
-]
-```
-
-Defines five features. Anything that is a valid Solr query can be used to define
-a feature.
-
-### Filter Query Features
-The first feature isBook fires if the term 'book' matches the category field
-for the given examined document. Since in this feature q was not specified,
-either the score 1 (in case of a match) or the score 0 (in case of no match)
-will be returned.
-
-### Query Features
-In the second feature (documentRecency) q was specified using a function query.
-In this case the score for the feature on a given document is whatever the query
-returns (1 for docs dated now, 1/2 for docs dated 1 year ago, 1/3 for docs dated
-2 years ago, etc..) . If both an fq and q is used, documents that don't match
-the fq will receive a score of 0 for the documentRecency feature, all other
-documents will receive the score specified by the query for this feature.
-
-### Original Score Feature
-The third feature (originalScore) has no parameters, and uses the
-OriginalScoreFeature class instead of the SolrFeature class.  Its purpose is
-to simply return the score for the original search request against the current
-matching document.
-
-### External Features
-Users can specify external information that can to be passed in as
-part of the query to the ltr ranking framework. In this case, the
-fourth feature (userTextPhraseMatch) will be looking for an external field
-called 'user_text' passed in through the request, and will fire if there is
-a term match for the document field 'title' from the value of the external
-field 'user_text'.  You can provide default values for external features as
-well by specifying ${myField:myDefault}, similar to how you would in a Solr config.
-In this case, the fifth feature (userFromMobile) will be looking for an external parameter
-called 'userFromMobile' passed in through the request, if the ValueFeature is :
-required=true, it will throw an exception if the external feature is not passed
-required=false, it will silently ignore the feature and avoid the scoring ( at Document scoring time, the model will consider 0 as feature value)
-The advantage in defining a feature as not required, where possible, is to avoid wasting caching space and time in calculating the featureScore.
-See the [Run a Rerank Query](#run-a-rerank-query) section for how to pass in external information.
-
-### Custom Features
-Custom features can be created by extending from
-org.apache.solr.ltr.feature.Feature, however this is generally not recommended.
-The majority of features should be possible to create using the methods described
-above.
-
-# Defining Models
-Currently the Learning to Rank plugin supports 2 generalized forms of
-models: 1. Linear Model i.e. [RankSVM](http://www.cs.cornell.edu/people/tj/publications/joachims_02c.pdf), [Pranking](https://papers.nips.cc/paper/2023-pranking-with-ranking.pdf)
-and 2. Multiple Additive Trees i.e. [LambdaMART](http://research.microsoft.com/pubs/132652/MSR-TR-2010-82.pdf), [Gradient Boosted Regression Trees (GBRT)](https://papers.nips.cc/paper/3305-a-general-boosting-method-and-its-application-to-learning-ranking-functions-for-web-search.pdf)
-
-### Linear
-If you'd like to introduce a bias set a constant feature
-to the bias value you'd like and make a weight of 1.0 for that feature.
-
-###### model.json
-```json
-{
-    "class":"org.apache.solr.ltr.model.LinearModel",
-    "name":"myModelName",
-    "features":[
-        { "name": "userTextTitleMatch"},
-        { "name": "originalScore"},
-        { "name": "isBook"}
-    ],
-    "params":{
-        "weights": {
-            "userTextTitleMatch": 1.0,
-            "originalScore": 0.5,
-            "isBook": 0.1
-        }
-
-    }
-}
-```
-
-This is an example of a toy Linear model. Class specifies the class to be
-using to interpret the model. Name is the model identifier you will use 
-when making request to the ltr framework. Features specifies the feature 
-space that you want extracted when using this model. All features that 
-appear in the model params will be used for scoring and must appear in 
-the features list.  You can add extra features to the features list that 
-will be computed but not used in the model for scoring, which can be useful 
-for logging. Params are the Linear parameters.
-
-Good library for training SVM, an example of a Linear model, is 
-(https://www.csie.ntu.edu.tw/~cjlin/liblinear/ , https://www.csie.ntu.edu.tw/~cjlin/libsvm/) . 
-You will need to convert the libSVM model format to the format specified above.
-
-### Multiple Additive Trees
-
-###### model2.json
-```json
-{
-    "class":"org.apache.solr.ltr.model.MultipleAdditiveTreesModel",
-    "name":"multipleadditivetreesmodel",
-    "features":[
-        { "name": "userTextTitleMatch"},
-        { "name": "originalScore"}
-    ],
-    "params":{
-        "trees": [
-            {
-                "weight" : 1,
-                "root": {
-                    "feature": "userTextTitleMatch",
-                    "threshold": 0.5,
-                    "left" : {
-                        "value" : -100
-                    },
-                    "right": {
-                        "feature" : "originalScore",
-                        "threshold": 10.0,
-                        "left" : {
-                            "value" : 50
-                        },
-                        "right" : {
-                            "value" : 75
-                        }
-                    }
-                }
-            },
-            {
-                "weight" : 2,
-                "root": {
-                    "value" : -10
-                }
-            }
-        ]
-    }
-}
-```
-This is an example of a toy Multiple Additive Trees. Class specifies the class to be using to
-interpret the model. Name is the
-model identifier you will use when making request to the ltr framework.
-Features specifies the feature space that you want extracted when using this
-model. All features that appear in the model params will be used for scoring and
-must appear in the features list.  You can add extra features to the features
-list that will be computed but not used in the model for scoring, which can
-be useful for logging. Params are the Multiple Additive Trees specific parameters. In this
-case we have 2 trees, one with 3 leaf nodes and one with 1 leaf node.
-
-A good library for training LambdaMART, an example of Multiple Additive Trees, is ( http://sourceforge.net/p/lemur/wiki/RankLib/ ).
-You will need to convert the RankLib model format to the format specified above.
-
-# Deploy Models and Features
-To send features run
-
-`curl -XPUT 'http://localhost:8983/solr/collection1/schema/feature-store' --data-binary @/path/features.json -H 'Content-type:application/json'`
-
-To send models run
-
-`curl -XPUT 'http://localhost:8983/solr/collection1/schema/model-store' --data-binary @/path/model.json -H 'Content-type:application/json'`
-
-
-# View Models and Features
-`curl -XGET 'http://localhost:8983/solr/collection1/schema/feature-store'`
-
-`curl -XGET 'http://localhost:8983/solr/collection1/schema/model-store'`
-
-# Run a Rerank Query
-Add to your original solr query
-`rq={!ltr model=myModelName reRankDocs=25}`
-
-The model name is the name of the model you sent to solr earlier.
-The number of documents you want reranked, which can be larger than the
-number you display, is reRankDocs.
-
-### Pass in external information for external features
-Add to your original solr query
-`rq={!ltr reRankDocs=3 model=externalmodel efi.field1='text1' efi.field2='text2'}`
-
-Where "field1" specifies the name of the customized field to be used by one
-or more of your features, and text1 is the information to be pass in. As an
-example that matches the earlier shown userTextTitleMatch feature one could do:
-
-`rq={!ltr reRankDocs=3 model=externalmodel efi.user_text='Casablanca' efi.user_intent='movie'}`
-
-# Extract features
-To extract features you need to use the feature vector transformer `features`
-
-`fl=*,score,[features]&rq={!ltr model=yourModel reRankDocs=25}`
-
-If you use `[features]` together with your reranking model, it will return
-the array of features used by your model. Otherwise you can just ask solr to
-produce the features without doing the reranking:
-
-`fl=*,score,[features store=yourFeatureStore format=[dense|sparse] ]`
-
-This will return the values of the features in the given store. The format of the 
-extracted features will be based on the format parameter. The default is dense.
-
-# Assemble training data
-In order to train a learning to rank model you need training data. Training data is
-what "teaches" the model what the appropriate weight for each feature is. In general
-training data is a collection of queries with associated documents and what their ranking/score
-should be. As an example:
-```
-secretary of state|John Kerry|0.66|CROWDSOURCE
-secretary of state|Cesar A. Perales|0.33|CROWDSOURCE
-secretary of state|New York State|0.0|CROWDSOURCE
-secretary of state|Colorado State University Secretary|0.0|CROWDSOURCE
-
-microsoft ceo|Satya Nadella|1.0|CLICK_LOG
-microsoft ceo|Microsoft|0.0|CLICK_LOG
-microsoft ceo|State|0.0|CLICK_LOG
-microsoft ceo|Secretary|0.0|CLICK_LOG
-```
-In this example the first column indicates the query, the second column indicates a unique id for that doc,
-the third column indicates the relative importance or relevance of that doc, and the fourth column indicates the source.
-There are 2 primary ways you might collect data for use with your machine learning algorithim. The first
-is to collect the clicks of your users given a specific query. There are many ways of preparing this data
-to train a model (http://www.cs.cornell.edu/people/tj/publications/joachims_etal_05a.pdf). The general idea
-is that if a user sees multiple documents and clicks the one lower down, that document should be scored higher
-than the one above it. The second way is explicitly through a crowdsourcing platform like Mechanical Turk or
-CrowdFlower. These platforms allow you to show human workers documents associated with a query and have them
-tell you what the correct ranking should be.
+# Getting Started With Solr Learning To Rank
 
-At this point you'll need to collect feature vectors for each query document pair. You can use the information
-from the Extract features section above to do this. An example script has been included in example/train_and_upload_demo_model.py.
+For information on how to get started with solr ltr please see:
+[Solr Reference Guide's section on Result Reranking](https://cwiki.apache.org/confluence/display/solr/Result+Reranking)
 
-# Explanation of the core reranking logic
-An LTR model is plugged into the ranking through the [LTRQParserPlugin](/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java). The plugin will
-read from the request the model, an instance of [LTRScoringModel](/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LTRScoringModel.java),
-plus other parameters. The plugin will generate an LTRQuery, a particular [ReRankQuery](/solr/core/src/java/org/apache/solr/search/AbstractReRankQuery.java).
-It wraps the original solr query for the first pass ranking, and uses the provided model in an
-[LTRScoringQuery](/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java) to
-rescore and rerank the top documents.  The LTRScoringQuery will take care of computing the values of all the
-[features](/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/Feature.java) and then will delegate the final score
-generation to the LTRScoringModel.
+# Getting Started With Solr
 
-# Speeding up the weight creation with threads
-About half the time for ranking is spent in the creation of weights for each feature used in ranking. If the number of features is significantly high (say, 500 or more), this increases the ranking overhead proportionally. To alleviate this problem, parallel weight creation is provided as a configurable option. In order to use this feature, the following lines need to be added to the solrconfig.xml
-```xml
+For information on how to get started with solr please see:
+[solr/README.txt](../../README.txt)
+[Solr Quick Start](http://lucene.apache.org/solr/quickstart.html)
 
-<config>
-  <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin">
-     <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to share for all requests -->
-     <int name="threadModule.numThreadsPerRequest">5</int> <!-- Maximum threads to use for a single request -->
-  </queryParser>
-  
-  <!-- Transformer for extracting features -->
-  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
-     <str name="fvCacheName">QUERY_DOC_FV</str>
-     <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to share for all requests -->
-     <int name="threadModule.numThreadsPerRequest">5</int> <!-- Maximum threads to use for a single request -->
-  </transformer>
-</config>
+# How To Contribute
 
-```
-  
-The threadModule.totalPoolThreads option limits the total number of threads to be used across all query instances at any given time. threadModule.numThreadsPerRequest limits the number of threads used to process a single query. In the above example, 10 threads will be used to services all queries and a maximum of 5 threads to service a single query. If the solr instance is expected to receive no more than one query at a time, it is best to set both these numbers to the same value. If multiple queries need to be serviced simultaneously, the numbers can be adjusted based on the expected response times. If the value of threadModule.numThreadsPerRequest is higher, the response time for a single query will be improved upto a point. If multiple queries are serviced simultaneously, the threadModule.totalPoolThreads imposes a contention between the queries if (threadModule.numThreadsPerRequest*total parallel queries > threadModule.totalPoolThreads).
+For information on how to contribute see:
+http://wiki.apache.org/lucene-java/HowToContribute
+http://wiki.apache.org/solr/HowToContribute
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/README.md b/solr/contrib/ltr/example/README.md
new file mode 100644
index 0000000..1363d5d
--- /dev/null
+++ b/solr/contrib/ltr/example/README.md
@@ -0,0 +1,60 @@
+This README file is only about this example directory's content.
+
+Please refer to the Solr Reference Guide's section on [Result Reranking](https://cwiki.apache.org/confluence/display/solr/Result+Reranking) section for broader information on Learning to Rank (LTR) with Apache Solr.
+
+# Start Solr with the LTR plugin enabled
+
+   `./bin/solr -e techproducts -Dsolr.ltr.enabled=true`
+
+# Train an example machine learning model using LIBLINEAR
+
+1. Download and install [liblinear](https://www.csie.ntu.edu.tw/~cjlin/liblinear/)
+
+2. Change `contrib/ltr/example/config.json` "trainingLibraryLocation" to point to the train directory where you installed liblinear.
+
+   Alternatively, leave the `config.json` file unchanged and create a soft-link to your `liblinear` directory e.g.
+
+  `ln -s /Users/YourNameHere/Downloads/liblinear-2.1 ./contrib/ltr/example/liblinear`
+
+3. Extract features, train a reranking model, and deploy it to Solr.
+
+  `cd contrib/ltr/example`
+
+  `python train_and_upload_demo_model.py -c config.json`
+
+   This script deploys your features from `config.json` "solrFeaturesFile" to Solr.  Then it takes the relevance judged query
+   document pairs of "userQueriesFile" and merges it with the features extracted from Solr into a training
+   file.  That file is used to train a linear model, which is then deployed to Solr for you to rerank results.
+
+4. Search and rerank the results using the trained model
+
+   http://localhost:8983/solr/techproducts/query?indent=on&q=test&wt=json&rq={!ltr%20model=exampleModel%20reRankDocs=25%20efi.user_query=%27test%27}&fl=price,score,name
+
+# Assemble training data
+In order to train a learning to rank model you need training data. Training data is
+what "teaches" the model what the appropriate weight for each feature is. In general
+training data is a collection of queries with associated documents and what their ranking/score
+should be. As an example:
+```
+hard drive|SP2514N|0.6666666|CLICK_LOGS
+hard drive|6H500F0|0.330082034|CLICK_LOGS
+hard drive|F8V7067-APL-KIT|0.0|CLICK_LOGS
+hard drive|IW-02|0.0|CLICK_LOGS
+
+ipod|MA147LL/A|1.0|EXPLICIT
+ipod|F8V7067-APL-KIT|0.25|EXPLICIT
+ipod|IW-02|0.25|EXPLICIT
+ipod|6H500F0|0.0|EXPLICIT
+```
+In this example the first column indicates the query, the second column indicates a unique id for that doc,
+the third column indicates the relative importance or relevance of that doc, and the fourth column indicates the source.
+There are 2 primary ways you might collect data for use with your machine learning algorithim. The first
+is to collect the clicks of your users given a specific query. There are many ways of preparing this data
+to train a model (http://www.cs.cornell.edu/people/tj/publications/joachims_etal_05a.pdf). The general idea
+is that if a user sees multiple documents and clicks the one lower down, that document should be scored higher
+than the one above it. The second way is explicitly through a crowdsourcing platform like Mechanical Turk or
+CrowdFlower. These platforms allow you to show human workers documents associated with a query and have them
+tell you what the correct ranking should be.
+
+At this point you'll need to collect feature vectors for each query document pair. You can use the information
+from the Extract features section above to do this. An example script has been included in example/train_and_upload_demo_model.py.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/config.json
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/config.json b/solr/contrib/ltr/example/config.json
index 483fe69..0b07b0e 100644
--- a/solr/contrib/ltr/example/config.json
+++ b/solr/contrib/ltr/example/config.json
@@ -4,11 +4,14 @@
   "collection":               "techproducts",
   "requestHandler":           "query",
   "q":                        "*:*",
-  "otherParams":              "fl=id,score,[features efi.user_query='$USERQUERY']",
+  "efiParams":                "efi.user_query='$USERQUERY'",
   "userQueriesFile":          "user_queries.txt",
-  "trainingFile":             "ClickData",
-  "featuresFile":             "techproducts-features.json",
+  "trainingFile":             "exampleTrainingFile.txt",
+  "trainedModelFile":         "exampleTrainedModel.txt",
   "trainingLibraryLocation":  "liblinear/train",
-  "solrModelFile":            "solrModel.json",
-  "solrModelName":            "ExampleModel"
+  "trainingLibraryOptions":   "-q",
+  "solrFeaturesFile":         "exampleFeatures.json",
+  "solrFeatureStoreName":     "exampleFeatureStore",
+  "solrModelFile":            "exampleModel.json",
+  "solrModelName":            "exampleModel"
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/exampleFeatures.json
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/exampleFeatures.json b/solr/contrib/ltr/example/exampleFeatures.json
new file mode 100644
index 0000000..89423f3
--- /dev/null
+++ b/solr/contrib/ltr/example/exampleFeatures.json
@@ -0,0 +1,26 @@
+[
+  {
+    "store" : "exampleFeatureStore",
+    "name" : "isInStock",
+    "class" : "org.apache.solr.ltr.feature.FieldValueFeature",
+    "params" : { "field" : "inStock" }
+  },
+  {
+    "store" : "exampleFeatureStore",
+    "name" : "originalScore",
+    "class" : "org.apache.solr.ltr.feature.OriginalScoreFeature",
+    "params" : {}
+  },
+  {
+    "store" : "exampleFeatureStore",
+    "name" : "price",
+    "class" : "org.apache.solr.ltr.feature.FieldValueFeature",
+    "params" : { "field" : "price" }
+  },
+  {
+    "store" : "exampleFeatureStore",
+    "name" : "productNameMatchQuery",
+    "class" : "org.apache.solr.ltr.feature.SolrFeature",
+    "params" : { "q" : "{!field f=name}${user_query}" }
+  }
+]

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/libsvm_formatter.py
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/libsvm_formatter.py b/solr/contrib/ltr/example/libsvm_formatter.py
index 25cf10b..4bcb761 100644
--- a/solr/contrib/ltr/example/libsvm_formatter.py
+++ b/solr/contrib/ltr/example/libsvm_formatter.py
@@ -10,12 +10,12 @@ class LibSvmFormatter:
         for each query in a tuple composed of: (query , docId , relevance , source , featureVector).
         The list of documents that are part of the same query will generate comparisons
         against each other for training. '''
-        curQueryAndSource = "";
         with open(trainingFile,"w") as output:
             self.featureNameToId  = {}
             self.featureIdToName = {}
             self.curFeatIndex = 1;
             curListOfFv = []
+            curQueryAndSource = ""
             for query,docId,relevance,source,featureVector in docClickInfo:
                 if curQueryAndSource != query + source:
                     #Time to flush out all the pairs
@@ -31,7 +31,7 @@ class LibSvmFormatter:
         this requirement.'''
         features = {}
         for keyValuePairStr in featureVector:
-            featName,featValue = keyValuePairStr.split(":");
+            featName,featValue = keyValuePairStr.split("=");
             features[self._getFeatureId(featName)] = float(featValue);
         return features
 
@@ -42,10 +42,12 @@ class LibSvmFormatter:
                 self.curFeatIndex += 1;
         return self.featureNameToId[key];
 
-    def convertLibSvmModelToLtrModel(self,libSvmModelLocation, outputFile, modelName):
+    def convertLibSvmModelToLtrModel(self,libSvmModelLocation,outputFile,modelName,featureStoreName):
         with open(libSvmModelLocation, 'r') as inFile:
             with open(outputFile,'w') as convertedOutFile:
+                # TODO: use json module instead of direct write
                 convertedOutFile.write('{\n\t"class":"org.apache.solr.ltr.model.LinearModel",\n')
+                convertedOutFile.write('\t"store": "' + str(featureStoreName) + '",\n')
                 convertedOutFile.write('\t"name": "' + str(modelName) + '",\n')
                 convertedOutFile.write('\t"features": [\n')
                 isFirst = True;
@@ -117,8 +119,8 @@ def outputLibSvmLine(sign,fvMap,outputFile):
         outputFile.write(" " + str(feat) + ":" + str(fvMap[feat]));
     outputFile.write("\n")
 
-def trainLibSvm(libraryLocation,trainingFileName):
+def trainLibSvm(libraryLocation,libraryOptions,trainingFileName,trainedModelFileName):
     if os.path.isfile(libraryLocation):
-        call([libraryLocation, trainingFileName])
+        call([libraryLocation, libraryOptions, trainingFileName, trainedModelFileName])
     else:
         raise Exception("NO LIBRARY FOUND: " + libraryLocation);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/techproducts-features.json
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/techproducts-features.json b/solr/contrib/ltr/example/techproducts-features.json
deleted file mode 100644
index f358f8b..0000000
--- a/solr/contrib/ltr/example/techproducts-features.json
+++ /dev/null
@@ -1,26 +0,0 @@
-[
-{
-    "name":  "isInStock",
-    "class": "org.apache.solr.ltr.feature.FieldValueFeature",
-    "params": {
-        "field": "inStock"
-    }
-},
-{
-    "name":  "price",
-    "class": "org.apache.solr.ltr.feature.FieldValueFeature",
-    "params": {
-        "field": "price"
-    }
-},
-{
-    "name":"originalScore",
-    "class":"org.apache.solr.ltr.feature.OriginalScoreFeature",
-    "params":{}
-},
-{
-    "name" : "productNameMatchQuery",
-    "class" : "org.apache.solr.ltr.feature.SolrFeature",
-    "params" : { "q" : "{!field f=name}${user_query}" }
-}
-]

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/techproducts-model.json
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/techproducts-model.json b/solr/contrib/ltr/example/techproducts-model.json
deleted file mode 100644
index 0efded7..0000000
--- a/solr/contrib/ltr/example/techproducts-model.json
+++ /dev/null
@@ -1,18 +0,0 @@
-{
-    "class":"org.apache.solr.ltr.model.LinearModel",
-    "name":"linear",
-    "features":[
-    {"name":"isInStock"},
-    {"name":"price"},
-    {"name":"originalScore"},
-    {"name":"productNameMatchQuery"}
-    ],
-    "params":{
-        "weights":{
-            "isInStock":15.0,
-            "price":1.0,
-            "originalScore":5.0,
-            "productNameMatchQuery":1.0
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/example/train_and_upload_demo_model.py
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/train_and_upload_demo_model.py b/solr/contrib/ltr/example/train_and_upload_demo_model.py
index c3762de..3862456 100755
--- a/solr/contrib/ltr/example/train_and_upload_demo_model.py
+++ b/solr/contrib/ltr/example/train_and_upload_demo_model.py
@@ -10,22 +10,61 @@ from optparse import OptionParser
 
 solrQueryUrl = ""
 
-def generateQueries(config):
-        with open(config["userQueriesFile"]) as input:
+
+def setupSolr(collection, host, port, featuresFile, featureStoreName):
+    '''Sets up solr with the proper features for the test'''
+
+    conn = httplib.HTTPConnection(host, port)
+
+    baseUrl = "/solr/" + collection
+    featureUrl = baseUrl + "/schema/feature-store"
+
+    conn.request("DELETE", featureUrl+"/"+featureStoreName)
+    r = conn.getresponse()
+    msg = r.read()
+    if (r.status != httplib.OK and
+        r.status != httplib.CREATED and
+        r.status != httplib.ACCEPTED and
+        r.status != httplib.NOT_FOUND):
+        raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
+
+
+    # Add features
+    headers = {'Content-type': 'application/json'}
+    featuresBody = open(featuresFile)
+
+    conn.request("POST", featureUrl, featuresBody, headers)
+    r = conn.getresponse()
+    msg = r.read()
+    if (r.status != httplib.OK and
+        r.status != httplib.ACCEPTED):
+        print r.status
+        print ""
+        print r.reason;
+        raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
+
+    conn.close()
+
+
+def generateQueries(userQueriesFile, collection, requestHandler, solrFeatureStoreName, efiParams):
+        with open(userQueriesFile) as input:
             solrQueryUrls = [] #A list of tuples with solrQueryUrl,solrQuery,docId,scoreForPQ,source
 
             for line in input:
                 line = line.strip();
                 searchText,docId,score,source = line.split("|");
-                solrQuery = generateHttpRequest(config,searchText,docId)
+                solrQuery = generateHttpRequest(collection,requestHandler,solrFeatureStoreName,efiParams,searchText,docId)
                 solrQueryUrls.append((solrQuery,searchText,docId,score,source))
 
         return solrQueryUrls;
 
-def generateHttpRequest(config,searchText,docId):
+
+def generateHttpRequest(collection, requestHandler, solrFeatureStoreName, efiParams, searchText, docId):
     global solrQueryUrl
     if len(solrQueryUrl) < 1:
-        solrQueryUrl = "/solr/%(collection)s/%(requestHandler)s?%(otherParams)s&q=" % config
+        solrQueryUrl = "/".join([ "", "solr", collection, requestHandler ])
+        solrQueryUrl += ("?fl=" + ",".join([ "id", "score", "[features store="+solrFeatureStoreName+" "+efiParams+"]" ]))
+        solrQueryUrl += "&q="
         solrQueryUrl = solrQueryUrl.replace(" ","+")
         solrQueryUrl += urllib.quote_plus("id:")
 
@@ -36,10 +75,11 @@ def generateHttpRequest(config,searchText,docId):
 
     return solrQuery
 
-def generateTrainingData(solrQueries, config):
+
+def generateTrainingData(solrQueries, host, port):
     '''Given a list of solr queries, yields a tuple of query , docId , score , source , feature vector for each query.
-    Feature Vector is a list of strings of form "key:value"'''
-    conn = httplib.HTTPConnection(config["host"], config["port"])
+    Feature Vector is a list of strings of form "key=value"'''
+    conn = httplib.HTTPConnection(host, port)
     headers = {"Connection":" keep-alive"}
 
     try:
@@ -64,7 +104,7 @@ def generateTrainingData(solrQueries, config):
 
             if r.status == httplib.OK:
                 #print "http connection was ok for: " + queryUrl
-                yield(query,docId,score,source,fv.split(";"));
+                yield(query,docId,score,source,fv.split(","));
             else:
                 raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
     except Exception as e:
@@ -73,40 +113,29 @@ def generateTrainingData(solrQueries, config):
 
     conn.close()
 
-def setupSolr(config):
-    '''Sets up solr with the proper features for the test'''
-
-    conn = httplib.HTTPConnection(config["host"], config["port"])
-
-    baseUrl = "/solr/" + config["collection"]
-    featureUrl = baseUrl + "/schema/feature-store"
-
-    # CAUTION! This will delete all feature stores. This is just for demo purposes
-    conn.request("DELETE", featureUrl+"/*")
-    r = conn.getresponse()
-    msg = r.read()
-    if (r.status != httplib.OK and
-        r.status != httplib.CREATED and
-        r.status != httplib.ACCEPTED and
-        r.status != httplib.NOT_FOUND):
-        raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
-
 
-    # Add features
+def uploadModel(collection, host, port, modelFile, modelName):
+    modelUrl = "/solr/" + collection + "/schema/model-store"
     headers = {'Content-type': 'application/json'}
-    featuresBody = open(config["featuresFile"])
+    with open(modelFile) as modelBody:
+        conn = httplib.HTTPConnection(host, port)
 
-    conn.request("POST", featureUrl, featuresBody, headers)
-    r = conn.getresponse()
-    msg = r.read()
-    if (r.status != httplib.OK and
-        r.status != httplib.ACCEPTED):
-        print r.status
-        print ""
-        print r.reason;
-        raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
+        conn.request("DELETE", modelUrl+"/"+modelName)
+        r = conn.getresponse()
+        msg = r.read()
+        if (r.status != httplib.OK and
+            r.status != httplib.CREATED and
+            r.status != httplib.ACCEPTED and
+            r.status != httplib.NOT_FOUND):
+            raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
 
-    conn.close()
+        conn.request("POST", modelUrl, modelBody, headers)
+        r = conn.getresponse()
+        msg = r.read()
+        if (r.status != httplib.OK and
+            r.status != httplib.CREATED and
+            r.status != httplib.ACCEPTED):
+                raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
 
 
 def main(argv=None):
@@ -126,38 +155,26 @@ def main(argv=None):
     with open(options.configFile) as configFile:
         config = json.load(configFile)
 
-        print "Uploading feature space to Solr"
-        setupSolr(config)
+        print "Uploading features ("+config["solrFeaturesFile"]+") to Solr"
+        setupSolr(config["collection"], config["host"], config["port"], config["solrFeaturesFile"], config["solrFeatureStoreName"])
 
-        print "Generating feature extraction Solr queries"
-        reRankQueries = generateQueries(config)
+        print "Converting user queries ("+config["userQueriesFile"]+") into Solr queries for feature extraction"
+        reRankQueries = generateQueries(config["userQueriesFile"], config["collection"], config["requestHandler"], config["solrFeatureStoreName"], config["efiParams"])
 
-        print "Extracting features"
-        fvGenerator = generateTrainingData(reRankQueries, config);
+        print "Running Solr queries to extract features"
+        fvGenerator = generateTrainingData(reRankQueries, config["host"], config["port"])
         formatter = libsvm_formatter.LibSvmFormatter();
         formatter.processQueryDocFeatureVector(fvGenerator,config["trainingFile"]);
 
-        print "Training ranksvm model"
-        libsvm_formatter.trainLibSvm(config["trainingLibraryLocation"],config["trainingFile"])
+        print "Training model using '"+config["trainingLibraryLocation"]+" "+config["trainingLibraryOptions"]+"'"
+        libsvm_formatter.trainLibSvm(config["trainingLibraryLocation"],config["trainingLibraryOptions"],config["trainingFile"],config["trainedModelFile"])
 
-        print "Converting ranksvm model to solr model"
-        formatter.convertLibSvmModelToLtrModel(config["trainingFile"] + ".model", config["solrModelFile"], config["solrModelName"])
+        print "Converting trained model ("+config["trainedModelFile"]+") to solr model ("+config["solrModelFile"]+")"
+        formatter.convertLibSvmModelToLtrModel(config["trainedModelFile"], config["solrModelFile"], config["solrModelName"], config["solrFeatureStoreName"])
 
-        print "Uploading model to solr"
-        uploadModel(config["collection"], config["host"], config["port"], config["solrModelFile"])
+        print "Uploading model ("+config["solrModelFile"]+") to Solr"
+        uploadModel(config["collection"], config["host"], config["port"], config["solrModelFile"], config["solrModelName"])
 
-def uploadModel(collection, host, port, modelFile):    
-    modelUrl = "/solr/" + collection + "/schema/model-store"
-    headers = {'Content-type': 'application/json'}
-    with open(modelFile) as modelBody:
-        conn = httplib.HTTPConnection(host, port)
-        conn.request("POST", modelUrl, modelBody, headers)
-        r = conn.getresponse()
-        msg = r.read()
-        if (r.status != httplib.OK and
-            r.status != httplib.CREATED and
-            r.status != httplib.ACCEPTED):
-                raise Exception("Status: {0} {1}\nResponse: {2}".format(r.status, r.reason, msg))
 
 if __name__ == '__main__':
     sys.exit(main())

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java
index 2c7bce5..6b415a6 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedFeatureStore.java
@@ -149,9 +149,6 @@ public class ManagedFeatureStore extends ManagedResource implements ManagedResou
 
   @Override
   public synchronized void doDeleteChild(BaseSolrResource endpoint, String childId) {
-    if (childId.equals("*")) {
-      stores.clear();
-    }
     if (stores.containsKey(childId)) {
       stores.remove(childId);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
index 9c19b0a..79640c1 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/store/rest/ManagedModelStore.java
@@ -167,11 +167,7 @@ public class ManagedModelStore extends ManagedResource implements ManagedResourc
 
   @Override
   public synchronized void doDeleteChild(BaseSolrResource endpoint, String childId) {
-    if (childId.equals("*")) {
-      store.clear();
-    } else {
-      store.delete(childId);
-    }
+    store.delete(childId);
     storeManagedData(applyUpdatesToManagedData(null));
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/test-files/modelExamples/multipleadditivetreesmodel.json
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/modelExamples/multipleadditivetreesmodel.json b/solr/contrib/ltr/src/test-files/modelExamples/multipleadditivetreesmodel.json
index 37551a0..70c5563 100644
--- a/solr/contrib/ltr/src/test-files/modelExamples/multipleadditivetreesmodel.json
+++ b/solr/contrib/ltr/src/test-files/modelExamples/multipleadditivetreesmodel.json
@@ -16,7 +16,7 @@
                         "value" : "-100"
                     },
                     "right": {
-                        "feature" : "this_feature_doesnt_exist",
+                        "feature" : "constantScoreToForceMultipleAdditiveTreesScoreAllDocs",
                         "threshold": "10.0f",
                         "left" : {
                             "value" : "50"

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
index 5277821..6f5e0ce 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
@@ -43,6 +43,7 @@ import org.apache.solr.ltr.feature.ValueFeature;
 import org.apache.solr.ltr.model.LTRScoringModel;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.model.ModelException;
+import org.apache.solr.ltr.store.FeatureStore;
 import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
 import org.apache.solr.ltr.store.rest.ManagedModelStore;
 import org.apache.solr.request.SolrQueryRequestBase;
@@ -311,6 +312,12 @@ public class TestRerankBase extends RestTestBase {
 
   public static LTRScoringModel createModelFromFiles(String modelFileName,
       String featureFileName) throws ModelException, Exception {
+    return createModelFromFiles(modelFileName, featureFileName,
+        FeatureStore.DEFAULT_FEATURE_STORE_NAME);
+  }
+
+  public static LTRScoringModel createModelFromFiles(String modelFileName,
+      String featureFileName, String featureStoreName) throws ModelException, Exception {
     URL url = TestRerankBase.class.getResource("/modelExamples/"
         + modelFileName);
     final String modelJson = FileUtils.readFileToString(new File(url.toURI()),
@@ -331,7 +338,7 @@ public class TestRerankBase extends RestTestBase {
 
     final ManagedFeatureStore fs = getManagedFeatureStore();
     // fs.getFeatureStore(null).clear();
-    fs.doDeleteChild(null, "*"); // is this safe??
+    fs.doDeleteChild(null, featureStoreName); // is this safe??
     // based on my need to call this I dont think that
     // "getNewManagedFeatureStore()"
     // is actually returning a new feature store each time

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestMultipleAdditiveTreesModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestMultipleAdditiveTreesModel.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestMultipleAdditiveTreesModel.java
index 1824eb4..3c817d2 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestMultipleAdditiveTreesModel.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/model/TestMultipleAdditiveTreesModel.java
@@ -38,10 +38,6 @@ public class TestMultipleAdditiveTreesModel extends TestRerankBase {
     assertU(adoc("id", "4", "title", "w4", "description", "w4", "popularity","4"));
     assertU(adoc("id", "5", "title", "w5", "description", "w5", "popularity","5"));
     assertU(commit());
-
-    loadFeatures("multipleadditivetreesmodel_features.json"); // currently needed to force
-    // scoring on all docs
-    loadModels("multipleadditivetreesmodel.json");
   }
 
   @AfterClass
@@ -52,6 +48,9 @@ public class TestMultipleAdditiveTreesModel extends TestRerankBase {
 
   @Test
   public void testMultipleAdditiveTreesScoringWithAndWithoutEfiFeatureMatches() throws Exception {
+    loadFeatures("multipleadditivetreesmodel_features.json");
+    loadModels("multipleadditivetreesmodel.json");
+
     final SolrQuery query = new SolrQuery();
     query.setQuery("*:*");
     query.add("rows", "3");
@@ -75,7 +74,7 @@ public class TestMultipleAdditiveTreesModel extends TestRerankBase {
     query.add("rq", "{!ltr reRankDocs=3 model=multipleadditivetreesmodel efi.user_query=w3}");
 
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==-20.0");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==30.0");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==-120.0");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==-120.0");
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
index 5b43c60..125f452 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManager.java
@@ -23,10 +23,10 @@ import org.apache.solr.ltr.feature.FieldValueFeature;
 import org.apache.solr.ltr.feature.ValueFeature;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.search.LTRQParserPlugin;
+import org.apache.solr.ltr.store.FeatureStore;
 import org.apache.solr.rest.ManagedResource;
 import org.apache.solr.rest.ManagedResourceStorage;
 import org.apache.solr.rest.RestManager;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -37,13 +37,6 @@ public class TestModelManager extends TestRerankBase {
     setuptest(true);
   }
 
-  @Before
-  public void restart() throws Exception {
-    restTestHarness.delete(ManagedFeatureStore.REST_END_POINT + "/*");
-    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/*");
-
-  }
-
   @Test
   public void test() throws Exception {
     final SolrResourceLoader loader = new SolrResourceLoader(
@@ -76,6 +69,7 @@ public class TestModelManager extends TestRerankBase {
 
   @Test
   public void testRestManagerEndpoints() throws Exception {
+    final String TEST_FEATURE_STORE_NAME = "TEST";
     // relies on these ManagedResources being activated in the
     // schema-rest.xml used by this test
     assertJQ("/schema/managed", "/responseHeader/status==0");
@@ -95,7 +89,7 @@ public class TestModelManager extends TestRerankBase {
     assertJPut(ManagedFeatureStore.REST_END_POINT, feature,
         "/responseHeader/status==0");
 
-    feature = "{\"name\": \"test33\", \"store\": \"TEST\", \"class\": \""+valueFeatureClassName+"\", \"params\": {\"value\": 1} }";
+    feature = "{\"name\": \"test33\", \"store\": \""+TEST_FEATURE_STORE_NAME+"\", \"class\": \""+valueFeatureClassName+"\", \"params\": {\"value\": 1} }";
     assertJPut(ManagedFeatureStore.REST_END_POINT, feature,
         "/responseHeader/status==0");
 
@@ -136,17 +130,26 @@ public class TestModelManager extends TestRerankBase {
     assert (qryResult.contains("\"name\":\"testmodel3\"")
         && qryResult.contains("\"name\":\"testmodel4\"") && qryResult
           .contains("\"name\":\"testmodel5\""));
-    /*
-     * assertJQ(LTRParams.MSTORE_END_POINT, "/models/[0]/name=='testmodel3'");
-     * assertJQ(LTRParams.MSTORE_END_POINT, "/models/[1]/name=='testmodel4'");
-     * assertJQ(LTRParams.MSTORE_END_POINT, "/models/[2]/name=='testmodel5'");
-     */
+
+    assertJQ(ManagedModelStore.REST_END_POINT, "/models/[0]/name=='testmodel3'");
+    assertJQ(ManagedModelStore.REST_END_POINT, "/models/[1]/name=='testmodel4'");
+    assertJQ(ManagedModelStore.REST_END_POINT, "/models/[2]/name=='testmodel5'");
+    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/testmodel3");
+    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/testmodel4");
+    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/testmodel5");
+    assertJQ(ManagedModelStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+        "/models==[]'");
+
     assertJQ(ManagedFeatureStore.REST_END_POINT,
-        "/featureStores==['TEST','_DEFAULT_']");
-    assertJQ(ManagedFeatureStore.REST_END_POINT + "/_DEFAULT_",
+        "/featureStores==['"+TEST_FEATURE_STORE_NAME+"','"+FeatureStore.DEFAULT_FEATURE_STORE_NAME+"']");
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
         "/features/[0]/name=='test1'");
-    assertJQ(ManagedFeatureStore.REST_END_POINT + "/TEST",
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/"+TEST_FEATURE_STORE_NAME,
         "/features/[0]/name=='test33'");
+    restTestHarness.delete(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME);
+    restTestHarness.delete(ManagedFeatureStore.REST_END_POINT + "/"+TEST_FEATURE_STORE_NAME);
+    assertJQ(ManagedFeatureStore.REST_END_POINT,
+        "/featureStores==[]");
   }
 
   @Test
@@ -154,10 +157,16 @@ public class TestModelManager extends TestRerankBase {
     loadFeatures("features-linear.json");
     loadModels("linear-model.json");
 
+    final String modelName = "6029760550880411648";
     assertJQ(ManagedModelStore.REST_END_POINT,
-        "/models/[0]/name=='6029760550880411648'");
-    assertJQ(ManagedFeatureStore.REST_END_POINT + "/_DEFAULT_",
+        "/models/[0]/name=='"+modelName+"'");
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
+        "/features/[0]/name=='title'");
+    assertJQ(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
         "/features/[1]/name=='description'");
+
+    restTestHarness.delete(ManagedModelStore.REST_END_POINT + "/"+modelName);
+    restTestHarness.delete(ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/eb2a8ba2/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
index f707e55..9168dd0 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
@@ -106,9 +106,9 @@ public class TestModelManagerPersistence extends TestRerankBase {
     assertJQ(ManagedModelStore.REST_END_POINT + "/test-model2",
         "/models/[0]/name=='test-model'");
 
-    assertJDelete(ManagedModelStore.REST_END_POINT + "/*",
+    assertJDelete(ManagedModelStore.REST_END_POINT + "/test-model1",
         "/responseHeader/status==0");
-    assertJDelete(ManagedFeatureStore.REST_END_POINT + "/*",
+    assertJDelete(ManagedFeatureStore.REST_END_POINT + "/test1",
         "/responseHeader/status==0");
     assertJQ(ManagedFeatureStore.REST_END_POINT + "/test1",
         "/features/==[]");


[06/25] lucene-solr:jira/solr-8593: SOLR-8530: Add support for single quoted aggregate HAVING comparisons

Posted by kr...@apache.org.
SOLR-8530: Add support for single quoted aggregate HAVING comparisons


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

Branch: refs/heads/jira/solr-8593
Commit: ccdbb6ac0e0094985e5145c84b3cc2814ababf1d
Parents: db7d2ff
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 20:27:22 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 20:27:22 2017 -0500

----------------------------------------------------------------------
 .../client/solrj/io/ops/EqualsOperation.java    |  2 +-
 .../io/ops/GreaterThanEqualToOperation.java     |  2 +-
 .../solrj/io/ops/GreaterThanOperation.java      |  2 +-
 .../solr/client/solrj/io/ops/LeafOperation.java |  9 ++++++
 .../solrj/io/ops/LessThanEqualToOperation.java  |  2 +-
 .../client/solrj/io/ops/LessThanOperation.java  |  2 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 31 ++++++++++++++++++--
 7 files changed, 43 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ccdbb6ac/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
index b6ad897..9a5c407 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
@@ -45,6 +45,7 @@ public abstract class LeafOperation implements BooleanOperation {
 
   public LeafOperation(StreamExpression expression, StreamFactory factory) throws IOException {
     this.field = factory.getValueOperand(expression, 0);
+    this.field = this.field.replace("'","");
     this.val = Double.parseDouble(factory.getValueOperand(expression, 1));
   }
 
@@ -56,4 +57,12 @@ public abstract class LeafOperation implements BooleanOperation {
         .withImplementingClass(getClass().getName())
         .withExpression(toExpression(factory).toString());
   }
+
+  protected String quote(String s) {
+    if(s.contains("(")) {
+      return "'"+s+"'";
+    }
+
+    return s;
+  }
 }

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ccdbb6ac/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index fd088f1..83c093c 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -837,6 +837,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
         .withFunctionName("search", CloudSolrStream.class)
         .withFunctionName("having", HavingStream.class)
+        .withFunctionName("rollup", RollupStream.class)
+        .withFunctionName("sum", SumMetric.class)
         .withFunctionName("and", AndOperation.class)
         .withFunctionName("or", OrOperation.class)
         .withFunctionName("not", NotOperation.class)
@@ -895,7 +897,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     stream.setStreamContext(context);
     tuples = getTuples(stream);
 
-    System.out.println("####Tuples:"+tuples.size());
     assert(tuples.size() == 2);
 
     t = tuples.get(0);
@@ -904,6 +905,18 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     t = tuples.get(1);
     assertTrue(t.getString("id").equals("9"));
 
+
+    stream = factory.constructStream("having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")), eq('sum(a_i)', 9))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+
+    t = tuples.get(0);
+    assertTrue(t.getDouble("a_f") == 10.0D);
+
     solrClientCache.close();
   }
 
@@ -933,6 +946,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
         .withFunctionName("search", CloudSolrStream.class)
         .withFunctionName("having", HavingStream.class)
+        .withFunctionName("rollup", RollupStream.class)
+        .withFunctionName("sum", SumMetric.class)
         .withFunctionName("and", AndOperation.class)
         .withFunctionName("or", OrOperation.class)
         .withFunctionName("not", NotOperation.class)
@@ -992,7 +1007,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     stream.setStreamContext(context);
     tuples = getTuples(stream);
 
-    System.out.println("####Tuples:"+tuples.size());
     assert(tuples.size() == 2);
 
     t = tuples.get(0);
@@ -1001,6 +1015,19 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     t = tuples.get(1);
     assertTrue(t.getString("id").equals("9"));
 
+    stream = factory.constructStream("parallel("+COLLECTIONORALIAS+", workers=2, sort=\"a_f asc\", having(rollup(over=a_f, sum(a_i), search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=a_f)), eq('sum(a_i)', 9)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+
+    t = tuples.get(0);
+    assertTrue(t.getDouble("a_f") == 10.0D);
+
+
+
     solrClientCache.close();
   }
 


[16/25] lucene-solr:jira/solr-8593: SOLR-7495: Support Facet.field on a non-DocValued, single-value, int field

Posted by kr...@apache.org.
SOLR-7495: Support Facet.field on a non-DocValued, single-value, int field


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

Branch: refs/heads/jira/solr-8593
Commit: f9e3554838f3a43742928d11a7dcd9a8409e0c97
Parents: bd39ae9
Author: Dennis Gove <dp...@gmail.com>
Authored: Wed Jan 4 22:25:29 2017 -0500
Committer: Dennis Gove <dp...@gmail.com>
Committed: Wed Jan 4 22:25:29 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../org/apache/solr/request/SimpleFacets.java   | 37 ++++++++++++--------
 .../apache/solr/request/SimpleFacetsTest.java   | 12 +++++--
 3 files changed, 33 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9e35548/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 556ab23..e3a4b4b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -327,6 +327,8 @@ Bug Fixes
 
 * SOLR-9919: random Streaming Expression is not registered in /stream or /graph handler (Joel Bernstein)
 
+* SOLR-7495: Support Facet.field on a non-DocValued, single-value, int field (Varun Thacker, Scott Stults)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9e35548/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
index 93ca4fa..f29a767 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -45,6 +45,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.LeafCollector;
@@ -649,21 +650,10 @@ public class SimpleFacets {
     BytesRef prefixBytesRef = prefix != null ? new BytesRef(prefix) : null;
     final TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBytesRef, 128);
     
-    SchemaField sf = searcher.getSchema().getFieldOrNull(groupField);
-    
-    if (sf != null && sf.hasDocValues() == false && sf.multiValued() == false && sf.getType().getNumericType() != null) {
-      // it's a single-valued numeric field: we must currently create insanity :(
-      // there isn't a GroupedFacetCollector that works on numerics right now...
-      searcher.search(base.getTopFilter(), new FilterCollector(collector) {
-        @Override
-        public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
-          LeafReader insane = Insanity.wrapInsanity(context.reader(), groupField);
-          return in.getLeafCollector(insane.getContext());
-        }
-      });
-    } else {
-      searcher.search(base.getTopFilter(), collector);
-    }
+    Collector groupWrapper = getInsanityWrapper(groupField, collector);
+    Collector fieldWrapper = getInsanityWrapper(field, groupWrapper);
+    // When GroupedFacetCollector can handle numerics we can remove the wrapped collectors
+    searcher.search(base.getTopFilter(), fieldWrapper);
     
     boolean orderByCount = sort.equals(FacetParams.FACET_SORT_COUNT) || sort.equals(FacetParams.FACET_SORT_COUNT_LEGACY);
     TermGroupFacetCollector.GroupedFacetResult result 
@@ -691,6 +681,23 @@ public class SimpleFacets {
 
     return facetCounts;
   }
+  
+  private Collector getInsanityWrapper(final String field, Collector collector) {
+    SchemaField sf = searcher.getSchema().getFieldOrNull(field);
+    if (sf != null && !sf.hasDocValues() && !sf.multiValued() && sf.getType().getNumericType() != null) {
+      // it's a single-valued numeric field: we must currently create insanity :(
+      // there isn't a GroupedFacetCollector that works on numerics right now...
+      return new FilterCollector(collector) {
+        @Override
+        public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+          LeafReader insane = Insanity.wrapInsanity(context.reader(), field);
+          return in.getLeafCollector(insane.getContext());
+        }
+      };
+    } else {
+      return collector;
+    }
+  }
 
 
   static final Executor directExecutor = new Executor() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9e35548/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
index 85035b9..a21135e 100644
--- a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
+++ b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
@@ -384,7 +384,7 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
         "*[count(//doc)=5]"
     );
     assertQ(
-        "Return two facet counts for field airport_a",
+        "Return two facet counts for field airport_a and duration_i1",
          req(
              "q", "*:*",
              "fq", "id:[2000 TO 2004]",
@@ -393,12 +393,18 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
              "group.field", "hotel_s1",
              "facet", "true",
              "facet.limit", facetLimit,
-             "facet.field", "airport_s1"
+             "facet.field", "airport_s1",
+             "facet.field", "duration_i1"
          ),
         "//lst[@name='facet_fields']/lst[@name='airport_s1']",
         "*[count(//lst[@name='airport_s1']/int)=2]",
         "//lst[@name='airport_s1']/int[@name='ams'][.='2']",
-        "//lst[@name='airport_s1']/int[@name='dus'][.='1']"
+        "//lst[@name='airport_s1']/int[@name='dus'][.='1']",
+
+        "//lst[@name='facet_fields']/lst[@name='duration_i1']",
+        "*[count(//lst[@name='duration_i1']/int)=2]",
+        "//lst[@name='duration_i1']/int[@name='5'][.='2']",
+        "//lst[@name='duration_i1']/int[@name='10'][.='2']"
     );
     assertQ(
         "Return one facet count for field airport_a using facet.offset",


[11/25] lucene-solr:jira/solr-8593: SOLR-9919: random Streaming Expression is not registered in /stream or /graph handler

Posted by kr...@apache.org.
SOLR-9919: random Streaming Expression is not registered in /stream or /graph handler


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

Branch: refs/heads/jira/solr-8593
Commit: b8f4adfd575a1b0c8ddba9f19609f8b94527745e
Parents: d3f83bb
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Jan 4 10:25:14 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Jan 4 10:33:29 2017 -0500

----------------------------------------------------------------------
 .../core/src/java/org/apache/solr/handler/GraphHandler.java | 3 ++-
 .../src/java/org/apache/solr/handler/StreamHandler.java     | 1 +
 .../solr/client/solrj/io/stream/StreamExpressionTest.java   | 9 +++++++++
 3 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8f4adfd/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index 339b4d7..5e5934f 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -118,7 +118,8 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
         .withFunctionName("shortestPath", ShortestPathStream.class)
         .withFunctionName("gatherNodes", GatherNodesStream.class)
         .withFunctionName("sort", SortStream.class)
-            .withFunctionName("scoreNodes", ScoreNodesStream.class)
+        .withFunctionName("scoreNodes", ScoreNodesStream.class)
+        .withFunctionName("random", RandomStream.class)
 
         // metrics
         .withFunctionName("min", MinMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8f4adfd/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index e0f1186..4b319f9 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -119,6 +119,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("jdbc", JDBCStream.class)
       .withFunctionName("topic", TopicStream.class)
       .withFunctionName("commit", CommitStream.class)
+      .withFunctionName("random", RandomStream.class)
       
       // decorator streams
       .withFunctionName("merge", MergeStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8f4adfd/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 83c093c..49814a3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -728,6 +728,15 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       List<Tuple> tuples3 = getTuples(stream);
       assert (tuples3.size() == 1);
 
+
+      //Exercise the /stream handler
+      ModifiableSolrParams sParams = new ModifiableSolrParams(StreamingTest.mapParams(CommonParams.QT, "/stream"));
+      sParams.add("expr", "random(" + COLLECTIONORALIAS + ", q=\"*:*\", rows=\"1\", fl=\"id, a_i\")" );
+      JettySolrRunner jetty = cluster.getJettySolrRunner(0);
+      SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/collection1", sParams);
+      List<Tuple> tuples4 = getTuples(solrStream);
+      assert (tuples4.size() == 1);
+
     } finally {
       cache.close();
     }


[20/25] lucene-solr:jira/solr-8593: SOLR-9877: Add a null check for target

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


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

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

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


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


[19/25] lucene-solr:jira/solr-8593: SOLR-9928: MetricsDirectoryFactory::renameWithOverwrite incorrectly calls super (Mike Drob via ab)

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


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

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

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


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


[04/25] lucene-solr:jira/solr-8593: SOLR-8530: Fixed javadoc

Posted by kr...@apache.org.
SOLR-8530: Fixed javadoc


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

Branch: refs/heads/jira/solr-8593
Commit: 390a01b0fea0dda8d231d165a929ff937d7a8928
Parents: 5bbd4d6
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 14:52:30 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 15:06:14 2017 -0500

----------------------------------------------------------------------
 .../client/solrj/io/ops/BooleanOperation.java   |  6 ++++--
 .../client/solrj/io/stream/HavingStream.java    | 21 ++------------------
 2 files changed, 6 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/390a01b0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
index 609e4e1..d455999 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
@@ -16,8 +16,10 @@
  */
 package org.apache.solr.client.solrj.io.ops;
 
-import org.apache.solr.client.solrj.io.Tuple;
-
+/**
+ *  A BooleanOperation returns true or false for each tuple that it evaluates. The HavingStream applies a BooleanOperation to
+ *  determine which tuples to emit.
+ */
 
 public interface BooleanOperation extends StreamOperation {
   public abstract boolean evaluate();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/390a01b0/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
index 36ca113..38c1a6b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -22,12 +22,7 @@ import java.util.List;
 import java.util.Locale;
 
 import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.comp.FieldComparator;
-import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.eq.FieldEqualitor;
-import org.apache.solr.client.solrj.io.eq.MultipleFieldEqualitor;
-import org.apache.solr.client.solrj.io.eq.StreamEqualitor;
 import org.apache.solr.client.solrj.io.ops.BooleanOperation;
 import org.apache.solr.client.solrj.io.ops.StreamOperation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -35,23 +30,11 @@ import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 /**
- *  Iterates over a TupleStream and buffers Tuples that are equal based on a comparator.
- *  This allows tuples to be grouped by common field(s).
- *
- *  The read() method emits one tuple per group. The fields of the emitted Tuple reflect the first tuple
- *  encountered in the group.
- *
- *  Use the Tuple.getMaps() method to return all the Tuples in the group. This method returns
- *  a list of maps (including the group head), which hold the data for each Tuple in the group.
- *
- *  Note: The ReducerStream requires that it's underlying stream be sorted and partitioned by the same
- *  fields as it's comparator.
- *
+ * The HavingStream iterates over an internal stream and applies a BooleanOperation to each tuple. If the BooleanOperation
+ * evaluates to true then the HavingStream emits the tuple, if it returns false the tuple is not emitted.
  **/
 
 public class HavingStream extends TupleStream implements Expressible {


[15/25] lucene-solr:jira/solr-8593: SOLR-3990: Moves getIndexSize() from ReplicationHandler to SolrCore

Posted by kr...@apache.org.
SOLR-3990: Moves getIndexSize() from ReplicationHandler to SolrCore


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

Branch: refs/heads/jira/solr-8593
Commit: bd39ae9c9d8500d92306478fb51ee6e19009cee9
Parents: eb2a8ba
Author: Dennis Gove <dg...@bloomberg.net>
Authored: Wed Jan 4 12:31:14 2017 -0500
Committer: Dennis Gove <dg...@bloomberg.net>
Committed: Wed Jan 4 13:48:44 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  1 +
 .../src/java/org/apache/solr/core/SolrCore.java | 20 ++++++++++++++++++++
 .../apache/solr/handler/ReplicationHandler.java | 20 ++------------------
 3 files changed, 23 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd39ae9c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index cb41602..556ab23 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -386,6 +386,7 @@ Other Changes
 * SOLR-9915: PeerSync alreadyInSync check is not backwards compatible and results in full replication during a rolling restart
   (Tim Owen via noble)
 
+* SOLR-3990: Moves getIndexSize() from ReplicationHandler to SolrCore (Shawn Heisey)
 
 ==================  6.3.0 ==================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd39ae9c/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index a9fec5a..08072e1 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -146,6 +146,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain.ProcessorInfo;
 import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.NumberUtils;
 import org.apache.solr.util.PropertiesInputStream;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
@@ -391,6 +392,22 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   public IndexReaderFactory getIndexReaderFactory() {
     return indexReaderFactory;
   }
+  
+  public long getIndexSize() {
+    Directory dir;
+    long size = 0;
+    try {
+      dir = directoryFactory.get(getIndexDir(), DirContext.DEFAULT, solrConfig.indexConfig.lockType);
+      try {
+        size = DirectoryFactory.sizeOfDirectory(dir);
+      } finally {
+        directoryFactory.release(dir);
+      }
+    } catch (IOException e) {
+      SolrException.log(log, "IO error while trying to get the size of the Directory", e);
+    }
+    return size;
+  }
 
   @Override
   public String getName() {
@@ -2653,6 +2670,9 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     lst.add("refCount", getOpenCount());
     lst.add("instanceDir", resourceLoader.getInstancePath());
     lst.add("indexDir", getIndexDir());
+    long size = getIndexSize();
+    lst.add("sizeInBytes", size);
+    lst.add("size", NumberUtils.readableSize(size));
 
     CoreDescriptor cd = getCoreDescriptor();
     if (cd != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd39ae9c/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 08b6f39..edf5e94 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -813,22 +813,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     numTimesReplicated++;
   }
 
-  long getIndexSize() {
-    Directory dir;
-    long size = 0;
-    try {
-      dir = core.getDirectoryFactory().get(core.getIndexDir(), DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
-      try {
-        size = core.getDirectoryFactory().size(dir);
-      } finally {
-        core.getDirectoryFactory().release(dir);
-      }
-    } catch (IOException e) {
-      SolrException.log(LOG, "IO error while trying to get the size of the Directory", e);
-    }
-    return size;
-  }
-
   @Override
   public String getDescription() {
     return "ReplicationHandler provides replication of index and configuration files from Master to Slaves";
@@ -855,7 +839,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   public NamedList getStatistics() {
     NamedList list = super.getStatistics();
     if (core != null) {
-      list.add("indexSize", NumberUtils.readableSize(getIndexSize()));
+      list.add("indexSize", NumberUtils.readableSize(core.getIndexSize()));
       CommitVersionInfo vInfo = (core != null && !core.isClosed()) ? getIndexVersion(): null;
       list.add("indexVersion", null == vInfo ? 0 : vInfo.version);
       list.add(GENERATION, null == vInfo ? 0 : vInfo.generation);
@@ -907,7 +891,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     NamedList<Object> master = new SimpleOrderedMap<>();
     NamedList<Object> slave = new SimpleOrderedMap<>();
 
-    details.add("indexSize", NumberUtils.readableSize(getIndexSize()));
+    details.add("indexSize", NumberUtils.readableSize(core.getIndexSize()));
     details.add("indexPath", core.getIndexDir());
     details.add(CMD_SHOW_COMMITS, getCommits());
     details.add("isMaster", String.valueOf(isMaster));


[13/25] lucene-solr:jira/solr-8593: SOLR-9820: change PerSegmentSingleValuedFaceting.(contains|ignoreCase) from default to private visibility. (Jonny Marks via Christine Poerschke)

Posted by kr...@apache.org.
SOLR-9820: change PerSegmentSingleValuedFaceting.(contains|ignoreCase) from default to private visibility. (Jonny Marks via Christine Poerschke)


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

Branch: refs/heads/jira/solr-8593
Commit: 2f62facc0b93095d0be616a5cbda7d0f9ae20747
Parents: 194f516
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Jan 4 16:16:00 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Jan 4 16:16:00 2017 +0000

----------------------------------------------------------------------
 .../apache/solr/request/PerSegmentSingleValuedFaceting.java | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f62facc/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java b/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
index 3db7b4c..c729b3b 100644
--- a/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
+++ b/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
@@ -46,7 +46,9 @@ import org.apache.solr.search.Filter;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.BoundedTreeSet;
 
-
+/**
+ * A class which performs per-segment field faceting for single-valued string fields.
+ */
 class PerSegmentSingleValuedFaceting {
 
   // input params
@@ -59,8 +61,9 @@ class PerSegmentSingleValuedFaceting {
   boolean missing;
   String sort;
   String prefix;
-  String contains;
-  boolean ignoreCase;
+
+  private String contains;
+  private boolean ignoreCase;
 
   Filter baseSet;
 


[22/25] lucene-solr:jira/solr-8593: SOLR-9931: return 0 for hll on field with no values in bucket

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


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

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

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


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

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

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

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


[24/25] lucene-solr:jira/solr-8593: SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules

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


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

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

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


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

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

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

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