You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/11/12 11:55:10 UTC

[01/50] [abbrv] lucene-solr:jira/http2: SOLR-12947: Add SolrJ helper for making JSON DSL requests

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/http2 66da5265d -> 6dd3ef2b8


SOLR-12947: Add SolrJ helper for making JSON DSL requests

The JSON request API is great, but it's hard to use from SolrJ.  This
commit adds 'JsonQueryRequest', which makes it much easier to write
JSON API requests in SolrJ applications.


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

Branch: refs/heads/jira/http2
Commit: 2d95b740db1fa4ae25ccf53432e3060565cc8da2
Parents: 08fcce4
Author: Jason Gerlowski <ge...@apache.org>
Authored: Mon Nov 5 18:36:35 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Tue Nov 6 07:34:53 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/json-request-api.adoc   |  62 +++-
 .../request/json/DirectJsonQueryRequest.java    |  50 ++++
 .../solrj/request/json/JsonQueryRequest.java    | 290 +++++++++++++++++++
 .../client/solrj/request/json/package-info.java |  21 ++
 .../solr/client/solrj/util/ClientUtils.java     |   3 +-
 .../ref_guide_examples/JsonRequestApiTest.java  | 122 ++++++++
 .../json/JsonQueryRequestIntegrationTest.java   | 286 ++++++++++++++++++
 .../request/json/JsonQueryRequestUnitTest.java  | 220 ++++++++++++++
 8 files changed, 1051 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solr-ref-guide/src/json-request-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/json-request-api.adoc b/solr/solr-ref-guide/src/json-request-api.adoc
index a46aeb4..bc676cc 100644
--- a/solr/solr-ref-guide/src/json-request-api.adoc
+++ b/solr/solr-ref-guide/src/json-request-api.adoc
@@ -1,4 +1,6 @@
 = JSON Request API
+:solr-root-path: ../../
+:example-source-dir: {solr-root-path}solrj/src/test/org/apache/solr/client/ref_guide_examples/
 :page-children: json-query-dsl
 // Licensed to the Apache Software Foundation (ASF) under one
 // or more contributor license agreements.  See the NOTICE file
@@ -26,12 +28,31 @@ Here's an example of a search request using query parameters only:
 curl "http://localhost:8983/solr/techproducts/query?q=memory&fq=inStock:true"
 
 The same request when passed as JSON in the body:
+[.dynamic-tabs]
+--
+[example.tab-pane#curlsimplejsonquery]
+====
+[.tab-label]*curl*
 [source,bash]
+----
 curl http://localhost:8983/solr/techproducts/query -d '
 {
   "query" : "memory",
   "filter" : "inStock:true"
 }'
+----
+====
+
+[example.tab-pane#solrjsimplejsonquery]
+====
+[.tab-label]*SolrJ*
+
+[source,java,indent=0]
+----
+include::{example-source-dir}JsonRequestApiTest.java[tag=solrj-json-query-simple]
+----
+====
+--
 
 == Passing JSON via Request Parameter
 It may sometimes be more convenient to pass the JSON body as a request parameter rather than in the actual body of the HTTP request. Solr treats a `json` parameter the same as a JSON body.
@@ -104,7 +125,13 @@ Note: `debug=true` as well as `debugQuery=true` might have too much performance
 == Passing Parameters via JSON
 We can also pass normal query request parameters in the JSON body within the params block:
 
+[.dynamic-tabs]
+--
+[example.tab-pane#curljsonqueryparamsblock]
+====
+[.tab-label]*curl*
 [source,bash]
+----
 curl "http://localhost:8983/solr/techproducts/query?fl=name,price"-d '
 {
   params: {
@@ -112,6 +139,19 @@ curl "http://localhost:8983/solr/techproducts/query?fl=name,price"-d '
     rows: 1
   }
 }'
+----
+====
+
+[example.tab-pane#solrjjsonqueryparamsblock]
+====
+[.tab-label]*SolrJ*
+
+[source,java,indent=0]
+----
+include::{example-source-dir}JsonRequestApiTest.java[tag=solrj-json-query-params-block]
+----
+====
+--
 
 Which is equivalent to
 
@@ -167,9 +207,27 @@ And we get an error back containing the error string:
 Of course request templating via parameter substitution works fully with JSON request bodies or parameters as well.
 For example:
 
+[.dynamic-tabs]
+--
+[example.tab-pane#curljsonquerymacroexpansion]
+====
+[.tab-label]*curl*
 [source,bash]
-curl "http://localhost:8983/solr/techproducts/query?FIELD=text&TERM=memory&HOWMANY=10" -d '
+----
+curl "http://localhost:8983/solr/techproducts/query?FIELD=text&TERM=memory" -d '
 {
   query:"${FIELD}:${TERM}",
-  limit:${HOWMANY}
 }'
+----
+====
+
+[example.tab-pane#solrjjsonquerymacroexpansion]
+====
+[.tab-label]*SolrJ*
+
+[source,java,indent=0]
+----
+include::{example-source-dir}JsonRequestApiTest.java[tag=solrj-json-query-macro-expansion]
+----
+====
+--

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequest.java
new file mode 100644
index 0000000..43e56dc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.request.json;
+
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+
+/**
+ * Represents a query using the <a href="https://lucene.apache.org/solr/guide/json-request-api.html">JSON Query DSL</a>
+ *
+ * This class doesn't construct the request body itself.  It uses a provided String without any modification.  Often
+ * used in combination with the JSON DSL's <a href="https://lucene.apache.org/solr/guide/json-request-api.html#parameter-substitution-macro-expansion">macro expansion capabilities</a>.
+ * The JSON body can contain template parameters which are replaced with values fetched from the {@link SolrParams}
+ * used by this request.  For a more flexible, guided approach to constructing JSON DSL requests, see
+ * {@link JsonQueryRequest}.
+ */
+public class DirectJsonQueryRequest extends QueryRequest {
+  private final String jsonString;
+
+  public DirectJsonQueryRequest(String jsonString) {
+    this(jsonString, new ModifiableSolrParams());
+  }
+
+  public DirectJsonQueryRequest(String jsonString, SolrParams params) {
+    super(params, METHOD.POST);
+    this.jsonString = jsonString;
+  }
+
+  public RequestWriter.ContentWriter getContentWriter(String expectedType) {
+    return new RequestWriter.StringPayloadContentWriter(jsonString, ClientUtils.TEXT_JSON);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
new file mode 100644
index 0000000..3a570d0
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
@@ -0,0 +1,290 @@
+/*
+ * 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.request.json;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.Utils;
+
+/**
+ * Represents a query using the <a href="https://lucene.apache.org/solr/guide/json-request-api.html">JSON Query DSL</a>
+ *
+ * This class constructs the request using setters for individual properties.  For a more monolithic approach to
+ * constructing the JSON request, see {@link DirectJsonQueryRequest}
+ */
+public class JsonQueryRequest extends QueryRequest {
+  private final Map<String, Object> jsonRequestMap;
+
+  /**
+   * Creates a {@link JsonQueryRequest} with an empty {@link SolrParams} object
+   */
+  public JsonQueryRequest() {
+    this(new ModifiableSolrParams());
+  }
+
+  /**
+   * Creates a {@link JsonQueryRequest} using the provided {@link SolrParams}
+   */
+  public JsonQueryRequest(SolrParams params) {
+    super(params, METHOD.POST);
+    this.jsonRequestMap = new HashMap<>();
+  }
+
+  /**
+   * Specify the query sent as a part of this JSON request
+   *
+   * This method may be called multiple times, but each call overwrites the value specified by previous calls.
+   *
+   * @param query a String in either of two formats: a query string for the default deftype (e.g. "title:solr"), or a
+   *              localparams query (e.g. "{!lucene df=text v='solr'}" )
+   *
+   * @throws IllegalArgumentException if {@code query} is null
+   */
+  public JsonQueryRequest setQuery(String query) {
+    if (query == null) {
+      throw new IllegalArgumentException("'query' parameter must be non-null");
+    }
+    jsonRequestMap.put("query", query);
+    return this;
+  }
+
+  /**
+   * Specify the query sent as a part of this JSON request.
+   *
+   * This method may be called multiple times, but each call overwrites the value specified by previous calls.
+   * <p>
+   * <b>Example:</b> You wish to send the JSON request: "{'limit': 5, 'query': {'lucene': {'df':'genre_s', 'query': 'scifi'}}}".  The
+   * query subtree of this request is: "{'lucene': {'df': 'genre_s', 'query': 'scifi'}}".  You would represent this query
+   * JSON as follows:
+   * <pre>{@code
+   *     final Map<String, Object> queryMap = new HashMap<>();
+   *     final Map<String, Object> luceneQueryParamMap = new HashMap<>();
+   *     queryMap.put("lucene", luceneQueryParamMap);
+   *     luceneQueryParamMap.put("df", "genre_s");
+   *     luceneQueryParamMap.put("query", "scifi");
+   * }</pre>
+   *
+   * @param queryJson a Map of values representing the query subtree of the JSON request you wish to send.
+   * @throws IllegalArgumentException if {@code queryJson} is null.
+   */
+  public JsonQueryRequest setQuery(Map<String, Object> queryJson) {
+    if (queryJson == null) {
+      throw new IllegalArgumentException("'queryJson' parameter must be non-null");
+    }
+    jsonRequestMap.put("query", queryJson);
+    return this;
+  }
+
+  /**
+   * Specify whether results should be fetched starting from a particular offset (or 'start').
+   *
+   * Defaults to 0 if not set.
+   *
+   * @param offset a non-negative integer representing the offset (or 'start') to use when returning results
+   *
+   * @throws IllegalArgumentException if {@code offset} is negative
+   */
+  public JsonQueryRequest setOffset(int offset) {
+    if (offset < 0) {
+      throw new IllegalArgumentException("'offset' parameter must be non-negative");
+    }
+    jsonRequestMap.put("offset", offset);
+    return this;
+  }
+
+  /**
+   * Specify how many results should be returned from the JSON request
+   *
+   * @param limit a non-negative integer representing the maximum results to return from a search
+   * @throws IllegalArgumentException if {@code limit} is negative
+   */
+  public JsonQueryRequest setLimit(int limit) {
+    if (limit < 0) {
+      throw new IllegalArgumentException("'limit' parameter must be non-negative");
+    }
+    jsonRequestMap.put("limit", limit);
+    return this;
+  }
+
+  /**
+   * Specify how results to the JSON request should be sorted before being returned by Solr
+   *
+   * @param sort a string representing the desired result sort order (e.g. "price asc")
+   *
+   * @throws IllegalArgumentException if {@code sort} is null
+   */
+  public JsonQueryRequest setSort(String sort) {
+    if (sort == null) {
+      throw new IllegalArgumentException("'sort' parameter must be non-null");
+    }
+    jsonRequestMap.put("sort", sort);
+    return this;
+  }
+
+  /**
+   * Add a filter query to run as a part of the JSON request
+   *
+   * This method may be called multiple times; each call will add a new filter to the request
+   *
+   * @param filterQuery a String in either of two formats: a query string for the default deftype (e.g. "title:solr"), or a
+   *                    localparams query (e.g. "{!lucene df=text v='solr'}" )
+   * @throws IllegalArgumentException if {@code filterQuery} is null
+   */
+  public JsonQueryRequest withFilter(String filterQuery) {
+    if (filterQuery == null) {
+      throw new IllegalArgumentException("'filterQuery' must be non-null");
+    }
+    jsonRequestMap.putIfAbsent("filter", new ArrayList<Object>());
+    final List<Object> filters = (List<Object>) jsonRequestMap.get("filter");
+    filters.add(filterQuery);
+    return this;
+  }
+
+  /**
+   * Add a filter query to run as a part of the JSON request
+   *
+   * This method may be called multiple times; each call will add a new filter to the request
+   * <p>
+   * <b>Example:</b> You wish to send the JSON request: "{'query':'*:*', 'filter': [{'lucene': {'df':'genre_s', 'query': 'scifi'}}]}".
+   * The filter you want to add is: "{'lucene': {'df': 'genre_s', 'query': 'scifi'}}".  You would represent this filter
+   * query as follows:
+   * <pre>{@code
+   *     final Map<String, Object> filterMap = new HashMap<>();
+   *     final Map<String, Object> luceneQueryParamMap = new HashMap<>();
+   *     filterMap.put("lucene", luceneQueryParamMap);
+   *     luceneQueryParamMap.put("df", "genre_s");
+   *     luceneQueryParamMap.put("query", "scifi");
+   * }</pre>
+   *
+   * @param filterQuery a Map of values representing the filter request you wish to send.
+   * @throws IllegalArgumentException if {@code filterQuery} is null
+   */
+  public JsonQueryRequest withFilter(Map<String, Object> filterQuery) {
+    if (filterQuery == null) {
+      throw new IllegalArgumentException("'filterQuery' parameter must be non-null");
+    }
+    jsonRequestMap.putIfAbsent("filter", new ArrayList<Object>());
+    final List<Object> filters = (List<Object>) jsonRequestMap.get("filter");
+    filters.add(filterQuery);
+    return this;
+  }
+
+  /**
+   * Specify fields which should be returned by the JSON request.
+   *
+   * This method may be called multiple times; each call will add a new field to the list of those to be returned.
+   *
+   * @param fieldNames the field names that should be returned by the request
+   */
+  public JsonQueryRequest returnFields(String... fieldNames) {
+    jsonRequestMap.putIfAbsent("fields", new ArrayList<String>());
+    final List<String> fields = (List<String>) jsonRequestMap.get("fields");
+    for (String fieldName : fieldNames) {
+      fields.add(fieldName);
+    }
+    return this;
+  }
+
+  /**
+   * Specify fields which should be returned by the JSON request.
+   *
+   * This method may be called multiple times; each call will add a new field to the list of those to be returned.
+   *
+   * @param fieldNames the field names that should be returned by the request
+   * @throws IllegalArgumentException if {@code fieldNames} is null
+   */
+  public JsonQueryRequest returnFields(Iterable<String> fieldNames) {
+    if (fieldNames == null) {
+      throw new IllegalArgumentException("'fieldNames' parameter must be non-null");
+    }
+    jsonRequestMap.putIfAbsent("fields", new ArrayList<String>());
+    final List<String> fields = (List<String>) jsonRequestMap.get("fields");
+    for (String fieldName : fieldNames) {
+      fields.add(fieldName);
+    }
+    return this;
+  }
+
+  /**
+   * Add a property to the "params" block supported by the JSON query DSL
+   *
+   * The JSON query DSL has special support for a few query parameters (limit/rows, offset/start, filter/fq, etc.).  But
+   * many other query parameters are not explicitly covered by the query DSL.  This method can be used to add any of
+   * these other parameters to the JSON request.
+   * <p>
+   * This method may be called multiple times; each call with a different {@code name} will add a new param name/value
+   * to the params subtree. Invocations that repeat a {@code name} will overwrite the previously specified parameter
+   * values associated with that name.
+   *
+   * @param name the name of the parameter to add
+   * @param value the value of the parameter to add.  Usually a String, Number (Integer, Long, Double), or Boolean.
+   *
+   * @throws IllegalArgumentException if either {@code name} or {@code value} are null
+   */
+  public JsonQueryRequest withParam(String name, Object value) {
+    if (name == null) {
+      throw new IllegalArgumentException("'name' parameter must be non-null");
+    }
+    if (value == null) {
+      throw new IllegalArgumentException("'value' parameter must be non-null");
+    }
+
+    jsonRequestMap.putIfAbsent("params", new HashMap<String, Object>());
+    final Map<String, Object> miscParamsMap = (Map<String, Object>) jsonRequestMap.get("params");
+    miscParamsMap.put(name, value);
+    return this;
+  }
+
+  public RequestWriter.ContentWriter getContentWriter(String expectedType) {
+    return new RequestWriter.ContentWriter() {
+      @Override
+      public void write(OutputStream os) throws IOException {
+        //TODO consider whether using Utils.writeJson would work here as that'd be more mem efficient
+        OutputStreamWriter writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
+
+        writer.write(Utils.toJSONString(jsonRequestMap));
+        writer.flush();
+      }
+
+      @Override
+      public String getContentType() {
+        return ClientUtils.TEXT_JSON;
+      }
+    };
+  }
+
+  @Override
+  public void setMethod(METHOD m) {
+    if (METHOD.POST != m) {
+      final String message = getClass().getName() + " only supports POST for sending JSON queries.";
+      throw new UnsupportedOperationException(message);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/package-info.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/package-info.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/package-info.java
new file mode 100644
index 0000000..f9e9533
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Allows sending of requests using Solr's JSON query/faceting API
+ */
+package org.apache.solr.client.solrj.request.json;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java b/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
index 26a188d..2a1dfad 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
@@ -42,7 +42,8 @@ import org.apache.solr.common.util.XML;
 public class ClientUtils 
 {
   // Standard Content types
-  public static final String TEXT_XML = "application/xml; charset=UTF-8";  
+  public static final String TEXT_XML = "application/xml; charset=UTF-8";
+  public static final String TEXT_JSON = "application/json; charset=UTF-8";
   
   /**
    * Take a string and make it an iterable ContentStream

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
new file mode 100644
index 0000000..b941f2d
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.ref_guide_examples;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Example SolrJ usage of the JSON Request API.
+ *
+ * Snippets surrounded by "tag" and "end" comments are extracted and used in the Solr Reference Guide.
+ */
+public class JsonRequestApiTest extends SolrCloudTestCase {
+  private static final String COLLECTION_NAME = "techproducts";
+  private static final String CONFIG_NAME = "techproducts_config";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig(CONFIG_NAME, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+        .configure();
+
+    final List<String> solrUrls = new ArrayList<>();
+    solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
+
+    ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
+    up.setParam("collection", COLLECTION_NAME);
+    up.addFile(getFile("solrj/docs2.xml"), "application/xml"); // A subset of the 'techproducts' documents
+    up.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
+    UpdateResponse updateResponse = up.process(cluster.getSolrClient());
+    assertEquals(0, updateResponse.getStatus());
+  }
+
+  @Test
+  public void testSimpleJsonQuery() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+    final int expectedResults = 3;
+
+    // tag::solrj-json-query-simple[]
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("memory")
+        .withFilter("inStock:true");
+    QueryResponse queryResponse = simpleQuery.process(solrClient, COLLECTION_NAME);
+    // end::solrj-json-query-simple[]
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(expectedResults, queryResponse.getResults().size());
+  }
+
+  @Test
+  public void testJsonQueryUsingParamsBlock() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+
+    //tag::solrj-json-query-params-block[]
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("fl", "name", "price");
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest(params)
+        .withParam("q", "memory")
+        .withParam("rows", 1);
+    QueryResponse queryResponse = simpleQuery.process(solrClient, COLLECTION_NAME);
+    // end::solrj-json-query-params-block[]
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(1, queryResponse.getResults().size());
+    final SolrDocument doc = queryResponse.getResults().get(0);
+    final Collection<String> returnedFields = doc.getFieldNames();
+    assertEquals(2, doc.getFieldNames().size());
+    assertTrue("Expected returned field list to include 'name'", returnedFields.contains("name"));
+    assertTrue("Expected returned field list to include 'price'", returnedFields.contains("price"));
+  }
+
+  @Test
+  public void testJsonQueryMacroExpansion() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+
+    //tag::solrj-json-query-macro-expansion[]
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("FIELD", "text");
+    params.set("TERM", "memory");
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest(params)
+        .setQuery("${FIELD}:${TERM}");
+    QueryResponse queryResponse = simpleQuery.process(solrClient, COLLECTION_NAME);
+    // end::solrj-json-query-macro-expansion[]
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(3, queryResponse.getResults().size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
new file mode 100644
index 0000000..807f8b6
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
@@ -0,0 +1,286 @@
+/*
+ * 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.request.json;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Integration tests for {@link JsonQueryRequest}
+ */
+public class JsonQueryRequestIntegrationTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION_NAME = "books";
+  private static final String CONFIG_NAME = "techproducts_config";
+
+  private static final int NUM_BOOKS_TOTAL = 10;
+  private static final int NUM_SCIFI_BOOKS = 2;
+  private static final int NUM_IN_STOCK = 8;
+  private static final int NUM_IN_STOCK_AND_FIRST_IN_SERIES = 5;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig(CONFIG_NAME, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+        .configure();
+
+    final List<String> solrUrls = new ArrayList<>();
+    solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
+
+    ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
+    up.setParam("collection", COLLECTION_NAME);
+    up.addFile(getFile("solrj/books.csv"), "application/csv");
+    up.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
+    UpdateResponse updateResponse = up.process(cluster.getSolrClient());
+    assertEquals(0, updateResponse.getStatus());
+  }
+
+  @Test
+  public void testEmptyJson() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest();
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    // No q.alt in techproducts configset, so request should gracefully find no results
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(0, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testCanRunSimpleQueries() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testQueriesCanUseLocalParamsSyntax() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("{!lucene df=genre_s v='scifi'}");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_SCIFI_BOOKS, queryResponse.getResults().getNumFound());
+  }
+
+
+  @Test
+  public void testQueriesCanUseExpandedSyntax() throws Exception {
+    //Construct a tree representing the JSON: {lucene: {df:'genre_s', 'query': 'scifi'}}
+    final Map<String, Object> queryMap = new HashMap<>();
+    final Map<String, Object> luceneQueryParamMap = new HashMap<>();
+    queryMap.put("lucene", luceneQueryParamMap);
+    luceneQueryParamMap.put("df", "genre_s");
+    luceneQueryParamMap.put("query", "scifi");
+
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery(queryMap);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_SCIFI_BOOKS, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testQueriesCanBeNested() throws Exception {
+    final Map<String, Object> queryJsonMap = new HashMap<>();
+    final Map<String, Object> clausesJsonMap = new HashMap<>();
+    queryJsonMap.put("bool", clausesJsonMap);
+    clausesJsonMap.put("must", "genre_s:scifi");
+    clausesJsonMap.put("must_not", "series_t:Ender");
+
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery(queryJsonMap);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(1, queryResponse.getResults().getNumFound()); // 2 scifi books, only 1 is NOT "Ender's Game"
+  }
+
+  @Test
+  public void testFiltersCanBeAddedToQueries() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFilter("inStock:true");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_IN_STOCK, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testFiltersCanUseLocalParamsSyntax() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFilter("{!lucene df=inStock v='true'}");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_IN_STOCK, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testFiltersCanUseExpandedSyntax() throws Exception {
+    final Map<String, Object> filterJsonMap = new HashMap<>();
+    final Map<String, Object> luceneQueryParamsMap = new HashMap<>();
+    filterJsonMap.put("lucene", luceneQueryParamsMap);
+    luceneQueryParamsMap.put("df", "genre_s");
+    luceneQueryParamsMap.put("query", "scifi");
+
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFilter(filterJsonMap);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_SCIFI_BOOKS, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testMultipleFiltersCanBeUsed() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFilter("sequence_i:1") // 7 books are the first of a series
+        .withFilter("inStock:true");// but only 5 are in stock
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_IN_STOCK_AND_FIRST_IN_SERIES, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void canSpecifyFieldsToBeReturned() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .returnFields("id", "name");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    final SolrDocumentList docs = queryResponse.getResults();
+    assertEquals(NUM_BOOKS_TOTAL, docs.getNumFound());
+    for (SolrDocument returnedDoc : docs) {
+      final Collection<String> fields = returnedDoc.getFieldNames();
+      assertEquals(2, fields.size());
+      assertTrue("Expected field list to contain 'id'", fields.contains("id"));
+      assertTrue("Expected field list to contain 'name'", fields.contains("name"));
+    }
+  }
+
+  @Test
+  public void testObeysResultLimit() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .setLimit(5);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL, queryResponse.getResults().getNumFound());
+    assertEquals(5, queryResponse.getResults().size());
+  }
+
+  @Test
+  public void testAcceptsTraditionalQueryParamNamesInParamsBlock() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .withParam("q", "*:*")
+        .withParam("rows", 4);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL, queryResponse.getResults().getNumFound());
+    assertEquals(4, queryResponse.getResults().size());
+  }
+
+  @Test
+  public void testReturnsResultsStartingAtOffset() throws Exception {
+    final JsonQueryRequest originalDocsQuery = new JsonQueryRequest()
+        .setQuery("*:*");
+    QueryResponse originalDocsResponse = originalDocsQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, originalDocsResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL, originalDocsResponse.getResults().size());
+    final SolrDocumentList originalDocs = originalDocsResponse.getResults();
+
+    final int offset = 2;
+    final JsonQueryRequest offsetDocsQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .setOffset(offset);
+    QueryResponse offsetDocsResponse = offsetDocsQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, offsetDocsResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL - offset, offsetDocsResponse.getResults().size());
+    final SolrDocumentList offsetDocs = offsetDocsResponse.getResults();
+
+    // Ensure the same docs are returned, shifted by 'offset'
+    for (int i = 0; i < offsetDocs.size(); i++) {
+      final String offsetId = (String) offsetDocs.get(i).getFieldValue("id");
+      final String originalId = (String) originalDocs.get(i + offset).getFieldValue("id");
+      assertEquals(offsetId, originalId);
+    }
+  }
+
+  @Test
+  public void testReturnsReturnsResultsWithSpecifiedSort() throws Exception {
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery("*:*")
+        .setSort("price desc");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_BOOKS_TOTAL, queryResponse.getResults().getNumFound());
+    final SolrDocumentList docs = queryResponse.getResults();
+    for (int i = 0; i < docs.size() - 1; i++) {
+      final float pricierDocPrice = (Float) docs.get(i).getFieldValue("price");
+      final float cheaperDocPrice = (Float) docs.get(i+1).getFieldValue("price");
+      assertTrue("Expected doc at index " + i + " doc to be more expensive than doc at " + (i+1),
+          pricierDocPrice >= cheaperDocPrice);
+    }
+  }
+
+  @Test
+  public void testCombinesJsonParamsWithUriParams() throws Exception {
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("fq", "inStock:true");
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest(params)
+        .setQuery("*:*");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_IN_STOCK, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
+  public void testExpandsParameterMacros() throws Exception {
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("FIELD", "inStock");
+    params.set("VALUE", "true");
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest(params)
+        .setQuery("${FIELD}:${VALUE}");
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_IN_STOCK, queryResponse.getResults().getNumFound());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d95b740/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
new file mode 100644
index 0000000..c6661fe
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.request.json;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.junit.Test;
+import static org.junit.internal.matchers.StringContains.containsString;
+
+/**
+ * Unit tests for {@link JsonQueryRequest}
+ */
+public class JsonQueryRequestUnitTest extends LuceneTestCase {
+
+  private static final boolean LEAVE_WHITESPACE = false;
+
+  @Test
+  public void testRejectsNullQueryString() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setQuery((String)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsNullQueryMap() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setQuery((Map<String, Object>)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testWritesProvidedQueryStringToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().setQuery("text:solr");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"query\":\"text:solr\""));
+  }
+
+  @Test
+  public void testWritesProvidedQueryMapToJsonCorrectly() {
+    final Map<String, Object> queryMap = new HashMap<>();
+    final Map<String, Object> paramsMap = new HashMap<>();
+    queryMap.put("lucene", paramsMap);
+    paramsMap.put("df", "text");
+    paramsMap.put("q", "*:*");
+    final JsonQueryRequest request = new JsonQueryRequest().setQuery(queryMap);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"query\":{\"lucene\":{\"q\":\"*:*\",\"df\":\"text\"}}"));
+  }
+
+  @Test
+  public void testRejectsInvalidLimit() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setLimit(-1);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-negative"));
+  }
+
+  @Test
+  public void testWritesProvidedLimitToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().setLimit(5);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"limit\":5"));
+  }
+
+  @Test
+  public void testRejectsInvalidOffset() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setOffset(-1);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-negative"));
+
+  }
+
+  @Test
+  public void testWritesProvidedOffsetToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().setOffset(5);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"offset\":5"));
+  }
+
+  @Test
+  public void testRejectsInvalidSort() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setSort(null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+
+  }
+
+  @Test
+  public void testWritesProvidedSortToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().setSort("price asc");
+    final String requestBody = writeRequestToJson(request, LEAVE_WHITESPACE);
+    assertThat(requestBody, containsString("\"sort\":\"price asc"));
+  }
+
+  @Test
+  public void testRejectsInvalidFilterString() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withFilter((String)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidFilterMap() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withFilter((Map<String,Object>)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testWritesProvidedFilterToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().withFilter("text:solr");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"filter\":[\"text:solr\"]"));
+  }
+
+  @Test
+  public void testWritesMultipleProvidedFiltersToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().withFilter("text:solr").withFilter("text:lucene");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"filter\":[\"text:solr\",\"text:lucene\"]"));
+  }
+
+  @Test
+  public void testRejectsInvalidFieldsIterable() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().returnFields((Iterable<String>)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testWritesProvidedFieldsToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().returnFields("price");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"fields\":[\"price\"]"));
+  }
+
+  @Test
+  public void testWritesMultipleProvidedFieldsToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().returnFields("price", "name");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"fields\":[\"price\",\"name\"]"));
+  }
+
+  @Test
+  public void testRejectsInvalidMiscParamName() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withParam(null, "any-value");
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidMiscParamValue() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withParam("any-name", null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+
+  }
+
+  @Test
+  public void testWritesMiscParamsToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().withParam("fq", "inStock:true");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"params\":{\"fq\":\"inStock:true\"}"));
+  }
+
+  private String writeRequestToJson(JsonQueryRequest request, boolean trimWhitespace) {
+    final RequestWriter.ContentWriter writer = request.getContentWriter(ClientUtils.TEXT_JSON);
+    final ByteArrayOutputStream os = new ByteArrayOutputStream();
+    try {
+      writer.write(os);
+      final String rawJsonString = new String(os.toByteArray(), StandardCharsets.UTF_8);
+      // Trimming whitespace makes our assertions in these tests more stable (independent of JSON formatting) so we do
+      // it by default.  But we leave the option open in case the JSON fields have spaces.
+      if (trimWhitespace) {
+        return rawJsonString.replaceAll("\n", "").replaceAll(" ","");
+      } else {
+        return rawJsonString;
+      }
+    } catch (IOException e) {
+      /* Unreachable in practice, since we're not doing any I/O here */
+      throw new RuntimeException(e);
+    }
+  }
+
+  private String writeRequestToJson(JsonQueryRequest request) {
+    return writeRequestToJson(request, true);
+  }
+}


[03/50] [abbrv] lucene-solr:jira/http2: SOLR-12956: Add Javadoc @since tag to Analyzer component classes

Posted by da...@apache.org.
SOLR-12956: Add Javadoc @since tag to Analyzer component classes


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

Branch: refs/heads/jira/http2
Commit: c07df196664b84cd2d58ce1ba9040a6b06e0a3c5
Parents: 2d95b74
Author: Alexandre Rafalovitch <ar...@apache.org>
Authored: Tue Nov 6 07:36:14 2018 -0500
Committer: Alexandre Rafalovitch <ar...@apache.org>
Committed: Tue Nov 6 07:38:20 2018 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java   | 3 ++-
 .../lucene/analysis/ar/ArabicNormalizationFilterFactory.java     | 2 ++
 .../org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java   | 2 ++
 .../java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java  | 2 ++
 .../java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java    | 2 ++
 .../apache/lucene/analysis/br/BrazilianStemFilterFactory.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java  | 2 ++
 .../lucene/analysis/charfilter/HTMLStripCharFilterFactory.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java     | 2 ++
 .../src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java  | 2 ++
 .../lucene/analysis/commongrams/CommonGramsFilterFactory.java    | 2 ++
 .../analysis/commongrams/CommonGramsQueryFilterFactory.java      | 2 ++
 .../compound/DictionaryCompoundWordTokenFilterFactory.java       | 2 ++
 .../java/org/apache/lucene/analysis/core/KeywordAnalyzer.java    | 2 ++
 .../org/apache/lucene/analysis/core/KeywordTokenizerFactory.java | 2 ++
 .../org/apache/lucene/analysis/core/LetterTokenizerFactory.java  | 2 ++
 .../org/apache/lucene/analysis/core/LowerCaseFilterFactory.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java | 2 ++
 .../src/java/org/apache/lucene/analysis/core/StopAnalyzer.java   | 2 ++
 .../java/org/apache/lucene/analysis/core/StopFilterFactory.java  | 2 ++
 .../apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java   | 2 ++
 .../java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java | 2 ++
 .../apache/lucene/analysis/core/WhitespaceTokenizerFactory.java  | 2 ++
 .../java/org/apache/lucene/analysis/custom/CustomAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java   | 2 ++
 .../org/apache/lucene/analysis/de/GermanStemFilterFactory.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java    | 2 ++
 .../apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java  | 2 ++
 .../org/apache/lucene/analysis/en/PorterStemFilterFactory.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java  | 2 ++
 .../org/apache/lucene/analysis/fa/PersianCharFilterFactory.java  | 2 ++
 .../lucene/analysis/fa/PersianNormalizationFilterFactory.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java | 2 ++
 .../src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java    | 2 ++
 .../java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java | 2 ++
 .../java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java  | 2 ++
 .../java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java  | 2 ++
 .../lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java | 2 ++
 .../analysis/miscellaneous/ConditionalTokenFilterFactory.java    | 2 ++
 .../analysis/miscellaneous/HyphenatedWordsFilterFactory.java     | 2 ++
 .../lucene/analysis/miscellaneous/KeepWordFilterFactory.java     | 2 ++
 .../lucene/analysis/miscellaneous/LengthFilterFactory.java       | 2 ++
 .../lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java   | 2 ++
 .../lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java   | 2 ++
 .../analysis/miscellaneous/ProtectedTermFilterFactory.java       | 2 ++
 .../miscellaneous/RemoveDuplicatesTokenFilterFactory.java        | 2 ++
 .../apache/lucene/analysis/miscellaneous/TrimFilterFactory.java  | 2 ++
 .../analysis/miscellaneous/TypeAsSynonymFilterFactory.java       | 2 ++
 .../analysis/miscellaneous/WordDelimiterFilterFactory.java       | 2 ++
 .../org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java | 2 ++
 .../apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java  | 2 ++
 .../org/apache/lucene/analysis/ngram/NGramFilterFactory.java     | 2 ++
 .../org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java    | 2 ++
 .../java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java    | 2 ++
 .../lucene/analysis/path/PathHierarchyTokenizerFactory.java      | 2 ++
 .../lucene/analysis/pattern/PatternReplaceFilterFactory.java     | 2 ++
 .../analysis/pattern/SimplePatternSplitTokenizerFactory.java     | 2 ++
 .../lucene/analysis/pattern/SimplePatternTokenizerFactory.java   | 2 ++
 .../analysis/payloads/DelimitedPayloadTokenFilterFactory.java    | 2 ++
 .../analysis/payloads/NumericPayloadTokenFilterFactory.java      | 2 ++
 .../analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java  | 2 ++
 .../analysis/payloads/TypeAsPayloadTokenFilterFactory.java       | 2 ++
 .../java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java   | 2 ++
 .../apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java | 2 ++
 .../src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java  | 2 ++
 .../lucene/analysis/shingle/FixedShingleFilterFactory.java       | 2 ++
 .../apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java   | 2 ++
 .../org/apache/lucene/analysis/shingle/ShingleFilterFactory.java | 2 ++
 .../lucene/analysis/snowball/SnowballPorterFilterFactory.java    | 2 ++
 .../org/apache/lucene/analysis/standard/ClassicAnalyzer.java     | 2 ++
 .../apache/lucene/analysis/standard/ClassicTokenizerFactory.java | 2 ++
 .../lucene/analysis/standard/StandardTokenizerFactory.java       | 2 ++
 .../apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java   | 2 ++
 .../lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java  | 2 ++
 .../src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java  | 2 ++
 .../org/apache/lucene/analysis/synonym/SynonymFilterFactory.java | 4 +++-
 .../src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java     | 2 ++
 .../java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java | 2 ++
 .../src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java  | 2 ++
 .../java/org/apache/lucene/analysis/util/CharFilterFactory.java  | 2 ++
 .../org/apache/lucene/analysis/util/ElisionFilterFactory.java    | 2 ++
 .../java/org/apache/lucene/analysis/util/TokenFilterFactory.java | 2 ++
 .../java/org/apache/lucene/analysis/util/TokenizerFactory.java   | 2 ++
 .../lucene/analysis/wikipedia/WikipediaTokenizerFactory.java     | 2 ++
 .../java/org/apache/lucene/collation/CollationKeyAnalyzer.java   | 2 ++
 .../lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java     | 2 ++
 .../lucene/analysis/icu/segmentation/ICUTokenizerFactory.java    | 2 ++
 .../org/apache/lucene/collation/ICUCollationKeyAnalyzer.java     | 2 ++
 .../src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java | 2 ++
 .../analysis/ja/JapaneseIterationMarkCharFilterFactory.java      | 2 ++
 .../org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java  | 2 ++
 .../apache/lucene/analysis/morfologik/MorfologikAnalyzer.java    | 2 ++
 .../apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java   | 2 ++
 .../src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java   | 2 ++
 .../lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java  | 2 ++
 .../lucene/analysis/ko/KoreanReadingFormFilterFactory.java       | 2 ++
 .../org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java    | 2 ++
 .../lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java   | 2 ++
 .../apache/lucene/analysis/phonetic/PhoneticFilterFactory.java   | 2 ++
 .../lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java     | 2 ++
 .../apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java   | 2 ++
 lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java    | 2 ++
 .../src/java/org/apache/lucene/analysis/AnalyzerWrapper.java     | 2 ++
 .../org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java    | 4 +++-
 .../java/org/apache/lucene/analysis/StopwordAnalyzerBase.java    | 4 +++-
 .../org/apache/lucene/analysis/standard/StandardAnalyzer.java    | 2 ++
 .../lucene/search/suggest/document/CompletionAnalyzer.java       | 2 ++
 solr/CHANGES.txt                                                 | 2 ++
 .../org/apache/solr/analysis/ReversedWildcardFilterFactory.java  | 1 +
 solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java    | 1 +
 solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java  | 2 ++
 solr/core/src/java/org/apache/solr/schema/FieldType.java         | 2 +-
 127 files changed, 254 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java
index 61100dd..90a60a5 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicAnalyzer.java
@@ -46,7 +46,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
  *  <li>{@link ArabicStemFilter}: Arabic light stemming
  *  <li>Arabic stop words file: a set of default Arabic stop words.
  * </ul>
- * 
+ *
+ * @since 3.1
  */
 public final class ArabicAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicNormalizationFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicNormalizationFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicNormalizationFilterFactory.java
index 0568614..8921cd8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicNormalizationFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicNormalizationFilterFactory.java
@@ -34,6 +34,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.ArabicNormalizationFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ArabicNormalizationFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java
index ec8b2ed..c83ab87 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ArabicStemFilterFactory.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.ArabicStemFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ArabicStemFilterFactory extends TokenFilterFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java
index 4873b0a..7077d2b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/bg/BulgarianAnalyzer.java
@@ -36,6 +36,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
  * This analyzer implements light-stemming as specified by: <i> Searching
  * Strategies for the Bulgarian Language </i>
  * http://members.unine.ch/jacques.savoy/Papers/BUIR.pdf
+ *
+ * @since 3.1
  */
 public final class BulgarianAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java
index 1e43f4a..fe4d08a 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/bn/BengaliAnalyzer.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
 
 /**
  * Analyzer for Bengali.
+ *
+ * @since 7.1.0
  */
 public final class BengaliAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java
index 23f168d..93c565f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianAnalyzer.java
@@ -44,6 +44,8 @@ import org.apache.lucene.util.IOUtils;
  *
  * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
+ *
+ * @since 3.1
  */
 public final class BrazilianAnalyzer extends StopwordAnalyzerBase {
   /** File containing default Brazilian Portuguese stopwords. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianStemFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianStemFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianStemFilterFactory.java
index 19aa2be..2ceb363 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianStemFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/br/BrazilianStemFilterFactory.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.BrazilianStemFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class BrazilianStemFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java
index de64af7..8d90e95 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ca/CatalanAnalyzer.java
@@ -36,6 +36,8 @@ import org.tartarus.snowball.ext.CatalanStemmer;
 
 /**
  * {@link Analyzer} for Catalan.
+ *
+ * @since 3.1
  */
 public final class CatalanAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterFactory.java
index 9a237c2..8ad1488 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilterFactory.java
@@ -33,6 +33,8 @@ import java.util.regex.Pattern;
  *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class HTMLStripCharFilterFactory extends CharFilterFactory {
   final Set<String> escapedTags;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java
index d4214a1..403f43a 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/cjk/CJKAnalyzer.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
  * normalizes content with {@link CJKWidthFilter}, folds case with
  * {@link LowerCaseFilter}, forms bigrams of CJK with {@link CJKBigramFilter},
  * and filters stopwords with {@link StopFilter}
+ *
+ * @since 3.1
  */
 public final class CJKAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java
index 8e1724e..4b70886 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ckb/SoraniAnalyzer.java
@@ -36,6 +36,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Sorani Kurdish.
+ *
+ * @since 4.10.0
  */
 public final class SoraniAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
index 9712451..77ba8a8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsFilterFactory.java
@@ -37,6 +37,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.CommonGramsFilterFactory" words="commongramsstopwords.txt" ignoreCase="false"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class CommonGramsFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   // TODO: shared base class for Stop/Keep/CommonGrams? 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsQueryFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsQueryFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsQueryFilterFactory.java
index 034ed56..1523e30 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsQueryFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/commongrams/CommonGramsQueryFilterFactory.java
@@ -34,6 +34,8 @@ import org.apache.lucene.analysis.commongrams.CommonGramsQueryFilter;
  *     &lt;filter class="solr.CommonGramsQueryFilterFactory" words="commongramsquerystopwords.txt" ignoreCase="false"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class CommonGramsQueryFilterFactory extends CommonGramsFilterFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilterFactory.java
index d31cdf8..db475e3 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilterFactory.java
@@ -36,6 +36,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *         minWordSize="5" minSubwordSize="2" maxSubwordSize="15" onlyLongestMatch="true"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class DictionaryCompoundWordTokenFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   private CharArraySet dictionary;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java
index 25436f9..580a0ca 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordAnalyzer.java
@@ -22,6 +22,8 @@ import org.apache.lucene.analysis.Analyzer;
 /**
  * "Tokenizes" the entire stream as a single token. This is useful
  * for data like zip codes, ids, and some product names.
+ *
+ * @since 3.1
  */
 public final class KeywordAnalyzer extends Analyzer {
   public KeywordAnalyzer() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java
index 86f65d6..f2d84f4 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizerFactory.java
@@ -38,6 +38,8 @@ import static org.apache.lucene.analysis.standard.StandardTokenizer.MAX_TOKEN_LE
  *        MAX_TOKEN_LENGTH_LIMIT (1024*1024). It is rare to need to change this
  *      else {@link KeywordTokenizer}::DEFAULT_BUFFER_SIZE</li>
  * </ul>
+ *
+ * @since 3.1
  */
 public class KeywordTokenizerFactory extends TokenizerFactory {
   private final int maxTokenLen;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java
index 41ada68..ab5cefd 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LetterTokenizerFactory.java
@@ -40,6 +40,8 @@ import static org.apache.lucene.analysis.standard.StandardTokenizer.MAX_TOKEN_LE
  *       It is rare to need to change this
  *      else {@link CharTokenizer}::DEFAULT_MAX_TOKEN_LEN</li>
  * </ul>
+ *
+ * @since 3.1
  */
 public class LetterTokenizerFactory extends TokenizerFactory {
   private final int maxTokenLen;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java
index 91ff877..52a2090 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/LowerCaseFilterFactory.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.LowerCaseFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class LowerCaseFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java
index 3fcb92c..33095ff 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/SimpleAnalyzer.java
@@ -24,6 +24,8 @@ import org.apache.lucene.analysis.Tokenizer;
 
 /** An {@link Analyzer} that filters {@link LetterTokenizer} 
  *  with {@link LowerCaseFilter} 
+ *
+ * @since 3.1
  **/
 public final class SimpleAnalyzer extends Analyzer {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java
index dde74c0..443a46b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopAnalyzer.java
@@ -31,6 +31,8 @@ import org.apache.lucene.analysis.WordlistLoader;
 
 /** 
  * Filters {@link LetterTokenizer} with {@link LowerCaseFilter} and {@link StopFilter}.
+ *
+ * @since 3.1
  */
 public final class StopAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
index 3b1be48..3806b83 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/StopFilterFactory.java
@@ -69,6 +69,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *      for details.
  *  </li>
  * </ul>
+ *
+ * @since 3.1
  */
 public class StopFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   public static final String FORMAT_WORDSET = "wordset";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java
index 0dcaad9..2e5d505 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/UnicodeWhitespaceAnalyzer.java
@@ -21,6 +21,8 @@ import org.apache.lucene.analysis.Analyzer;
 
 /**
  * An Analyzer that uses {@link UnicodeWhitespaceTokenizer}.
+ *
+ * @since 5.4.0
  **/
 public final class UnicodeWhitespaceAnalyzer extends Analyzer {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java
index a71039a..14c3219 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceAnalyzer.java
@@ -21,6 +21,8 @@ import org.apache.lucene.analysis.Analyzer;
 
 /**
  * An Analyzer that uses {@link WhitespaceTokenizer}.
+ *
+ * @since 3.1
  **/
 public final class WhitespaceAnalyzer extends Analyzer {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java
index 29e9ed5..b04e9ac 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/core/WhitespaceTokenizerFactory.java
@@ -45,6 +45,8 @@ import static org.apache.lucene.analysis.standard.StandardTokenizer.MAX_TOKEN_LE
  *       It is rare to need to change this
  *      else {@link CharTokenizer}::DEFAULT_MAX_TOKEN_LEN</li>
  * </ul>
+ *
+ * @since 3.1
  */
 public class WhitespaceTokenizerFactory extends TokenizerFactory {
   public static final String RULE_JAVA = "java";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index f60c6a2..db5a29d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -89,6 +89,8 @@ import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassI
  *    .endwhen()
  *    .build();
  * </pre>
+ *
+ * @since 5.0.0
  */
 public final class CustomAnalyzer extends Analyzer {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java
index 17111b4..e04e944 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/cz/CzechAnalyzer.java
@@ -40,6 +40,8 @@ import org.apache.lucene.util.IOUtils;
  * all). A default set of stopwords is used unless an alternative list is
  * specified.
  * </p>
+ *
+ * @since 3.1
  */
 public final class CzechAnalyzer extends StopwordAnalyzerBase {
   /** File containing default Czech stopwords. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java
index 88fb38a..ce7a102 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/da/DanishAnalyzer.java
@@ -37,6 +37,8 @@ import org.tartarus.snowball.ext.DanishStemmer;
 
 /**
  * {@link Analyzer} for Danish.
+ *
+ * @since 3.1
  */
 public final class DanishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java
index 47db402..00438e7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanAnalyzer.java
@@ -48,6 +48,8 @@ import org.apache.lucene.util.IOUtils;
  * 
  * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
+ *
+ * @since 3.1
  */
 public final class GermanAnalyzer extends StopwordAnalyzerBase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanStemFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanStemFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanStemFilterFactory.java
index d7f578e..0112ee7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanStemFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/de/GermanStemFilterFactory.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.GermanStemFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class GermanStemFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java
index 3ed3266..99e1ac7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekAnalyzer.java
@@ -38,6 +38,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
  * 
  * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
+ *
+ * @since 3.1
  */
 public final class GreekAnalyzer extends StopwordAnalyzerBase {
   /** File containing default Greek stopwords. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java
index ce1ec08..57d542e 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/el/GreekLowerCaseFilterFactory.java
@@ -34,6 +34,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.GreekLowerCaseFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class GreekLowerCaseFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java
index 06e238e..4f69ea9 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/EnglishAnalyzer.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
 
 /**
  * {@link Analyzer} for English.
+ *
+ * @since 3.1
  */
 public final class EnglishAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/PorterStemFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/PorterStemFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/PorterStemFilterFactory.java
index 61b5662..644a263 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/PorterStemFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/en/PorterStemFilterFactory.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.PorterStemFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class PorterStemFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java
index fece249..7e17b87 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/es/SpanishAnalyzer.java
@@ -36,6 +36,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Spanish.
+ *
+ * @since 3.1
  */
 public final class SpanishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java
index f9a77fb..f5be377 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/eu/BasqueAnalyzer.java
@@ -34,6 +34,8 @@ import org.tartarus.snowball.ext.BasqueStemmer;
 
 /**
  * {@link Analyzer} for Basque.
+ *
+ * @since 3.1
  */
 public final class BasqueAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java
index 0b74309..e085897 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java
@@ -38,6 +38,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
  * zero-width non-joiner in addition to whitespace. Some persian-specific variant forms (such as farsi
  * yeh and keheh) are standardized. "Stemming" is accomplished via stopwords.
  * </p>
+ *
+ * @since 3.1
  */
 public final class PersianAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianCharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianCharFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianCharFilterFactory.java
index 69288fc..61d0e76 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianCharFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianCharFilterFactory.java
@@ -35,6 +35,8 @@ import org.apache.lucene.analysis.util.MultiTermAwareComponent;
  *     &lt;tokenizer class="solr.StandardTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class PersianCharFilterFactory extends CharFilterFactory implements MultiTermAwareComponent {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianNormalizationFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianNormalizationFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianNormalizationFilterFactory.java
index e5d78de..584a7a8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianNormalizationFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianNormalizationFilterFactory.java
@@ -35,6 +35,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.PersianNormalizationFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class PersianNormalizationFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java
index bb78934..a130e03 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishAnalyzer.java
@@ -37,6 +37,8 @@ import org.tartarus.snowball.ext.FinnishStemmer;
 
 /**
  * {@link Analyzer} for Finnish.
+ *
+ * @since 3.1
  */
 public final class FinnishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java
index bc27366..a9cdd7f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchAnalyzer.java
@@ -49,6 +49,8 @@ import org.apache.lucene.util.IOUtils;
  *
  * <p><b>NOTE</b>: This class uses the same {@link org.apache.lucene.util.Version}
  * dependent settings as {@link StandardAnalyzer}.</p>
+ *
+ * @since 3.1
  */
 public final class FrenchAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java
index 07bd825..d11ea97 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ga/IrishAnalyzer.java
@@ -35,6 +35,8 @@ import org.tartarus.snowball.ext.IrishStemmer;
 
 /**
  * {@link Analyzer} for Irish.
+ *
+ * @since 3.6.0
  */
 public final class IrishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java
index 636bff9..97071e4 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/gl/GalicianAnalyzer.java
@@ -35,6 +35,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Galician.
+ *
+ * @since 3.1
  */
 public final class GalicianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java
index c47f56d..d518fdb 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hi/HindiAnalyzer.java
@@ -33,6 +33,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
 
 /**
  * Analyzer for Hindi.
+ *
+ * @since 3.1
  */
 public final class HindiAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java
index 5e49534..30e471e 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianAnalyzer.java
@@ -37,6 +37,8 @@ import org.tartarus.snowball.ext.HungarianStemmer;
 
 /**
  * {@link Analyzer} for Hungarian.
+ *
+ * @since 3.1
  */
 public final class HungarianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java
index 7084155..7dffc67 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hy/ArmenianAnalyzer.java
@@ -34,6 +34,8 @@ import org.tartarus.snowball.ext.ArmenianStemmer;
 
 /**
  * {@link Analyzer} for Armenian.
+ *
+ * @since 3.1
  */
 public final class ArmenianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java
index f4850a5..fe6cb64 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/id/IndonesianAnalyzer.java
@@ -31,6 +31,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
 
 /**
  * Analyzer for Indonesian (Bahasa)
+ *
+ * @since 3.1
  */
 public final class IndonesianAnalyzer extends StopwordAnalyzerBase {
   /** File containing default Indonesian stopwords. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java
index 7baec96..b47645a 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/it/ItalianAnalyzer.java
@@ -38,6 +38,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Italian.
+ *
+ * @since 3.1
  */
 public final class ItalianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java
index fadb55d..3cc0dd1 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/lt/LithuanianAnalyzer.java
@@ -34,6 +34,8 @@ import org.tartarus.snowball.ext.LithuanianStemmer;
 
 /**
  * {@link Analyzer} for Lithuanian.
+ *
+ * @since 5.3.0
  */
 public final class LithuanianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java
index 0c077ad..9e663d8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/lv/LatvianAnalyzer.java
@@ -35,6 +35,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Latvian.
+ *
+ * @since 3.2
  */
 public final class LatvianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java
index 4e64abe..31e7ba2 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ASCIIFoldingFilterFactory.java
@@ -35,6 +35,8 @@ import org.apache.lucene.analysis.TokenStream;
  *     &lt;filter class="solr.ASCIIFoldingFilterFactory" preserveOriginal="false"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ASCIIFoldingFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
   private static final String PRESERVE_ORIGINAL = "preserveOriginal";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
index 9fe27ac..6296082 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConditionalTokenFilterFactory.java
@@ -30,6 +30,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
 
 /**
  * Abstract parent class for analysis factories that create {@link ConditionalTokenFilter} instances
+ *
+ * @since 7.4.0
  */
 public abstract class ConditionalTokenFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilterFactory.java
index 4df514e..a81bb98 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/HyphenatedWordsFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.HyphenatedWordsFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class HyphenatedWordsFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
index 8967c5b..b1c4bae 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/KeepWordFilterFactory.java
@@ -35,6 +35,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.KeepWordFilterFactory" words="keepwords.txt" ignoreCase="false"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class KeepWordFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   private final boolean ignoreCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilterFactory.java
index 4e1ec6b..29d789d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilterFactory.java
@@ -31,6 +31,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.LengthFilterFactory" min="0" max="1" /&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class LengthFilterFactory extends TokenFilterFactory {
   final int min;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java
index 8e6cee6..dbd7c65 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java
@@ -24,6 +24,8 @@ import org.apache.lucene.analysis.AnalyzerWrapper;
  * This Analyzer limits the number of tokens while indexing. It is
  * a replacement for the maximum field length setting inside {@link org.apache.lucene.index.IndexWriter}.
  * @see LimitTokenCountFilter
+ *
+ * @since 3.1
  */
 public final class LimitTokenCountAnalyzer extends AnalyzerWrapper {
   private final Analyzer delegate;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java
index 2d9b1da..e49a929 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.java
@@ -47,6 +47,8 @@ import java.util.Map;
  * 
  * <p>A PerFieldAnalyzerWrapper can be used like any other analyzer, for both indexing
  * and query parsing.
+ *
+ * @since 3.1
  */
 public final class PerFieldAnalyzerWrapper extends DelegatingAnalyzerWrapper {
   private final Analyzer defaultAnalyzer;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
index 1cde6c3..c528996 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ProtectedTermFilterFactory.java
@@ -75,6 +75,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * &lt;/fieldType&gt;</pre>
  *
  * <p>See related {@link org.apache.lucene.analysis.custom.CustomAnalyzer.Builder#whenTerm(Predicate)}
+ *
+ * @since 7.4.0
  */
 public class ProtectedTermFilterFactory extends ConditionalTokenFilterFactory implements ResourceLoaderAware {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/RemoveDuplicatesTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/RemoveDuplicatesTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/RemoveDuplicatesTokenFilterFactory.java
index ee0a0c9..06e8bb0 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/RemoveDuplicatesTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/RemoveDuplicatesTokenFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.RemoveDuplicatesTokenFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class RemoveDuplicatesTokenFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
index 1aad81d..eeb3da3 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
@@ -36,6 +36,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * &lt;/fieldType&gt;</pre>
  *
  * @see TrimFilter
+ *
+ * @since 3.1
  */
 public class TrimFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
index 69708b7..8487877 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
@@ -36,6 +36,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * If the optional {@code prefix} parameter is used, the specified value will be prepended
  * to the type, e.g. with prefix="_type_", for a token "example.com" with type "&lt;URL&gt;",
  * the emitted synonym will have text "_type_&lt;URL&gt;".
+ *
+ * @since 7.3.0
  */
 public class TypeAsSynonymFilterFactory extends TokenFilterFactory {
   private final String prefix;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilterFactory.java
index 0002d65..8e29ac2 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterFilterFactory.java
@@ -52,6 +52,8 @@ import static org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter.*;
  * @deprecated Use {@link WordDelimiterGraphFilterFactory} instead: it produces a correct
  * token graph so that e.g. {@link PhraseQuery} works correctly when it's used in
  * the search time analyzer.
+ *
+ * @since 3.1
  */
 @Deprecated
 public class WordDelimiterFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java
index db6a22a..efabafb 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.EdgeNGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class EdgeNGramFilterFactory extends TokenFilterFactory {
   private final int maxGramSize;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java
index fcdcd4d..4cc3d2f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/EdgeNGramTokenizerFactory.java
@@ -31,6 +31,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.EdgeNGramTokenizerFactory" minGramSize="1" maxGramSize="1"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class EdgeNGramTokenizerFactory extends TokenizerFactory {
   private final int maxGramSize;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramFilterFactory.java
index 9a681df..a04a107 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.NGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class NGramFilterFactory extends TokenFilterFactory {
   private final int maxGramSize;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java
index e50e03a..b6c55ba 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/NGramTokenizerFactory.java
@@ -33,6 +33,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.NGramTokenizerFactory" minGramSize="1" maxGramSize="2"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class NGramTokenizerFactory extends TokenizerFactory {
   private final int maxGramSize;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java
index 7112025..bc4afc7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/nl/DutchAnalyzer.java
@@ -47,6 +47,8 @@ import org.apache.lucene.util.IOUtils;
  * A default set of stopwords is used unless an alternative list is specified, but the
  * exclusion list is empty by default.
  * </p>
+ *
+ * @since 3.1
  */
 // TODO: extend StopwordAnalyzerBase
 public final class DutchAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java
index 9c74109..cb58411 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/no/NorwegianAnalyzer.java
@@ -37,6 +37,8 @@ import org.tartarus.snowball.ext.NorwegianStemmer;
 
 /**
  * {@link Analyzer} for Norwegian.
+ *
+ * @since 3.1
  */
 public final class NorwegianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizerFactory.java
index bdcc68e..aebd6bd 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizerFactory.java
@@ -66,6 +66,8 @@ import org.apache.lucene.util.AttributeFactory;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;
  * </pre>
+ *
+ * @since 3.1
  */
 public class PathHierarchyTokenizerFactory extends TokenizerFactory {
   private final char delimiter;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceFilterFactory.java
index 57e5e11..8c4572c 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceFilterFactory.java
@@ -36,6 +36,8 @@ import java.util.regex.Pattern;
  * &lt;/fieldType&gt;</pre>
  *
  * @see PatternReplaceFilter
+ *
+ * @since 3.1
  */
 public class PatternReplaceFilterFactory extends TokenFilterFactory {
   final Pattern pattern;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternSplitTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternSplitTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternSplitTokenizerFactory.java
index 4af6286..200a975 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternSplitTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternSplitTokenizerFactory.java
@@ -53,6 +53,8 @@ import org.apache.lucene.util.automaton.RegExp;
  * @lucene.experimental
  *
  * @see SimplePatternSplitTokenizer
+ *
+ * @since 6.5.0
  */
 public class SimplePatternSplitTokenizerFactory extends TokenizerFactory {
   public static final String PATTERN = "pattern";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternTokenizerFactory.java
index 3e74d02..5c73ad8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/SimplePatternTokenizerFactory.java
@@ -53,6 +53,8 @@ import org.apache.lucene.util.automaton.RegExp;
  * @lucene.experimental
  * 
  * @see SimplePatternTokenizer
+ *
+ * @since 6.5.0
  */
 public class SimplePatternTokenizerFactory extends TokenizerFactory {
   public static final String PATTERN = "pattern";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/DelimitedPayloadTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/DelimitedPayloadTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/DelimitedPayloadTokenFilterFactory.java
index cd261ac..f0bcd02 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/DelimitedPayloadTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/DelimitedPayloadTokenFilterFactory.java
@@ -33,6 +33,8 @@ import java.util.Map;
  *     &lt;filter class="solr.DelimitedPayloadTokenFilterFactory" encoder="float" delimiter="|"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class DelimitedPayloadTokenFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   public static final String ENCODER_ATTR = "encoder";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterFactory.java
index 98783b0..0d3a23f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterFactory.java
@@ -30,6 +30,8 @@ import java.util.Map;
  *     &lt;filter class="solr.NumericPayloadTokenFilterFactory" payload="24" typeMatch="word"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class NumericPayloadTokenFilterFactory extends TokenFilterFactory {
   private final float payload;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java
index 56c38da..f7d760b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TokenOffsetPayloadTokenFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.TokenOffsetPayloadTokenFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class TokenOffsetPayloadTokenFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterFactory.java
index a0c5989..3f072da 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.TypeAsPayloadTokenFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class TypeAsPayloadTokenFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java
index d880497..c87513d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseAnalyzer.java
@@ -36,6 +36,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * {@link Analyzer} for Portuguese.
+ *
+ * @since 3.1
  */
 public final class PortugueseAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
index eabdd8e..e863206 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
@@ -47,6 +47,8 @@ import org.apache.lucene.util.CharsRefBuilder;
  * a 38 million doc index which had a term in around 50% of docs and was causing TermQueries for 
  * this term to take 2 seconds.
  * </p>
+ *
+ * @since 3.1
  */
 public final class QueryAutoStopWordAnalyzer extends AnalyzerWrapper {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java
index 9429828..77208d9 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ro/RomanianAnalyzer.java
@@ -34,6 +34,8 @@ import org.tartarus.snowball.ext.RomanianStemmer;
 
 /**
  * {@link Analyzer} for Romanian.
+ *
+ * @since 3.1
  */
 public final class RomanianAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java
index f604e37..2b9af9c 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/ru/RussianAnalyzer.java
@@ -40,6 +40,8 @@ import org.apache.lucene.util.IOUtils;
  * Supports an external list of stopwords (words that
  * will not be indexed at all).
  * A default set of stopwords is used unless an alternative list is specified.
+ *
+ * @since 3.1
  */
 public final class RussianAnalyzer extends StopwordAnalyzerBase {
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
index de824b8..0176d1c 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
@@ -31,6 +31,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *   <li>tokenSeparator - how tokens should be joined together in the shingle (default: space)
  *   <li>fillerToken - what should be added in place of stop words (default: _ )
  * </ul>
+ *
+ * @since 7.4.0
  */
 public class FixedShingleFilterFactory extends TokenFilterFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
index 4729116..5d9dc44 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapper.java
@@ -26,6 +26,8 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
  * <p>
  * A shingle is another name for a token based n-gram.
  * </p>
+ *
+ * @since 3.1
  */
 public final class ShingleAnalyzerWrapper extends AnalyzerWrapper {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilterFactory.java
index c6b1519..3c86e9b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilterFactory.java
@@ -32,6 +32,8 @@ import java.util.Map;
  *             outputUnigrams="true" outputUnigramsIfNoShingles="false" tokenSeparator=" " fillerToken="_"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ShingleFilterFactory extends TokenFilterFactory {
   private final int minShingleSize;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballPorterFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballPorterFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballPorterFilterFactory.java
index d598a09..ee3876d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballPorterFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/snowball/SnowballPorterFilterFactory.java
@@ -41,6 +41,8 @@ import org.tartarus.snowball.SnowballProgram;
  *     &lt;filter class="solr.SnowballPorterFilterFactory" protected="protectedkeyword.txt" language="English"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class SnowballPorterFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   public static final String PROTECTED_TOKENS = "protected";


[20/50] [abbrv] lucene-solr:jira/http2: LUCENE-8558: Replace O(n) lookup with O(1) lookup in PerFieldMergeState#FilterFieldInfos

Posted by da...@apache.org.
LUCENE-8558: Replace O(n) lookup with O(1) lookup in PerFieldMergeState#FilterFieldInfos


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

Branch: refs/heads/jira/http2
Commit: ff1df8a15c6eaed81ad7c033014a173175383e5d
Parents: f2cb936
Author: Simon Willnauer <si...@apache.org>
Authored: Wed Nov 7 20:20:42 2018 +0100
Committer: Simon Willnauer <si...@apache.org>
Committed: Wed Nov 7 20:34:05 2018 +0100

----------------------------------------------------------------------
 .../java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ff1df8a1/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
index 291a384..99a04da 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
@@ -125,7 +125,7 @@ final class PerFieldMergeState {
       this.filteredNames = new HashSet<>(filterFields);
       this.filtered = new ArrayList<>(filterFields.size());
       for (FieldInfo fi : src) {
-        if (filterFields.contains(fi.name)) {
+        if (this.filteredNames.contains(fi.name)) {
           this.filtered.add(fi);
           hasVectors |= fi.hasVectors();
           hasProx |= fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;


[48/50] [abbrv] lucene-solr:jira/http2: SOLR-12978: In autoscaling NPE thrown for nodes where value is absent

Posted by da...@apache.org.
SOLR-12978: In autoscaling NPE thrown for nodes where value is absent


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

Branch: refs/heads/jira/http2
Commit: cd1e829732157399f7e38d810a38df3f4c2e0792
Parents: e6e6ad2
Author: Noble Paul <no...@apache.org>
Authored: Mon Nov 12 18:53:16 2018 +1100
Committer: Noble Paul <no...@apache.org>
Committed: Mon Nov 12 18:53:16 2018 +1100

----------------------------------------------------------------------
 solr/CHANGES.txt                                                   | 2 ++
 .../org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java   | 1 +
 2 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd1e8297/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6fca62d..8a0e36b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -290,6 +290,8 @@ used with DVHASH method in json.facet. (Tim Underwood via Mikhail Khludnev)
 
 * SOLR-12977: Autoscaling tries to fetch metrics from dead nodes (noble)
 
+* SOLR-12978: In autoscaling NPE thrown for nodes where value is absent (noble)
+
 Improvements
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cd1e8297/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java
index b9f6b8d..11c5ab3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/RangeVal.java
@@ -31,6 +31,7 @@ class RangeVal implements MapWriter {
   }
 
   public boolean match(Number testVal) {
+    if (testVal == null) return false;
     return Double.compare(testVal.doubleValue(), min.doubleValue()) >= 0 &&
         Double.compare(testVal.doubleValue(), max.doubleValue()) <= 0;
   }


[15/50] [abbrv] lucene-solr:jira/http2: SOLR-12971: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-12971: 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/e1bdb9a7
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e1bdb9a7
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e1bdb9a7

Branch: refs/heads/jira/http2
Commit: e1bdb9a7515e4420232d68945a0c15c20eff039b
Parents: 259925d
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 09:20:33 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 09:20:33 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e1bdb9a7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 77e8665..bd4570d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -179,6 +179,8 @@ New Features
 
 * SOLR-11997: Suggestions API/UI should show an entry where a violation could not be resolved (noble)
 
+* SOLR-12971: Add pivot Stream Evaluator to pivot facet co-occurrence counts into a matrix (Joel Bernstein)
+
 Other Changes
 ----------------------
 


[36/50] [abbrv] lucene-solr:jira/http2: SOLR-12962: Added a new 'uninvertible' option for fields and fieldtypes. This defaults to 'true' for backcompat allowing a FieldCache to be built for indexed fields as needed, but users are encouraged to set this t

Posted by da...@apache.org.
SOLR-12962: Added a new 'uninvertible' option for fields and fieldtypes. This defaults to 'true' for backcompat allowing a FieldCache to be built for indexed fields as needed, but users are encouraged to set this to false (using docValues as needed) to reduce the risk of large fluxuations in heap size due to unexpected attempts to sort/facet/function on non-docValue fields.


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

Branch: refs/heads/jira/http2
Commit: 77a4bfaa90637cd3d9a8a2ef4889e163dab143aa
Parents: 74e3ff5
Author: Chris Hostetter <ho...@apache.org>
Authored: Fri Nov 9 08:30:04 2018 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Fri Nov 9 08:30:04 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   5 +
 .../solr/handler/admin/LukeRequestHandler.java  |   2 +
 .../org/apache/solr/request/SimpleFacets.java   |   7 +
 .../org/apache/solr/schema/CurrencyField.java   |   8 +
 .../org/apache/solr/schema/FieldProperties.java |   4 +-
 .../java/org/apache/solr/schema/FieldType.java  |  11 +-
 .../org/apache/solr/schema/IndexSchema.java     |  16 +-
 .../org/apache/solr/schema/SchemaField.java     |  22 +-
 .../schema/SpatialPointVectorFieldType.java     |   8 +-
 .../solr/search/CollapsingQParserPlugin.java    |  12 +-
 .../apache/solr/search/facet/FacetField.java    |   7 +-
 .../facet/FacetFieldProcessorByArrayUIF.java    |   6 +-
 .../bad-schema-not-indexed-but-uninvertible.xml |  35 +++
 .../solr/collection1/conf/schema-behavior.xml   |  13 +
 .../test-files/solr/collection1/conf/schema.xml |  14 +-
 .../solr/collection1/conf/schema11.xml          |   3 +
 .../solr/collection1/conf/schema_latest.xml     |  12 +
 .../org/apache/solr/BasicFunctionalityTest.java |  39 +--
 .../apache/solr/request/SimpleFacetsTest.java   |  84 +++++
 .../apache/solr/request/TestFacetMethods.java   | 312 ++++++++++---------
 .../solr/rest/schema/TestBulkSchemaAPI.java     |  11 +-
 .../solr/rest/schema/TestFieldResource.java     |   4 +-
 .../solr/rest/schema/TestFieldTypeResource.java |   4 +-
 .../apache/solr/schema/BadIndexSchemaTest.java  |   1 +
 .../SchemaVersionSpecificBehaviorTest.java      |   5 +
 .../solr/search/TestCollapseQParserPlugin.java  |  28 +-
 .../solr/search/facet/TestJsonFacets.java       |  71 +++++
 solr/solr-ref-guide/src/defining-fields.adoc    |   1 +
 .../field-type-definitions-and-properties.adoc  |   1 +
 .../org/apache/solr/common/luke/FieldFlag.java  |   1 +
 .../webapp/web/js/angular/controllers/schema.js |   3 +-
 solr/webapp/web/partials/schema.html            |   7 +
 32 files changed, 562 insertions(+), 195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3c58d35..3405384 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -191,6 +191,11 @@ New Features
 
 * SOLR-12975: Add ltrim and rtrim Stream Evaluators (Joel Bernstein)
 
+* SOLR-12962: Added a new 'uninvertible' option for fields and fieldtypes. This defaults to 'true' for
+  backcompat allowing a FieldCache to be built for indexed fields as needed, but users are encouraged
+  to set this to false (using docValues as needed) to reduce the risk of large fluxuations in heap
+  size due to unexpected attempts to sort/facet/function on non-docValue fields. (hossman)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
index 4938f82..fea31ce 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
@@ -205,6 +205,7 @@ public class LukeRequestHandler extends RequestHandlerBase
     flags.append( (f != null && f.fieldType().tokenized())                   ? FieldFlag.TOKENIZED.getAbbreviation() : '-' );
     flags.append( (f != null && f.fieldType().stored())                      ? FieldFlag.STORED.getAbbreviation() : '-' );
     flags.append( (f != null && f.fieldType().docValuesType() != DocValuesType.NONE)        ? FieldFlag.DOC_VALUES.getAbbreviation() : "-" );
+    flags.append( (false)                                          ? FieldFlag.UNINVERTIBLE.getAbbreviation() : '-' ); // SchemaField Specific
     flags.append( (false)                                          ? FieldFlag.MULTI_VALUED.getAbbreviation() : '-' ); // SchemaField Specific
     flags.append( (f != null && f.fieldType().storeTermVectors())            ? FieldFlag.TERM_VECTOR_STORED.getAbbreviation() : '-' );
     flags.append( (f != null && f.fieldType().storeTermVectorOffsets())   ? FieldFlag.TERM_VECTOR_OFFSET.getAbbreviation() : '-' );
@@ -244,6 +245,7 @@ public class LukeRequestHandler extends RequestHandlerBase
     flags.append( (t != null && t.isTokenized())         ? FieldFlag.TOKENIZED.getAbbreviation() : '-' );
     flags.append( (f != null && f.stored())              ? FieldFlag.STORED.getAbbreviation() : '-' );
     flags.append( (f != null && f.hasDocValues())        ? FieldFlag.DOC_VALUES.getAbbreviation() : "-" );
+    flags.append( (f != null && f.isUninvertible())      ? FieldFlag.UNINVERTIBLE.getAbbreviation() : "-" );
     flags.append( (f != null && f.multiValued())         ? FieldFlag.MULTI_VALUED.getAbbreviation() : '-' );
     flags.append( (f != null && f.storeTermVector() )    ? FieldFlag.TERM_VECTOR_STORED.getAbbreviation() : '-' );
     flags.append( (f != null && f.storeTermOffsets() )   ? FieldFlag.TERM_VECTOR_OFFSET.getAbbreviation() : '-' );

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/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 8b2c939..a506ca1 100644
--- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
@@ -670,6 +670,13 @@ public class SimpleFacets {
        method = field.multiValued() ? FacetMethod.FC : FacetMethod.FCS;
      }
 
+     /* Unless isUninvertible() is true, we prohibit any use of UIF...
+        Here we just force FC(S) instead, and trust that the DocValues faceting logic will
+        do the right thing either way (with or w/o docvalues) */
+     if (FacetMethod.UIF == method && ! field.isUninvertible()) {
+       method = field.multiValued() ? FacetMethod.FC : FacetMethod.FCS;
+     }
+     
      /* ENUM can't deal with trie fields that index several terms per value */
      if (method == FacetMethod.ENUM
          && TrieField.getMainValuePrefix(type) != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/CurrencyField.java b/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
index e2676fe..301dba3 100644
--- a/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
+++ b/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
@@ -67,6 +67,13 @@ public class CurrencyField extends CurrencyFieldType implements SchemaAware, Res
       args.remove(PARAM_PRECISION_STEP);
     }
 
+    // NOTE: because we're not using the PluginLoader to register these field types, they aren't "real"
+    // field types and never get Schema default properties (based on schema.xml's version attribute)
+    // so only the properties explicitly set here (or on the SchemaField's we create from them) are used.
+    //
+    // In theory we should fix this, but since this class is already deprecated, we'll leave it alone
+    // to simplify the risk of back-compat break for existing users.
+    
     // Initialize field type for amount
     fieldTypeAmountRaw = new TrieLongField();
     fieldTypeAmountRaw.setTypeName(FIELD_TYPE_AMOUNT_RAW);
@@ -91,6 +98,7 @@ public class CurrencyField extends CurrencyFieldType implements SchemaAware, Res
     props.put("stored", "false");
     props.put("multiValued", "false");
     props.put("omitNorms", "true");
+    props.put("uninvertible", "true");
     int p = SchemaField.calcProps(name, type, props);
     schema.registerDynamicFields(SchemaField.create(name, type, p, null));
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
index 6762345..e0bdfd5 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
@@ -54,6 +54,7 @@ public abstract class FieldProperties {
   protected final static int STORE_TERMPAYLOADS  = 0b10000000000000000;
   protected final static int USE_DOCVALUES_AS_STORED  = 0b100000000000000000;
   protected final static int LARGE_FIELD         = 0b1000000000000000000;
+  protected final static int UNINVERTIBLE        = 0b10000000000000000000;
 
   static final String[] propertyNames = {
           "indexed", "tokenized", "stored",
@@ -61,7 +62,8 @@ public abstract class FieldProperties {
           "termVectors", "termPositions", "termOffsets",
           "multiValued",
           "sortMissingFirst","sortMissingLast","required", "omitPositions",
-          "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored", "large"
+          "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored", "large",
+          "uninvertible"
   };
 
   static final Map<String,Integer> propertyMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index ae73e09..5919c9e 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -168,7 +168,9 @@ public abstract class FieldType extends FieldProperties {
       args.remove("compressThreshold");
     }
     if (schemaVersion >= 1.6f) properties |= USE_DOCVALUES_AS_STORED;
-
+    
+    properties |= UNINVERTIBLE;
+    
     this.args = Collections.unmodifiableMap(args);
     Map<String,String> initArgs = new HashMap<>(args);
     initArgs.remove(CLASS_NAME); // consume the class arg 
@@ -456,12 +458,18 @@ public abstract class FieldType extends FieldProperties {
   }
   
   /**
+   * <p>
    * If DocValues is not enabled for a field, but it's indexed, docvalues can be constructed 
    * on the fly (uninverted, aka fieldcache) on the first request to sort, facet, etc. 
    * This specifies the structure to use.
+   * </p>
+   * <p>
+   * This method will not be used if the field is (effectively) <code>uninvertible="false"</code>
+   * </p>
    * 
    * @param sf field instance
    * @return type to uninvert, or {@code null} (to disallow uninversion for the field)
+   * @see SchemaField#isUninvertible()
    */
   public abstract UninvertingReader.Type getUninversionType(SchemaField sf);
 
@@ -1009,6 +1017,7 @@ public abstract class FieldType extends FieldProperties {
       namedPropertyValues.add(getPropertyName(STORE_OFFSETS), hasProperty(STORE_OFFSETS));
       namedPropertyValues.add(getPropertyName(MULTIVALUED), hasProperty(MULTIVALUED));
       namedPropertyValues.add(getPropertyName(LARGE_FIELD), hasProperty(LARGE_FIELD));
+      namedPropertyValues.add(getPropertyName(UNINVERTIBLE), hasProperty(UNINVERTIBLE));
       if (hasProperty(SORT_MISSING_FIRST)) {
         namedPropertyValues.add(getPropertyName(SORT_MISSING_FIRST), true);
       } else if (hasProperty(SORT_MISSING_LAST)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 1129a2c..b31f6e9 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -358,7 +358,21 @@ public class IndexSchema {
       if (sf == null) {
         return null;
       }
-      return sf.getType().getUninversionType(sf);
+
+      if (sf.isUninvertible()) {
+        return sf.getType().getUninversionType(sf);
+      }
+      // else...
+      
+      // It would be nice to throw a helpful error here, with a good useful message for the user,
+      // but unfortunately, inspite of the UninvertingReader class jdoc claims that the uninversion
+      // process is lazy, that doesn't mean it's lazy as of "When a caller attempts ot use doc values"
+      //
+      // The *mapping* function is consulted on LeafReader init/wrap for every FieldInfos found w/o docValues.
+      //
+      // So if we throw an error here instead of returning null, the act of just opening a
+      // newSearcher will trigger that error for any field, even if no one ever attempts to uninvert it
+      return null;
     };
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/SchemaField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
index e28629e..7d9449e 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -88,6 +88,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
   public FieldType getType() { return type; }
   public int getProperties() { return properties; }
 
+  public boolean isUninvertible() { return (properties & UNINVERTIBLE)!=0; }
   public boolean indexed() { return (properties & INDEXED)!=0; }
   public boolean stored() { return (properties & STORED)!=0; }
   public boolean hasDocValues() { return (properties & DOC_VALUES) != 0; }
@@ -171,18 +172,18 @@ public final class SchemaField extends FieldProperties implements IndexableField
                               + getName() + " of type: " + this.type.getTypeName());
     }
     if (! hasDocValues() ) {
-      if ( ! ( indexed() && null != this.type.getUninversionType(this) ) ) {
+      if ( ! ( indexed() && isUninvertible() && null != this.type.getUninversionType(this) ) ) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, 
-                                "can not sort on a field w/o docValues unless it is indexed and supports Uninversion: " 
+                                "can not sort on a field w/o docValues unless it is indexed=true uninvertible=true and the type supports Uninversion: " 
                                 + getName());
       }
     }
   }
 
   /** 
-   * Sanity checks that the properties of this field type are plausible 
-   * for a field that may be used to get a FieldCacheSource, throwing
-   * an appropriate exception (including the field name) if it is not.  
+   * Sanity checks that the properties of this field type are plausible for a field
+   * that may be used to get a {@link org.apache.lucene.queries.function.valuesource.FieldCacheSource},
+   * throwing an appropriate exception (including the field name) if it is not.  
    * FieldType subclasses can choose to call this method in their 
    * getValueSource implementation 
    * @see FieldType#getValueSource
@@ -194,9 +195,9 @@ public final class SchemaField extends FieldProperties implements IndexableField
                               + getName());
     }
     if (! hasDocValues() ) {
-      if ( ! ( indexed() && null != this.type.getUninversionType(this) ) ) {
+      if ( ! ( indexed() && isUninvertible() && null != this.type.getUninversionType(this) ) ) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, 
-                                "can not use FieldCache on a field w/o docValues unless it is indexed and supports Uninversion: " 
+                                "can not use FieldCache on a field w/o docValues unless it is indexed uninvertible=true and the type supports Uninversion: " 
                                 + getName());
       }
     }
@@ -247,17 +248,17 @@ public final class SchemaField extends FieldProperties implements IndexableField
 
     if (on(falseProps,INDEXED)) {
       int pp = (INDEXED 
-              | STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS | STORE_TERMPAYLOADS);
+              | STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS | STORE_TERMPAYLOADS | UNINVERTIBLE);
       if (on(pp,trueProps)) {
         throw new RuntimeException("SchemaField: " + name + " conflicting 'true' field options for non-indexed field:" + props);
       }
       p &= ~pp;
     }
     
-    if (on(falseProps,INDEXED) && on(falseProps,DOC_VALUES)) {
+    if (on(falseProps,UNINVERTIBLE) && on(falseProps,DOC_VALUES)) {
       int pp = (SORT_MISSING_FIRST | SORT_MISSING_LAST);
       if (on(pp,trueProps)) {
-        throw new RuntimeException("SchemaField: " + name + " conflicting 'true' field options for non-indexed/non-docValues field:" + props);
+        throw new RuntimeException("SchemaField: " + name + " conflicting 'true' field options for non-docValues/non-uninvertible field:" + props);
       }
       p &= ~pp;
     }
@@ -341,6 +342,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
       properties.add(getPropertyName(STORE_OFFSETS), storeOffsetsWithPositions());
       properties.add(getPropertyName(MULTIVALUED), multiValued());
       properties.add(getPropertyName(LARGE_FIELD), isLarge());
+      properties.add(getPropertyName(UNINVERTIBLE), isUninvertible());
       if (sortMissingFirst()) {
         properties.add(getPropertyName(SORT_MISSING_FIRST), sortMissingFirst());
       } else if (sortMissingLast()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
index 64e42ef..006d8d2 100644
--- a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
@@ -64,8 +64,12 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
     }
     precisionStep = ((TrieField)fieldType).getPrecisionStep();
 
-    //Just set these, delegate everything else to the field type
-    final int p = (INDEXED | TOKENIZED | OMIT_NORMS | OMIT_TF_POSITIONS);
+    // NOTE: the SchemaField constructor we're using ignores any properties of the fieldType
+    // so only the ones we're explicitly setting get used.
+    //
+    // In theory we should fix this, but since this class is already deprecated, we'll leave it alone
+    // to simplify the risk of back-compat break for existing users.
+    final int p = (INDEXED | TOKENIZED | OMIT_NORMS | OMIT_TF_POSITIONS | UNINVERTIBLE);
     List<SchemaField> newFields = new ArrayList<>();
     for( SchemaField sf : schema.getFields().values() ) {
       if( sf.getType() == this ) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index e2c86a6..53e5acd 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -392,9 +392,19 @@ public class CollapsingQParserPlugin extends QParserPlugin {
    * This is VERY fast at query time but slower to warm and causes insanity.
    */
   public static LeafReader getTopFieldCacheReader(SolrIndexSearcher searcher, String collapseField) {
+    UninvertingReader.Type type = null;
+    final SchemaField f = searcher.getSchema().getFieldOrNull(collapseField);
+    assert null != f;        // should already be enforced higher up
+    assert !f.multiValued(); // should already be enforced higher up
+    
+    assert f.getType() instanceof StrField; // this method shouldn't be called otherwise
+    if (f.indexed() && f.isUninvertible()) {
+      type = UninvertingReader.Type.SORTED;
+    }
+    
     return UninvertingReader.wrap(
         new ReaderWrapper(searcher.getSlowAtomicReader(), collapseField),
-        Collections.singletonMap(collapseField, UninvertingReader.Type.SORTED)::get);
+        Collections.singletonMap(collapseField, type)::get);
   }
 
   private static class ReaderWrapper extends FilterLeafReader {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
index 6388e74..a5ca1df 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetField.java
@@ -24,6 +24,7 @@ import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.NumberType;
 import org.apache.solr.schema.SchemaField;
 
+
 // Any type of facet request that generates a variable number of buckets
 // and the ability to sort by those generated buckets.
 abstract class FacetRequestSorted extends FacetRequest {
@@ -110,7 +111,7 @@ public class FacetField extends FacetRequestSorted {
 
     if (fcontext.facetInfo != null) {
       // refinement... we will end up either skipping the entire facet, or doing calculating only specific facet buckets
-      if (multiToken && !sf.hasDocValues() && method!=FacetMethod.DV) {
+      if (multiToken && !sf.hasDocValues() && method!=FacetMethod.DV && sf.isUninvertible()) {
         // Match the access method from the first phase.
         // It won't always matter, but does currently for an all-values bucket
         return new FacetFieldProcessorByArrayUIF(fcontext, this, sf);
@@ -118,7 +119,7 @@ public class FacetField extends FacetRequestSorted {
       return new FacetFieldProcessorByArrayDV(fcontext, this, sf);
     }
 
-      NumberType ntype = ft.getNumberType();
+    NumberType ntype = ft.getNumberType();
     // ensure we can support the requested options for numeric faceting:
     if (ntype != null) {
       if (prefix != null) {
@@ -163,7 +164,7 @@ public class FacetField extends FacetRequestSorted {
 
     // multi-valued after this point
 
-    if (sf.hasDocValues() || method == FacetMethod.DV) {
+    if (sf.hasDocValues() || method == FacetMethod.DV || !sf.isUninvertible()) {
       // single and multi-valued string docValues
       return new FacetFieldProcessorByArrayDV(fcontext, this, sf);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayUIF.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayUIF.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayUIF.java
index d5cceef..6c90b3e 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayUIF.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArrayUIF.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
@@ -22,6 +21,7 @@ import java.io.IOException;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.schema.SchemaField;
 
 /** {@link UnInvertedField} implementation of field faceting.
@@ -32,6 +32,10 @@ class FacetFieldProcessorByArrayUIF extends FacetFieldProcessorByArray {
 
   FacetFieldProcessorByArrayUIF(FacetContext fcontext, FacetField freq, SchemaField sf) {
     super(fcontext, freq, sf);
+    if (! sf.isUninvertible()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                              getClass()+" can not be used on fields where uninvertible='false'");
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test-files/solr/collection1/conf/bad-schema-not-indexed-but-uninvertible.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-not-indexed-but-uninvertible.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-not-indexed-but-uninvertible.xml
new file mode 100644
index 0000000..8f814df
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-not-indexed-but-uninvertible.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<schema name="bad-schema-not-indexed-but-uninvertible" version="1.6">
+  <fieldType name="string" class="solr.StrField"/>
+
+  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
+
+  <field name="signatureField" type="string" indexed="true" stored="false"/>
+
+  <!-- BEGIN BAD STUFF -->
+  <field name="bad_field" type="string" indexed="false" uninvertible="true" />
+  <!-- END BAD STUFF -->
+
+  <dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
+
+
+  <uniqueKey>id</uniqueKey>
+
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test-files/solr/collection1/conf/schema-behavior.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-behavior.xml b/solr/core/src/test-files/solr/collection1/conf/schema-behavior.xml
index 2fb50c7..663de18 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-behavior.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-behavior.xml
@@ -50,6 +50,9 @@
   <fieldType name="int_dvas_t" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" useDocValuesAsStored="true"/>
   <fieldType name="int_dvas_f" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" useDocValuesAsStored="false"/>
 
+  <fieldType name="str_uninvert_f" class="solr.StrField" uninvertible="false"/>
+  <fieldType name="str_uninvert_t" class="solr.StrField" uninvertible="true"/>
+  
   <!-- all behavior is default -->
 
   <field name="text" type="text"/>
@@ -98,6 +101,11 @@
   <dynamicField name="*_dyn_ft_intdvas_t" type="int_dvas_t"/>
   <dynamicField name="*_dyn_ft_intdvas_f" type="int_dvas_f"/>
 
+  <field name="ft_uninvert_t" type="str_uninvert_t"/>
+  <field name="ft_uninvert_f" type="str_uninvert_f"/>
+  <dynamicField name="*_dyn_ft_uninvert_t" type="str_uninvert_t"/>
+  <dynamicField name="*_dyn_ft_uninvert_f" type="str_uninvert_f"/>
+
   <!-- explicit props on field -->
   <field name="multi_f" type="str" multiValued="false"/>
   <field name="multi_t" type="str" multiValued="true"/>
@@ -128,5 +136,10 @@
   <field name="intdvas_f" type="int" useDocValuesAsStored="false"/>
   <dynamicField name="*_dyn_intdvas_t" type="int" useDocValuesAsStored="true"/>
   <dynamicField name="*_dyn_intdvas_f" type="int" useDocValuesAsStored="false"/>
+  
+  <field name="uninvert_t" type="str" uninvertible="true"/>
+  <field name="uninvert_f" type="str" uninvertible="false"/>
+  <dynamicField name="*_dyn_uninvert_t" type="str" uninvertible="true"/>
+  <dynamicField name="*_dyn_uninvert_f" type="str" uninvertible="false"/>
 
 </schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test-files/solr/collection1/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml
index b61bbb1..acdf7cf 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml
@@ -638,6 +638,8 @@
 
   <field name="payloadDelimited" type="payloadDelimited"/>
 
+  <field name="sortabuse_not_uninvertible" type="string" indexed="true" multiValued="false" uninvertible="false" />
+  
   <!-- EnumType -->
   <field name="severity" type="severityType" docValues="true" indexed="true" stored="true" multiValued="false"/>
 
@@ -753,7 +755,13 @@
   <dynamicField name="*_dd_dvo" multiValued="true" type="double" docValues="true" indexed="false" stored="false"
                 useDocValuesAsStored="true"/>
                 
-                
+  <!-- Indexed, but NOT uninvertible -->
+  <dynamicField name="*_s_not_uninvert" type="string" indexed="true" stored="false" docValues="false" uninvertible="false" />
+  <!-- docValues, but NOT uninvertible -->
+  <dynamicField name="*_s_not_uninvert_dv" type="string" indexed="true" stored="false" docValues="true" uninvertible="false" />
+
+
+  
   <!-- Only Stored numerics -->
   <dynamicField name="*_i_os" type="int" indexed="false" stored="true" docValues="false"/>
   <dynamicField name="*_l_os" type="long" indexed="false" stored="true" docValues="false"/>
@@ -823,6 +831,10 @@
   <copyField source="range_facet_f" dest="range_facet_d_dv"/>
   <copyField source="bday" dest="range_facet_dt_dv"/>
 
+  <copyField source="trait_s" dest="trait_s_not_uninvert"/>
+  <copyField source="trait_s" dest="trait_s_not_uninvert_dv"/>
+  <copyField source="trait_s" dest="trait_s_not_indexed_sS"/>
+  
   <!-- dynamic destination -->
   <copyField source="*_dynamic" dest="dynamic_*"/>
   <copyField source="*_path" dest="*_ancestor"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index d3344a2..0dff57f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -455,6 +455,9 @@ valued. -->
    <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
    <dynamicField name="*_ws" type="text_ws" indexed="true"  stored="true"/>
    
+  <!-- Indexed, but NOT uninvertible -->
+  <dynamicField name="*_s_not_uninvert" type="string" indexed="true" stored="false" docValues="false" uninvertible="false" />
+   
    <!-- for testing tfidf functions, see TestFunctionQuery.testTFIDFFunctions -->
    <dynamicField name="*_tfidf"  type="tfidf_text"    indexed="true"  stored="true" />
    <fieldType name="tfidf_text" class="solr.TextField" positionIncrementGap="100">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
index 70abb39..78d5766 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
@@ -292,6 +292,12 @@
   <field name="signatureField" type="string" indexed="true" stored="false"/>
   <dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
 
+  <!-- Indexed, but NOT uninvertible -->
+  <field name="where_s_multi_not_uninvert" type="string" indexed="true" stored="false" docValues="false" uninvertible="false" multiValued="true" />
+  <field name="where_s_single_not_uninvert" type="string" indexed="true" stored="false" docValues="false" uninvertible="false" multiValued="false" />
+  <!-- docValues, but NOT uninvertible -->
+  <field name="where_s_multi_not_uninvert_dv" type="string" indexed="true" stored="false" docValues="true" uninvertible="false" multiValued="true" />
+  <field name="where_s_single_not_uninvert_dv" type="string" indexed="true" stored="false" docValues="true" uninvertible="false" multiValued="false" />
 
   <!-- Field to use to determine and enforce document uniqueness. 
        Unless this field is marked with required="false", it will be a required field
@@ -324,6 +330,12 @@
 
   <!-- Create a string version of author for faceting -->
   <copyField source="author" dest="author_s"/>
+  
+  <copyField source="where_s" dest="where_s_multi_not_uninvert"/>
+  <copyField source="where_s" dest="where_s_multi_not_uninvert_dv"/>
+  <copyField source="where_s" dest="where_s_single_not_uninvert"/>
+  <copyField source="where_s" dest="where_s_single_not_uninvert_dv"/>
+  <copyField source="where_s" dest="where_s_not_indexed_sS"/>
 
   <!-- Above, multiple source fields are copied to the [text] field. 
    Another way to map multiple source fields to the same 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
index fc69509..51abc0a 100644
--- a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
+++ b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
@@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -989,30 +990,32 @@ public class BasicFunctionalityTest extends SolrTestCaseJ4 {
   public void testAbuseOfSort() {
 
     assertU(adoc("id", "9999991",
-                 "sortabuse_b", "true",
+                 "sortabuse_not_uninvertible", "xxx",
                  "sortabuse_t", "zzz xxx ccc vvv bbb nnn aaa sss ddd fff ggg"));
     assertU(adoc("id", "9999992",
-                 "sortabuse_b", "true",
+                 "sortabuse_not_uninvertible", "yyy",
                  "sortabuse_t", "zzz xxx ccc vvv bbb nnn qqq www eee rrr ttt"));
 
     assertU(commit());
 
-    RuntimeException outerEx = expectThrows(RuntimeException.class, () -> {
-      ignoreException("can not sort on multivalued field: sortabuse_t");
-      assertQ("sort on something that shouldn't work",
-          req("q", "sortabuse_b:true",
-              "sort", "sortabuse_t asc"),
-          "*[count(//doc)=2]");
-    });
-    Throwable root = getRootCause(outerEx);
-    assertEquals("sort exception root cause",
-        SolrException.class, root.getClass());
-    SolrException e = (SolrException) root;
-    assertEquals("incorrect error type",
-        SolrException.ErrorCode.BAD_REQUEST,
-        SolrException.ErrorCode.getErrorCode(e.code()));
-    assertTrue("exception doesn't contain field name",
-        e.getMessage().contains("sortabuse_t"));
+    for (String f : Arrays.asList("sortabuse_not_uninvertible", "sortabuse_t")) {
+      RuntimeException outerEx = expectThrows(RuntimeException.class, () -> {
+          ignoreException("sortabuse");
+          assertQ("sort on something that shouldn't work",
+                  req("q", "*:*",
+                      "sort", f+ " asc"),
+                  "*[count(//doc)=2]");
+        });
+      Throwable root = getRootCause(outerEx);
+      assertEquals("sort exception root cause",
+                   SolrException.class, root.getClass());
+      SolrException e = (SolrException) root;
+      assertEquals("incorrect error type",
+                   SolrException.ErrorCode.BAD_REQUEST,
+                   SolrException.ErrorCode.getErrorCode(e.code()));
+      assertTrue("exception doesn't contain field name",
+                 e.getMessage().contains(f));
+    }
   }
 
 //   /** this doesn't work, but if it did, this is how we'd test it. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/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 e3100e2..6667d89 100644
--- a/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
+++ b/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
 import org.apache.solr.common.params.FacetParams.FacetRangeMethod;
 import org.apache.solr.common.params.FacetParams.FacetRangeOther;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.NumberType;
@@ -842,6 +843,89 @@ public class SimpleFacetsTest extends SolrTestCaseJ4 {
     );
   }
 
+  public void testBehaviorEquivilenceOfUninvertibleFalse() throws Exception {
+    // NOTE: mincount=0 affects method detection/coercion, so we include permutations of it
+    
+    { 
+      // an "uninvertible=false" field is not be facetable using the "default" method,
+      // or any explicit method other then "enum".
+      //
+      // it should behave the same as any attempt (using any method) at faceting on
+      // and "indexed=false docValues=false" field -- returning no buckets.
+      
+      final List<SolrParams> paramSets = new ArrayList<>();
+      for (String min : Arrays.asList("0", "1")) {
+        for (String f : Arrays.asList("trait_s_not_uninvert", "trait_s_not_indexed_sS")) {
+          paramSets.add(params("facet.field", "{!key=x}" + f));
+          for (String method : Arrays.asList("fc", "fcs", "uif")) {
+            paramSets.add(params("facet.field", "{!key=x}" + f,
+                                 "facet.mincount", min,
+                                 "facet.method", method));
+            paramSets.add(params("facet.field", "{!key=x}" + f,
+                                 "facet.mincount", min,
+                                 "facet.method", method));
+          }
+        }
+        paramSets.add(params("facet.field", "{!key=x}trait_s_not_indexed_sS",
+                             "facet.mincount", min,
+                             "facet.method", "enum"));
+      }
+      for (SolrParams p : paramSets) {
+        // "empty" results should be the same regardless of mincount
+        assertQ("expect no buckets when field is not-indexed or not-uninvertible",
+                req(p
+                    ,"rows","0"
+                    ,"q", "id_i1:[42 TO 47]"
+                    ,"fq", "id_i1:[42 TO 45]"
+                    ,"facet", "true"
+                    )
+                ,"//*[@numFound='4']"
+                ,"*[count(//lst[@name='x'])=1]"
+                ,"*[count(//lst[@name='x']/int)=0]"
+                );
+      }
+      
+    }
+    
+    { 
+      // the only way to facet on an "uninvertible=false" field is to explicitly request facet.method=enum
+      // in which case it should behave consistently with it's copyField source & equivilent docValues field
+      // (using any method for either of them)
+
+      final List<SolrParams> paramSets = new ArrayList<>();
+      for (String min : Arrays.asList("0", "1")) {
+        paramSets.add(params("facet.field", "{!key=x}trait_s_not_uninvert",
+                             "facet.method", "enum"));
+        for (String okField : Arrays.asList("trait_s", "trait_s_not_uninvert_dv")) {
+          paramSets.add(params("facet.field", "{!key=x}" + okField));
+          for (String method : Arrays.asList("enum","fc", "fcs", "uif")) {
+            paramSets.add(params("facet.field", "{!key=x}" + okField,
+                                 "facet.method", method));
+          }
+        }
+        for (SolrParams p : paramSets) {
+          assertQ("check counts for applied facet queries using filtering (fq)",
+                  req(p
+                      ,"rows","0"
+                      ,"q", "id_i1:[42 TO 47]"
+                      ,"fq", "id_i1:[42 TO 45]"
+                      ,"facet", "true"
+                      ,"facet.mincount", min
+                      )
+                  ,"//*[@numFound='4']"
+                  ,"*[count(//lst[@name='x'])=1]"
+                  ,"*[count(//lst[@name='x']/int)="+("0".equals(min) ? "4]" : "3]")
+                  ,"//lst[@name='x']/int[@name='Tool'][.='2']"
+                  ,"//lst[@name='x']/int[@name='Obnoxious'][.='1']"
+                  ,"//lst[@name='x']/int[@name='Chauvinist'][.='1']"
+                  ,"count(//lst[@name='x']/int[@name='Pig'][.='0'])=" + ("0".equals(min) ? "1" : "0")
+                  );
+        }
+      }
+    }
+  }
+
+  
   public static void indexDateFacets() {
     final String i = "id";
     final String f = "bday";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/request/TestFacetMethods.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/request/TestFacetMethods.java b/solr/core/src/test/org/apache/solr/request/TestFacetMethods.java
index fa23494..e70083d 100644
--- a/solr/core/src/test/org/apache/solr/request/TestFacetMethods.java
+++ b/solr/core/src/test/org/apache/solr/request/TestFacetMethods.java
@@ -17,8 +17,7 @@
 
 package org.apache.solr.request;
 
-import static junit.framework.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import java.util.Arrays;
 
 import org.apache.solr.request.SimpleFacets.FacetMethod;
 import org.apache.solr.schema.BoolField;
@@ -26,201 +25,214 @@ import org.apache.solr.schema.IntPointField;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.StrField;
 import org.apache.solr.schema.TrieIntField;
+import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
-public class TestFacetMethods {
+public class TestFacetMethods extends LuceneTestCase {
 
   // TODO - make these public in FieldProperties?
   protected final static int MULTIVALUED         = 0x00000200;
   protected final static int DOC_VALUES          = 0x00008000;
+  protected final static int UNINVERTIBLE        = 0b10000000000000000000;
+
+  protected static boolean propsMatch( int x, int y ) {
+    return (x & y) != 0;
+  }
 
   @Test
   public void testNumericSingleValuedDV() {
 
-    SchemaField field = new SchemaField("field", new TrieIntField(), DOC_VALUES, null);
-
-    // default is FCS, can't use ENUM due to trie-field terms, FC rewrites to FCS for efficiency
-
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-
+    for (int props : Arrays.asList(DOC_VALUES ^ UNINVERTIBLE,
+                                   DOC_VALUES)) {
+      SchemaField field = new SchemaField("field", new TrieIntField(), props, null);
+      // default is FCS, can't use ENUM due to trie-field terms, FC rewrites to FCS for efficiency
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+        
+        // UIF only allowed if field is UNINVERTIBLE
+        assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FCS,
+                     SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 0));
+      }
+    }
   }
 
   @Test
   public void testNumericMultiValuedDV() {
 
-    SchemaField field = new SchemaField("field", new TrieIntField(), DOC_VALUES ^ MULTIVALUED, null);
-
-    // default is FC, can't use ENUM due to trie-field terms, can't use FCS because of multivalues
-
-    // default value is FC
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-
+    for (int props : Arrays.asList(DOC_VALUES ^ MULTIVALUED ^ UNINVERTIBLE,
+                                   DOC_VALUES ^ MULTIVALUED)) {
+      SchemaField field = new SchemaField("field", new TrieIntField(), props, null);
+      // default value is FC
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        
+        // UIF only allowed if field is UNINVERTIBLE
+        assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FC,
+                     SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, mincount));
+      }
+    }
   }
 
   @Test
   public void testNumericSingleValuedNoDV() {
 
-    SchemaField field = new SchemaField("field", new TrieIntField(), 0, null);
-
-    // only works with FCS for mincount = 0, UIF for count > 0 is fine
-
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-
+    for (int props : Arrays.asList(0 ^ UNINVERTIBLE,
+                                   0)) {
+      SchemaField field = new SchemaField("field", new TrieIntField(), props, null);
+      // FCS is used by default for most requested methods other then UIF -- regardless of mincount
+      for (int mincount : Arrays.asList(0, 1)) {
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+      }
+      // UIF allowed only if UNINVERTIBLE *AND* mincount > 0
+      assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 0));
+      assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FCS,
+                   SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 1));
+    }
   }
 
   @Test
   public void testNumericMultiValuedNoDV() {
 
-    SchemaField field = new SchemaField("field", new TrieIntField(), MULTIVALUED, null);
-
-    // only works with FC for mincount = 0, UIF for count > 1 is fine
-
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-
+    for (int props : Arrays.asList(MULTIVALUED ^ UNINVERTIBLE,
+                                   MULTIVALUED)) {
+      SchemaField field = new SchemaField("field", new TrieIntField(), props, null);
+      // FC is used by default for most requested methods other then UIF -- regardless of mincount
+      for (int mincount : Arrays.asList(0, 1)) {
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+      }
+      // UIF allowed only if UNINVERTIBLE *AND* mincount > 0
+      assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 0));
+      assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FC,
+                   SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 1));
+    }
   }
 
   @Test
-  public void testTextSingleValuedDV() {
-
-    SchemaField field = new SchemaField("field", new StrField(), DOC_VALUES, null);
-
-    // default is FC, otherwise just uses the passed-in method
-
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-
+  public void testStringSingleValuedDV() {
+
+    for (int props : Arrays.asList(DOC_VALUES ^ UNINVERTIBLE,
+                                   DOC_VALUES)) {
+      SchemaField field = new SchemaField("field", new StrField(), props, null);
+      // default is FC, otherwise just uses the passed-in method as is unless UIF...
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+        // UIF only allowed if field is UNINVERTIBLE
+        assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FCS,
+                     SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, mincount));
+      }
+    }
   }
 
   @Test
-  public void testTextMultiValuedDV() {
-
-    SchemaField field = new SchemaField("field", new StrField(), DOC_VALUES ^ MULTIVALUED, null);
-
-    // default is FC, can't use FCS because of multivalues
-
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-
+  public void testStringMultiValuedDV() {
+
+    for (int props : Arrays.asList(MULTIVALUED ^ DOC_VALUES ^ UNINVERTIBLE,
+                                   MULTIVALUED ^ DOC_VALUES)) {
+      SchemaField field = new SchemaField("field", new StrField(), props, null);
+      // default is FC, can't use FCS because of multivalues...
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+        // UIF only allowed if field is UNINVERTIBLE
+        assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FC,
+                     SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, mincount));
+      }
+    }
   }
 
   @Test
-  public void testTextSingleValuedNoDV() {
-
-    SchemaField field = new SchemaField("field", new StrField(), 0, null);
-
-    // default is FC, UIF rewrites to FCS for mincount = 0
-    // TODO should it rewrite to FC instead?
-
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-
+  public void testStringSingleValuedNoDV() {
+
+    for (int props : Arrays.asList(0 ^ UNINVERTIBLE,
+                                   0)) {
+      SchemaField field = new SchemaField("field", new StrField(), props, null);
+      // default is FC, otherwise just uses the passed-in method as is unless UIF...
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+      }
+      // UIF allowed only if UNINVERTIBLE *AND* mincount > 0
+      assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 0));
+      assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FCS,
+                   SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 1));
+    }
   }
 
   @Test
-  public void testTextMultiValuedNoDV() {
-
-    SchemaField field = new SchemaField("field", new StrField(), MULTIVALUED, null);
-
-    // default is FC, can't use FCS for multivalued fields, UIF rewrites to FC for mincount = 0
-
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, 1));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.ENUM, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FCS, 1));
-    assertEquals(SimpleFacets.FacetMethod.UIF, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.UIF, 1));
-    assertEquals(SimpleFacets.FacetMethod.FC, SimpleFacets.selectFacetMethod(field, SimpleFacets.FacetMethod.FC, 1));
-
+  public void testStringMultiValuedNoDV() {
+
+    for (int props : Arrays.asList(MULTIVALUED ^ UNINVERTIBLE,
+                                   MULTIVALUED)) {
+      SchemaField field = new SchemaField("field", new StrField(), props, null);
+      // default is FC, can't use FCS because of multivalues...
+      for (int mincount : Arrays.asList(0, 1)) {
+        // behavior should be independent of mincount
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+      }
+      // UIF allowed only if UNINVERTIBLE *AND* mincount > 0
+      assertEquals(FacetMethod.FC, SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 0));
+      assertEquals(propsMatch(props, UNINVERTIBLE) ? FacetMethod.UIF : FacetMethod.FC,
+                   SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, 1));
+    }
   }
 
   @Test
   public void testBooleanDefaults() {
 
     // BoolField defaults to ENUM
-
-    SchemaField field = new SchemaField("field", new BoolField(), 0, null);
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, null, 1));
-
+    for (int props : Arrays.asList(0 ^ UNINVERTIBLE,
+                                   0)) {
+      SchemaField field = new SchemaField("field", new BoolField(), props, null);
+      assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, null, 0));
+      assertEquals(SimpleFacets.FacetMethod.ENUM, SimpleFacets.selectFacetMethod(field, null, 1));
+    }
   }
   
   @Test
   public void testPointFields() {
     // Methods other than FCS are not currently supported for PointFields
-    SchemaField field = new SchemaField("foo", new IntPointField());
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, 0));
-    field = new SchemaField("fooMV", new IntPointField(), 0x00000200, "0"); //MultiValued
-    assertTrue(field.multiValued());
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, 0));
-    assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, 0));
+    for (int props : Arrays.asList(MULTIVALUED ^ DOC_VALUES ^ UNINVERTIBLE,
+                                   MULTIVALUED ^ DOC_VALUES,
+                                   MULTIVALUED ^ UNINVERTIBLE,
+                                   UNINVERTIBLE,
+                                   MULTIVALUED,
+                                   DOC_VALUES,
+                                   0)) {
+      SchemaField field = new SchemaField("foo", new IntPointField(), props, null);
+      for (int mincount : Arrays.asList(0, 1)) {
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, mincount));
+        assertEquals(FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.UIF, mincount));
+      }
+    }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
index 3cc07c7..5d1dab1 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
@@ -390,13 +390,15 @@ public class TestBulkSchemaAPI extends RestTestBase {
         "                       'name':'a2',\n" +
         "                       'type': 'string',\n" +
         "                       'stored':true,\n" +
-        "                       'indexed':true\n" +
+        "                       'indexed':true,\n" +
+        "                       'uninvertible':true,\n" +
         "                       },\n" +
         "          'add-dynamic-field' : {\n" +
         "                       'name' :'*_lol',\n" +
         "                       'type':'string',\n" +
         "                       'stored':true,\n" +
-        "                       'indexed':true\n" +
+        "                       'indexed':true,\n" +
+        "                       'uninvertible':false,\n" +
         "                       },\n" +
         "          'add-copy-field' : {\n" +
         "                       'source' :'a1',\n" +
@@ -470,6 +472,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
         "          'add-field-type' : {" +
         "                       'name' : 'myWhitespaceTxtField',\n" +
         "                       'class':'solr.TextField',\n" +
+        "                       'uninvertible':false,\n" +
         "                       'analyzer' : {'class' : 'org.apache.lucene.analysis.core.WhitespaceAnalyzer'}\n" +
         "                       },\n"+
         "          'add-field' : {\n" +
@@ -532,6 +535,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
     assertEquals("string", m.get("type"));
     assertEquals(Boolean.TRUE, m.get("stored"));
     assertEquals(Boolean.TRUE, m.get("indexed"));
+    assertEquals(Boolean.TRUE, m.get("uninvertible"));
 
     m = getObj(harness,"*_lol", "dynamicFields");
     assertNotNull("field *_lol not created", m);
@@ -539,6 +543,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
     assertEquals("string", m.get("type"));
     assertEquals(Boolean.TRUE, m.get("stored"));
     assertEquals(Boolean.TRUE, m.get("indexed"));
+    assertEquals(Boolean.FALSE, m.get("uninvertible"));
 
     l = getSourceCopyFields(harness, "a1");
     s = new HashSet();
@@ -579,11 +584,13 @@ public class TestBulkSchemaAPI extends RestTestBase {
     
     m = getObj(harness, "myWhitespaceTxtField", "fieldTypes");
     assertNotNull(m);
+    assertEquals(Boolean.FALSE, m.get("uninvertible"));
     assertNull(m.get("similarity")); // unspecified, expect default
 
     m = getObj(harness, "a5", "fields");
     assertNotNull("field a5 not created", m);
     assertEquals("myWhitespaceTxtField", m.get("type"));
+    assertNull(m.get("uninvertible")); // inherited, but API shouldn't return w/o explicit showDefaults
     assertFieldSimilarity("a5", BM25Similarity.class); // unspecified, expect default
 
     m = getObj(harness, "wdf_nocase", "fields");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
index 4f53609..81620ed 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
@@ -23,11 +23,12 @@ public class TestFieldResource extends SolrRestletTestBase {
   public void testGetField() throws Exception {
     assertQ("/schema/fields/test_postv?indent=on&wt=xml&showDefaults=true",
             "count(/response/lst[@name='field']) = 1",
-            "count(/response/lst[@name='field']/*) = 18",
+            "count(/response/lst[@name='field']/*) = 19",
             "/response/lst[@name='field']/str[@name='name'] = 'test_postv'",
             "/response/lst[@name='field']/str[@name='type'] = 'text'",
             "/response/lst[@name='field']/bool[@name='indexed'] = 'true'",
             "/response/lst[@name='field']/bool[@name='stored'] = 'true'",
+            "/response/lst[@name='field']/bool[@name='uninvertible'] = 'true'",
             "/response/lst[@name='field']/bool[@name='docValues'] = 'false'",
             "/response/lst[@name='field']/bool[@name='termVectors'] = 'true'",
             "/response/lst[@name='field']/bool[@name='termPositions'] = 'true'",
@@ -59,6 +60,7 @@ public class TestFieldResource extends SolrRestletTestBase {
              "/field/type=='text'",
              "/field/indexed==true",
              "/field/stored==true",
+             "/field/uninvertible==true",
              "/field/docValues==false",
              "/field/termVectors==true",
              "/field/termPositions==true",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
index 08a3f1b..0e4fe7b 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
@@ -26,12 +26,13 @@ public class TestFieldTypeResource extends SolrRestletTestBase {
     final boolean expectedDocValues = Boolean.getBoolean(NUMERIC_DOCVALUES_SYSPROP);
     assertQ("/schema/fieldtypes/float?wt=xml&showDefaults=true",
             "count(/response/lst[@name='fieldType']) = 1",
-            "count(/response/lst[@name='fieldType']/*) = 17",
+            "count(/response/lst[@name='fieldType']/*) = 18",
             "/response/lst[@name='fieldType']/str[@name='name'] = 'float'",
             "/response/lst[@name='fieldType']/str[@name='class'] = '"+expectedFloatClass+"'",
             "/response/lst[@name='fieldType']/str[@name='precisionStep'] ='0'",
             "/response/lst[@name='fieldType']/bool[@name='indexed'] = 'true'",
             "/response/lst[@name='fieldType']/bool[@name='stored'] = 'true'",
+            "/response/lst[@name='fieldType']/bool[@name='uninvertible'] = 'true'",
             "/response/lst[@name='fieldType']/bool[@name='docValues'] = '"+expectedDocValues+"'",
             "/response/lst[@name='fieldType']/bool[@name='termVectors'] = 'false'",
             "/response/lst[@name='fieldType']/bool[@name='termPositions'] = 'false'",
@@ -63,6 +64,7 @@ public class TestFieldTypeResource extends SolrRestletTestBase {
              "/fieldType/precisionStep=='0'",
              "/fieldType/indexed==true",
              "/fieldType/stored==true",
+             "/fieldType/uninvertible==true",
              "/fieldType/docValues=="+expectedDocValues,
              "/fieldType/termVectors==false",
              "/fieldType/termPositions==false",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java b/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
index b9dc1aa..c829c17 100644
--- a/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
@@ -29,6 +29,7 @@ public class BadIndexSchemaTest extends AbstractBadConfigTestBase {
     doTest("bad-schema-not-indexed-but-norms.xml", "bad_field");
     doTest("bad-schema-not-indexed-but-tf.xml", "bad_field");
     doTest("bad-schema-not-indexed-but-pos.xml", "bad_field");
+    doTest("bad-schema-not-indexed-but-uninvertible.xml", "bad_field");
     doTest("bad-schema-omit-tf-but-not-pos.xml", "bad_field");
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java b/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java
index cf34e95..67a6291 100644
--- a/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/SchemaVersionSpecificBehaviorTest.java
@@ -64,6 +64,11 @@ public class SchemaVersionSpecificBehaviorTest extends SolrTestCaseJ4 {
           assertEquals(f + " field's type has wrong useDocValuesAsStored for ver=" + ver,
                        ( v < 1.6F ? false : true), 
                        field.useDocValuesAsStored());
+          
+          // uninvertable defaults to true (for now)
+          assertEquals(f + " field's type has wrong uninvertable for ver=" + ver,
+                       true,
+                       field.isUninvertible());
         }
 
         // regardless of version, explicit multiValued values on field or type 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
index 7a5fd8d..acdd599 100644
--- a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
@@ -809,6 +809,15 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
     params.add("q", "*:*");
     params.add("fq", "{!collapse field="+group+" "+optional_min_or_max+"}");
     assertQ(req(params), "*[count(//doc)=0]");
+
+    // if a field is uninvertible=false, it should behave the same as a field that is indexed=false
+    // this is currently ok on fields that don't exist on any docs in the index
+    for (String f : Arrays.asList("not_indexed_sS", "indexed_s_not_uninvert")) {
+      for (String hint : Arrays.asList("", " hint=top_fc")) {
+        assertQ(req(params("q", "*:*", "fq", "{!collapse field="+f+hint+"}"))
+                , "*[count(//doc)=0]");
+      }
+    }
   }
 
   public void testNoDocsHaveGroupField() throws Exception {
@@ -918,7 +927,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
 
   @Test
   public void testForNotSupportedCases() {
-    String[] doc = {"id","3", "term_s", "YYYY", "test_ii", "5000", "test_l", "100", "test_f", "200"};
+    String[] doc = {"id","3", "term_s", "YYYY", "test_ii", "5000", "test_l", "100", "test_f", "200",
+                    "not_indexed_sS", "zzz", "indexed_s_not_uninvert", "zzz"};
     assertU(adoc(doc));
     assertU(commit());
 
@@ -930,6 +940,22 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
     assertQEx("Should Fail with Bad Request", "org.apache.solr.search.SyntaxError: undefined field: \"bleh\"",
         req("q","*:*", "fq","{!collapse field=bleh}"), SolrException.ErrorCode.BAD_REQUEST);
 
+    // if a field is uninvertible=false, it should behave the same as a field that is indexed=false ...
+    for (String f : Arrays.asList("not_indexed_sS", "indexed_s_not_uninvert")) {
+      { // this currently propogates up the low level DocValues error in the common case...
+        Exception e = expectThrows(RuntimeException.class, IllegalStateException.class,
+                                    () -> h.query(req(params("q", "*:*",
+                                                             "fq", "{!collapse field="+f+"}"))));
+        assertTrue("unexpected Message: " + e.getMessage(),
+                   e.getMessage().contains("Re-index with correct docvalues type"));
+      }
+      { // ... but in the case of hint=top_fc a bare NPE gets propogated up (SOLR-12979)...
+        expectThrows(RuntimeException.class, NullPointerException.class, 
+                     () -> h.query(req(params("q", "*:*",
+                                              "fq", "{!collapse field="+f+" hint=top_fc}"))));
+      }
+      
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/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 01d29b7..e21c6d8 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
@@ -37,7 +37,9 @@ import org.apache.solr.SolrTestCaseHS;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.macro.MacroExpander;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -212,6 +214,75 @@ public class TestJsonFacets extends SolrTestCaseHS {
     client.commit();
   }
 
+  public void testBehaviorEquivilenceOfUninvertibleFalse() throws Exception {
+    Client client = Client.localClient();
+    indexSimple(client);
+
+    // regardless of the facet method (parameterized via default at test class level)
+    // faceting on an "uninvertible=false docValues=false" field is not supported.
+    //
+    // it should behave the same as any attempt (using any method) at faceting on
+    // and "indexed=false docValues=false" field...
+    for (String f : Arrays.asList("where_s_not_indexed_sS",
+                                  "where_s_multi_not_uninvert",
+                                  "where_s_single_not_uninvert")) {
+      SolrQueryRequest request = req("rows", "0", "q", "num_i:[* TO 2]", "json.facet",
+                                     "{x: {type:terms, field:'"+f+"'}}");
+      if (FacetField.FacetMethod.DEFAULT_METHOD == FacetField.FacetMethod.DVHASH
+          && !f.contains("multi")) {
+        // DVHASH is (currently) weird...
+        //
+        // it's ignored for multi valued fields -- but for single valued fields, it explicitly
+        // checks the *FieldInfos* on the reader to see if the DocVals type is ok.
+        //
+        // Which means that unlike most other facet method:xxx options, it fails hard if you try to use it
+        // on a field where no docs have been indexed (yet).
+        expectThrows(SolrException.class, () ->{
+            assertJQ(request);
+          });
+        
+      } else {
+        // In most cases, we should just get no buckets back...
+        assertJQ(request
+                 , "response/numFound==3"
+                 , "facets/count==3"
+                 , "facets/x=={buckets:[]}"
+
+                 );
+      }
+    }
+
+    // regardless of the facet method (parameterized via default at test class level)
+    // faceting on an "uninvertible=false docValues=true" field should work,
+    //
+    // it should behave equivilently to it's copyField source...
+    for (String f : Arrays.asList("where_s",
+                                  "where_s_multi_not_uninvert_dv",
+                                  "where_s_single_not_uninvert_dv")) {
+      assertJQ(req("rows", "0", "q", "num_i:[* TO 2]", "json.facet",
+                   "{x: {type:terms, field:'"+f+"'}}")
+               , "response/numFound==3"
+               , "facets/count==3"
+               , "facets/x=={buckets:[ {val:NY, count:2} , {val:NJ, count:1} ]}"
+               );
+    }
+   
+    // faceting on an "uninvertible=false docValues=false" field should be possible
+    // when using method:enum w/sort:index
+    //
+    // it should behave equivilent to it's copyField source...
+    for (String f : Arrays.asList("where_s",
+                                  "where_s_multi_not_uninvert",
+                                  "where_s_single_not_uninvert")) {
+      assertJQ(req("rows", "0", "q", "num_i:[* TO 2]", "json.facet",
+                                     "{x: {type:terms, sort:'index asc', method:enum, field:'"+f+"'}}")
+               , "response/numFound==3"
+               , "facets/count==3"
+               , "facets/x=={buckets:[ {val:NJ, count:1} , {val:NY, count:2} ]}"
+               );
+    }
+  }
+  
   /**
    * whitebox sanity checks that a shard request range facet that returns "between" or "after"
    * will cause the correct "actual_end" to be returned

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/solr-ref-guide/src/defining-fields.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/defining-fields.adoc b/solr/solr-ref-guide/src/defining-fields.adoc
index 04c1f3e..5e6b839 100644
--- a/solr/solr-ref-guide/src/defining-fields.adoc
+++ b/solr/solr-ref-guide/src/defining-fields.adoc
@@ -56,6 +56,7 @@ Fields can have many of the same properties as field types. Properties from the
 |docValues |If true, the value of the field will be put in a column-oriented <<docvalues.adoc#docvalues,DocValues>> structure. |true or false |false
 |sortMissingFirst sortMissingLast |Control the placement of documents when a sort field is not present. |true or false |false
 |multiValued |If true, indicates that a single document might contain multiple values for this field type. |true or false |false
+|uninvertible|If true, indicates that an `indexed="true" docValues="false"` field can be "un-inverted" at query time to build up large in memory data structure to serve in place of <<docvalues.adoc#docvalues,DocValues>>.  *Defaults to true for historical reasons, but users are strongly encouraged to set this to `false` for stability and use `docValues="true"` as needed.*|true or false |true
 |omitNorms |If true, omits the norms associated with this field (this disables length normalization for the field, and saves some memory). *Defaults to true for all primitive (non-analyzed) field types, such as int, float, data, bool, and string.* Only full-text fields or fields need norms. |true or false |*
 |omitTermFreqAndPositions |If true, omits term frequency, positions, and payloads from postings for this field. This can be a performance boost for fields that don't require that information. It also reduces the storage space required for the index. Queries that rely on position that are issued on a field with this option will silently fail to find documents. *This property defaults to true for all field types that are not text fields.* |true or false |*
 |omitPositions |Similar to `omitTermFreqAndPositions` but preserves term frequency information. |true or false |*

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/solr-ref-guide/src/field-type-definitions-and-properties.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/field-type-definitions-and-properties.adoc b/solr/solr-ref-guide/src/field-type-definitions-and-properties.adoc
index 0f64785..854132f 100644
--- a/solr/solr-ref-guide/src/field-type-definitions-and-properties.adoc
+++ b/solr/solr-ref-guide/src/field-type-definitions-and-properties.adoc
@@ -130,6 +130,7 @@ The default values for each property depend on the underlying `FieldType` class,
 |docValues |If true, the value of the field will be put in a column-oriented <<docvalues.adoc#docvalues,DocValues>> structure. |true or false |false
 |sortMissingFirst sortMissingLast |Control the placement of documents when a sort field is not present. |true or false |false
 |multiValued |If true, indicates that a single document might contain multiple values for this field type. |true or false |false
+|uninvertible|If true, indicates that an `indexed="true" docValues="false"` field can be "un-inverted" at query time to build up large in memory data structure to serve in place of <<docvalues.adoc#docvalues,DocValues>>.  *Defaults to true for historical reasons, but users are strongly encouraged to set this to `false` for stability and use `docValues="true"` as needed.*|true or false |true
 |omitNorms |If true, omits the norms associated with this field (this disables length normalization for the field, and saves some memory). *Defaults to true for all primitive (non-analyzed) field types, such as int, float, data, bool, and string.* Only full-text fields or fields need norms. |true or false |*
 |omitTermFreqAndPositions |If true, omits term frequency, positions, and payloads from postings for this field. This can be a performance boost for fields that don't require that information. It also reduces the storage space required for the index. Queries that rely on position that are issued on a field with this option will silently fail to find documents. *This property defaults to true for all field types that are not text fields.* |true or false |*
 |omitPositions |Similar to `omitTermFreqAndPositions` but preserves term frequency information. |true or false |*


[37/50] [abbrv] lucene-solr:jira/http2: SOLR-12947: Misc JsonQueryRequest code cleanup

Posted by da...@apache.org.
SOLR-12947: Misc JsonQueryRequest code cleanup


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

Branch: refs/heads/jira/http2
Commit: 4410ef941acf02e752a599b5403091f86e66a9a2
Parents: 77a4bfa
Author: Jason Gerlowski <ge...@apache.org>
Authored: Fri Nov 9 13:15:08 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Fri Nov 9 13:15:08 2018 -0500

----------------------------------------------------------------------
 .../client/solrj/request/json/JsonQueryRequest.java    | 13 ++++---------
 1 file changed, 4 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4410ef94/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
index bc79881..781d9c3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
@@ -191,9 +191,7 @@ public class JsonQueryRequest extends QueryRequest {
     if (filterQuery == null) {
       throw new IllegalArgumentException("'filterQuery' must be non-null");
     }
-    jsonRequestMap.putIfAbsent("filter", new ArrayList<Object>());
-    final List<Object> filters = (List<Object>) jsonRequestMap.get("filter");
-    filters.add(filterQuery);
+    ((List)jsonRequestMap.computeIfAbsent("filter", s -> new ArrayList<>())).add(filterQuery) ;
     return this;
   }
 
@@ -220,9 +218,7 @@ public class JsonQueryRequest extends QueryRequest {
     if (filterQuery == null) {
       throw new IllegalArgumentException("'filterQuery' parameter must be non-null");
     }
-    jsonRequestMap.putIfAbsent("filter", new ArrayList<Object>());
-    final List<Object> filters = (List<Object>) jsonRequestMap.get("filter");
-    filters.add(filterQuery);
+    ((List)jsonRequestMap.computeIfAbsent("filter", s -> new ArrayList<>())).add(filterQuery) ;
     return this;
   }
 
@@ -254,6 +250,7 @@ public class JsonQueryRequest extends QueryRequest {
     if (fieldNames == null) {
       throw new IllegalArgumentException("'fieldNames' parameter must be non-null");
     }
+
     jsonRequestMap.putIfAbsent("fields", new ArrayList<String>());
     final List<String> fields = (List<String>) jsonRequestMap.get("fields");
     for (String fieldName : fieldNames) {
@@ -286,9 +283,7 @@ public class JsonQueryRequest extends QueryRequest {
       throw new IllegalArgumentException("'value' parameter must be non-null");
     }
 
-    jsonRequestMap.putIfAbsent("params", new HashMap<String, Object>());
-    final Map<String, Object> miscParamsMap = (Map<String, Object>) jsonRequestMap.get("params");
-    miscParamsMap.put(name, value);
+    ((Map<String, Object>)jsonRequestMap.computeIfAbsent("params", s -> new HashMap<String, Object>())).put(name, value);
     return this;
   }
 


[19/50] [abbrv] lucene-solr:jira/http2: SOLR-12829: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-12829: 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/f2cb9360
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f2cb9360
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f2cb9360

Branch: refs/heads/jira/http2
Commit: f2cb93605c5ef77eff652a57a0503c11dfa348d7
Parents: a197904
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 11:22:03 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 11:22:03 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2cb9360/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 00588bb..6f46bde 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -187,6 +187,8 @@ New Features
 
 * SOLR-12936: Allow percentiles Stream Evaluator to accept an array of percentiles to calculate (Joel bernstein)
 
+* SOLR-12829: Add plist (parallel list) Streaming Expression (Joel Bernstein)
+
 Other Changes
 ----------------------
 


[35/50] [abbrv] lucene-solr:jira/http2: SOLR-12962: Added a new 'uninvertible' option for fields and fieldtypes. This defaults to 'true' for backcompat allowing a FieldCache to be built for indexed fields as needed, but users are encouraged to set this t

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/solrj/src/java/org/apache/solr/common/luke/FieldFlag.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/luke/FieldFlag.java b/solr/solrj/src/java/org/apache/solr/common/luke/FieldFlag.java
index e79fb4c..0e2f3c6 100644
--- a/solr/solrj/src/java/org/apache/solr/common/luke/FieldFlag.java
+++ b/solr/solrj/src/java/org/apache/solr/common/luke/FieldFlag.java
@@ -24,6 +24,7 @@ public enum FieldFlag {
   TOKENIZED('T', "Tokenized"), 
   STORED('S', "Stored"), 
   DOC_VALUES('D', "DocValues"),
+  UNINVERTIBLE('U', "UnInvertible"),
   MULTI_VALUED('M', "Multivalued"),
   TERM_VECTOR_STORED('V', "TermVector Stored"), 
   TERM_VECTOR_OFFSET('o', "Store Offset With TermVector"),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/webapp/web/js/angular/controllers/schema.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/controllers/schema.js b/solr/webapp/web/js/angular/controllers/schema.js
index 1eefb75..8a80d42 100644
--- a/solr/webapp/web/js/angular/controllers/schema.js
+++ b/solr/webapp/web/js/angular/controllers/schema.js
@@ -150,7 +150,8 @@ solrAdminApp.controller('SchemaController',
 
                 $scope.newField = {
                     stored: "true",
-                    indexed: "true"
+                    indexed: "true",
+                    uninvertible: "true"
                 }
                 delete $scope.addErrors;
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/77a4bfaa/solr/webapp/web/partials/schema.html
----------------------------------------------------------------------
diff --git a/solr/webapp/web/partials/schema.html b/solr/webapp/web/partials/schema.html
index 0c6ba20..e94b6da 100644
--- a/solr/webapp/web/partials/schema.html
+++ b/solr/webapp/web/partials/schema.html
@@ -49,6 +49,13 @@ limitations under the License.
             indexed
             </label>
           </p>
+          
+          <p class="clearfix">
+            <label class="checkbox" for="add_uninvertible">
+              <input type="checkbox" ng-model="newField.uninvertible" id="add_uninvertible" title="Field should be uninvertible, it is generally recomended to use docValues instead." ng-true-value="'true'" ng-false-value="'false'">
+            uninvertible
+            </label>
+          </p>
 
           <p class="clearfix">
             <label class="checkbox" for="add_docValues">


[27/50] [abbrv] lucene-solr:jira/http2: SOLR-12947: Add MapWriter compatibility to JsonQueryRequest

Posted by da...@apache.org.
SOLR-12947: Add MapWriter compatibility to JsonQueryRequest

JsonQueryRequest had `setQuery` methods that took in a query either as a
String or as a Map.  But no such overload for MapWriter, a SolrJ
interface used to transmit Maps via "push writing" over the wire.  This
commit adds an overload taking this type, so that users can specify
their queries this way as well.

This commit also changes JsonQueryRequest writes out the request, to
ensure it uses "push writing" in non-MapWriter cases as well.


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

Branch: refs/heads/jira/http2
Commit: 1b084db9017f108115a3081726359695bd5a8ff1
Parents: 65dc312
Author: Jason Gerlowski <ge...@apache.org>
Authored: Wed Nov 7 15:11:24 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Wed Nov 7 21:40:06 2018 -0500

----------------------------------------------------------------------
 .../solrj/request/json/JsonQueryRequest.java    | 40 ++++++++++++++++----
 .../json/JsonQueryRequestIntegrationTest.java   | 22 ++++++++++-
 .../request/json/JsonQueryRequestUnitTest.java  | 25 ++++++++++++
 3 files changed, 79 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b084db9/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
index 3a570d0..bc79881 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
@@ -19,8 +19,6 @@ package org.apache.solr.client.solrj.request.json;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -29,6 +27,7 @@ import java.util.Map;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
@@ -103,6 +102,37 @@ public class JsonQueryRequest extends QueryRequest {
   }
 
   /**
+   * Specify the query sent as a part of this JSON request.
+   *
+   * This method may be called multiple times, but each call overwrites the value specified by previous calls.
+   * <p>
+   * <b>Example:</b> You wish to send the JSON request: "{'limit': 5, 'query': {'lucene': {'df':'genre_s', 'query': 'scifi'}}}".  The
+   * query subtree of this request is: "{'lucene': {'df': 'genre_s', 'query': 'scifi'}}".  You would represent this query
+   * JSON as follows:
+   * <pre>
+   *     final MapWriter queryWriter = new MapWriter() {
+   *         &#64;Override
+   *         public void writeMap(EntryWriter ew) throws IOException {
+   *             ew.put("lucene", (MapWriter) queryParamWriter -&#62; {
+   *                 queryParamWriter.put("df", "genre_s");
+   *                 queryParamWriter.put("query", "scifi");
+   *             });
+   *         }
+   *     };
+   * </pre>
+   *
+   * @param queryWriter a MapWriter capable of writing out the query subtree of the JSON request you wish to send.
+   * @throws IllegalArgumentException if {@code queryWriter} is null.
+   */
+  public JsonQueryRequest setQuery(MapWriter queryWriter) {
+    if (queryWriter == null) {
+      throw new IllegalArgumentException("'queryWriter' parameter must be non-null");
+    }
+    jsonRequestMap.put("query", queryWriter);
+    return this;
+  }
+
+  /**
    * Specify whether results should be fetched starting from a particular offset (or 'start').
    *
    * Defaults to 0 if not set.
@@ -266,11 +296,7 @@ public class JsonQueryRequest extends QueryRequest {
     return new RequestWriter.ContentWriter() {
       @Override
       public void write(OutputStream os) throws IOException {
-        //TODO consider whether using Utils.writeJson would work here as that'd be more mem efficient
-        OutputStreamWriter writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
-
-        writer.write(Utils.toJSONString(jsonRequestMap));
-        writer.flush();
+        Utils.writeJson(jsonRequestMap, os, true);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b084db9/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
index 807f8b6..1982cf8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestIntegrationTest.java
@@ -18,6 +18,7 @@
 package org.apache.solr.client.solrj.request.json;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -27,10 +28,10 @@ import java.util.Map;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
-import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -116,6 +117,25 @@ public class JsonQueryRequestIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testQueriesCanBeRepresentedUsingMapWriters() throws Exception {
+    final MapWriter queryWriter = new MapWriter() {
+      @Override
+      public void writeMap(EntryWriter ew) throws IOException {
+        ew.put("lucene", (MapWriter) queryParamWriter -> {
+          queryParamWriter.put("df", "genre_s");
+          queryParamWriter.put("query", "scifi");
+        });
+      }
+    };
+
+    final JsonQueryRequest simpleQuery = new JsonQueryRequest()
+        .setQuery(queryWriter);
+    QueryResponse queryResponse = simpleQuery.process(cluster.getSolrClient(), COLLECTION_NAME);
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(NUM_SCIFI_BOOKS, queryResponse.getResults().getNumFound());
+  }
+
+  @Test
   public void testQueriesCanBeNested() throws Exception {
     final Map<String, Object> queryJsonMap = new HashMap<>();
     final Map<String, Object> clausesJsonMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b084db9/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
index c6661fe..84ef956 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
@@ -27,6 +27,7 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
 import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.MapWriter;
 import org.junit.Test;
 import static org.junit.internal.matchers.StringContains.containsString;
 
@@ -54,6 +55,14 @@ public class JsonQueryRequestUnitTest extends LuceneTestCase {
   }
 
   @Test
+  public void testRejectsNullQueryMapWriter() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().setQuery((MapWriter)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
   public void testWritesProvidedQueryStringToJsonCorrectly() {
     final JsonQueryRequest request = new JsonQueryRequest().setQuery("text:solr");
     final String requestBody = writeRequestToJson(request);
@@ -73,6 +82,22 @@ public class JsonQueryRequestUnitTest extends LuceneTestCase {
   }
 
   @Test
+  public void testWritesProvidedQueryMapWriterToJsonCorrectly() {
+    final MapWriter queryWriter = new MapWriter() {
+      @Override
+      public void writeMap(EntryWriter ew) throws IOException {
+        ew.put("lucene", (MapWriter) ew1 -> {
+          ew1.put("q", "*:*");
+          ew1.put("df", "text");
+        });
+      }
+    };
+    final JsonQueryRequest request = new JsonQueryRequest().setQuery(queryWriter);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"query\":{\"lucene\":{\"q\":\"*:*\",\"df\":\"text\"}}"));
+  }
+
+  @Test
   public void testRejectsInvalidLimit() {
     Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
       new JsonQueryRequest().setLimit(-1);


[38/50] [abbrv] lucene-solr:jira/http2: SOLR-12759: fix regexp (case insensitive)

Posted by da...@apache.org.
SOLR-12759: fix regexp (case insensitive)


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

Branch: refs/heads/jira/http2
Commit: 0330372f044b18b4bfac820d24fe5ddc783fbe7a
Parents: 4410ef9
Author: David Smiley <ds...@apache.org>
Authored: Fri Nov 9 15:58:40 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Fri Nov 9 15:58:40 2018 -0500

----------------------------------------------------------------------
 .../solr/handler/extraction/ExtractingRequestHandlerTest.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0330372f/solr/contrib/extraction/src/test/org/apache/solr/handler/extraction/ExtractingRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/extraction/src/test/org/apache/solr/handler/extraction/ExtractingRequestHandlerTest.java b/solr/contrib/extraction/src/test/org/apache/solr/handler/extraction/ExtractingRequestHandlerTest.java
index 132b371..00b7bf9 100644
--- a/solr/contrib/extraction/src/test/org/apache/solr/handler/extraction/ExtractingRequestHandlerTest.java
+++ b/solr/contrib/extraction/src/test/org/apache/solr/handler/extraction/ExtractingRequestHandlerTest.java
@@ -46,7 +46,7 @@ public class ExtractingRequestHandlerTest extends SolrTestCaseJ4 {
   public static void beforeClass() throws Exception {
     // Is the JDK/env affected by a known bug?
     final String tzDisplayName = TimeZone.getDefault().getDisplayName(false, TimeZone.SHORT, Locale.US);
-    if (!tzDisplayName.matches("[A-Z]{3,}([+-]\\d\\d(:\\d\\d)?)?")) {
+    if (!tzDisplayName.matches("[A-Za-z]{3,}([+-]\\d\\d(:\\d\\d)?)?")) {
       assertTrue("Is some other JVM affected?  Or bad regex? TzDisplayName: " + tzDisplayName,
           System.getProperty("java.version").startsWith("11"));
       assumeTrue("SOLR-12759 JDK 11 (1st release) and Tika 1.x can result in extracting dates in a bad format.", false);


[10/50] [abbrv] lucene-solr:jira/http2: SOLR-12600: Fix parameter names in Solr JSON documentation

Posted by da...@apache.org.
SOLR-12600: Fix parameter names in Solr JSON documentation


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

Branch: refs/heads/jira/http2
Commit: 419d42bf7bbd6710340ccd9cc4d6e19ead0b0a04
Parents: 9952af0
Author: Alexandre Rafalovitch <ar...@apache.org>
Authored: Wed Nov 7 07:01:06 2018 -0500
Committer: Alexandre Rafalovitch <ar...@apache.org>
Committed: Wed Nov 7 07:01:06 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                              | 2 ++
 solr/solr-ref-guide/src/json-request-api.adoc | 7 +++++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/419d42bf/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7b60eb0..928cd44 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -208,6 +208,8 @@ Other Changes
 
 * SOLR-12966: Add Javadoc @since tag to URP classes (Alexandre Rafalovitch)
 
+* SOLR-12600: Fix parameter names in Solr JSON documentation
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/419d42bf/solr/solr-ref-guide/src/json-request-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/json-request-api.adoc b/solr/solr-ref-guide/src/json-request-api.adoc
index bc676cc..dc05857 100644
--- a/solr/solr-ref-guide/src/json-request-api.adoc
+++ b/solr/solr-ref-guide/src/json-request-api.adoc
@@ -171,11 +171,14 @@ Right now only some standard query parameters have JSON equivalents. Unmapped pa
 |`fq`
 |`filter`
 
-|`offset`
 |`start`
+|`offset`
 
-|`limit`
 |`rows`
+|`limit`
+
+|`fl`
+|`fields`
 
 |`sort`
 |`sort`


[22/50] [abbrv] lucene-solr:jira/http2: SOLR-12913: grammer fix

Posted by da...@apache.org.
SOLR-12913: grammer fix


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

Branch: refs/heads/jira/http2
Commit: 9594ee38bbc1ece63dff9e876cf513b12e2469e5
Parents: 531b166
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 15:29:13 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 15:29:13 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/vectorization.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9594ee38/solr/solr-ref-guide/src/vectorization.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/vectorization.adoc b/solr/solr-ref-guide/src/vectorization.adoc
index acd56ec..9483cc3 100644
--- a/solr/solr-ref-guide/src/vectorization.adoc
+++ b/solr/solr-ref-guide/src/vectorization.adoc
@@ -248,7 +248,7 @@ When this expression is sent to the `/stream` handler it responds with:
 }
 ----
 
-== Facet Co-Occurrence Matrices
+== Facet Co-occurrence Matrices
 
 The `facet` function can be used to quickly perform mulit-dimension aggregations of categorical data from
 records stored in a Solr Cloud collection. These multi-dimension aggregations can represent co-occurrence


[02/50] [abbrv] lucene-solr:jira/http2: SOLR-12956: Add Javadoc @since tag to Analyzer component classes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicAnalyzer.java
index 868834f..0c18cdd 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicAnalyzer.java
@@ -36,6 +36,8 @@ import org.apache.lucene.analysis.en.EnglishAnalyzer;
  * ClassicAnalyzer was named StandardAnalyzer in Lucene versions prior to 3.1. 
  * As of 3.1, {@link StandardAnalyzer} implements Unicode text segmentation,
  * as specified by UAX#29.
+ *
+ * @since 3.1
  */
 public final class ClassicAnalyzer extends StopwordAnalyzerBase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizerFactory.java
index b2a7134..04c1fd6 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizerFactory.java
@@ -30,6 +30,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.ClassicTokenizerFactory" maxTokenLength="120"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ClassicTokenizerFactory extends TokenizerFactory {
   private final int maxTokenLength;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizerFactory.java
index c74c55c..74467a2 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizerFactory.java
@@ -30,6 +30,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.StandardTokenizerFactory" maxTokenLength="255"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre> 
+ *
+ * @since 3.1
  */
 public class StandardTokenizerFactory extends TokenizerFactory {
   private final int maxTokenLength;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java
index 9fd15fa..59b0de0 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailAnalyzer.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.en.EnglishAnalyzer;
  * with {@link org.apache.lucene.analysis.LowerCaseFilter} and
  * {@link org.apache.lucene.analysis.StopFilter}, using a list of
  * English stop words.
+ *
+ * @since 3.6.0
  */
 public final class UAX29URLEmailAnalyzer extends StopwordAnalyzerBase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java
index c7c5214..526f5f6 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizerFactory.java
@@ -30,6 +30,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.UAX29URLEmailTokenizerFactory" maxTokenLength="255"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre> 
+ *
+ * @since 3.1
  */
 public class UAX29URLEmailTokenizerFactory extends TokenizerFactory {
   private final int maxTokenLength;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java
index 5f44f13..e4acd11 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishAnalyzer.java
@@ -37,6 +37,8 @@ import org.tartarus.snowball.ext.SwedishStemmer;
 
 /**
  * {@link Analyzer} for Swedish.
+ *
+ * @since 3.1
  */
 public final class SwedishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
index 87ddc08..b0addcf 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
@@ -52,7 +52,7 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
  *             [optional tokenizer factory parameters]/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
- * 
+ *
  * <p>
  * An optional param name prefix of "tokenizerFactory." may be used for any 
  * init params that the SynonymFilterFactory needs to pass to the specified 
@@ -76,6 +76,8 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
  *
  * @deprecated Use {@link SynonymGraphFilterFactory} instead, but be sure to also
  * use {@link FlattenGraphFilterFactory} at index time (not at search time) as well.
+ *
+ * @since 3.1
  */
 @Deprecated
 public class SynonymFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
index c1426b8..399c8ae 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
@@ -30,6 +30,8 @@ import org.apache.lucene.analysis.core.DecimalDigitFilter;
 
 /**
  * {@link Analyzer} for Thai language. It uses {@link java.text.BreakIterator} to break words.
+ *
+ * @since 3.1
  */
 public final class ThaiAnalyzer extends StopwordAnalyzerBase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java
index a941c88..04c91e7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java
@@ -31,6 +31,8 @@ import org.apache.lucene.util.AttributeFactory;
  *     &lt;tokenizer class="solr.ThaiTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 4.10.0
  */
 public class ThaiTokenizerFactory extends TokenizerFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java
index f48d449..2ffb4f0 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/tr/TurkishAnalyzer.java
@@ -33,6 +33,8 @@ import org.tartarus.snowball.ext.TurkishStemmer;
 
 /**
  * {@link Analyzer} for Turkish.
+ *
+ * @since 3.1
  */
 public final class TurkishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
index 4c0df29..181beb5 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
@@ -26,6 +26,8 @@ import org.apache.lucene.analysis.CharFilter;
 /**
  * Abstract parent class for analysis factories that create {@link CharFilter}
  * instances.
+ *
+ * @since 3.1
  */
 public abstract class CharFilterFactory extends AbstractAnalysisFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/ElisionFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/ElisionFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/ElisionFilterFactory.java
index fff3edc..e27fbc9 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/ElisionFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/ElisionFilterFactory.java
@@ -35,6 +35,8 @@ import org.apache.lucene.analysis.fr.FrenchAnalyzer;
  *       articles="stopwordarticles.txt" ignoreCase="true"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ElisionFilterFactory extends TokenFilterFactory implements ResourceLoaderAware, MultiTermAwareComponent {
   private final String articlesFile;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
index 7eec6dd..180126d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
@@ -25,6 +25,8 @@ import org.apache.lucene.analysis.TokenStream;
 /**
  * Abstract parent class for analysis factories that create {@link org.apache.lucene.analysis.TokenFilter}
  * instances.
+ *
+ * @since 3.1
  */
 public abstract class TokenFilterFactory extends AbstractAnalysisFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java
index 8e2d64a..8b3e1f9 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java
@@ -27,6 +27,8 @@ import java.util.Set;
 /**
  * Abstract parent class for analysis factories that create {@link Tokenizer}
  * instances.
+ *
+ * @since 3.1
  */
 public abstract class TokenizerFactory extends AbstractAnalysisFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerFactory.java
index 83e08aa..e9833af 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerFactory.java
@@ -31,6 +31,8 @@ import org.apache.lucene.util.AttributeFactory;
  *     &lt;tokenizer class="solr.WikipediaTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class WikipediaTokenizerFactory extends TokenizerFactory {
   public static final String TOKEN_OUTPUT = "tokenOutput";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java
index 4d0f039..d2099fb 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java
@@ -71,6 +71,8 @@ import java.text.Collator;
  *   CollationKeyAnalyzer to generate index terms, do not use
  *   ICUCollationKeyAnalyzer on the query side, or vice versa.
  * </p>
+ *
+ * @since 3.1
  */
 public final class CollationKeyAnalyzer extends Analyzer {
   private final CollationAttributeFactory factory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java
index ab8e245..2f43b51 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUNormalizer2CharFilterFactory.java
@@ -44,6 +44,8 @@ import com.ibm.icu.text.UnicodeSet;
  * @see ICUNormalizer2CharFilter
  * @see Normalizer2
  * @see FilteredNormalizer2
+ *
+ * @since 4.10.0
  */
 public class ICUNormalizer2CharFilterFactory extends CharFilterFactory implements MultiTermAwareComponent {
   private final Normalizer2 normalizer;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
index 0cd4cf2..b57117e 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
@@ -72,6 +72,8 @@ import com.ibm.icu.text.RuleBasedBreakIterator;
  *                rulefiles="Latn:my.Latin.rules.rbbi,Cyrl:my.Cyrillic.rules.rbbi"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoaderAware {
   static final String RULEFILES = "rulefiles";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java b/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
index 4388969..8bc70c3 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
@@ -62,6 +62,8 @@ import org.apache.lucene.collation.CollationKeyAnalyzer; // javadocs
  *   generation timing and key length comparisons between ICU4J and
  *   java.text.Collator over several languages.
  * </p>
+ *
+ * @since 3.1
  */
 public final class ICUCollationKeyAnalyzer extends Analyzer {
   private final ICUCollationAttributeFactory factory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java
index 06e119e..82c71dc 100644
--- a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java
+++ b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseAnalyzer.java
@@ -34,6 +34,8 @@ import org.apache.lucene.analysis.ja.dict.UserDictionary;
 /**
  * Analyzer for Japanese that uses morphological analysis.
  * @see JapaneseTokenizer
+ *
+ * @since 3.6.0
  */
 public class JapaneseAnalyzer extends StopwordAnalyzerBase {
   private final Mode mode;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseIterationMarkCharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseIterationMarkCharFilterFactory.java b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseIterationMarkCharFilterFactory.java
index 0b5b72f..f888b2b 100644
--- a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseIterationMarkCharFilterFactory.java
+++ b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseIterationMarkCharFilterFactory.java
@@ -35,6 +35,8 @@ import java.util.Map;
  *     &lt;tokenizer class="solr.JapaneseTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 4.0.0
  */
 public class JapaneseIterationMarkCharFilterFactory extends CharFilterFactory implements MultiTermAwareComponent {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java
index 844684a..a45b3c1 100644
--- a/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java
+++ b/lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/JapaneseTokenizerFactory.java
@@ -76,6 +76,8 @@ import org.apache.lucene.analysis.util.ResourceLoaderAware;
  * <p>
  * Parameters nBestCost and nBestExamples work with all tokenizer
  * modes, but it makes the most sense to use them with NORMAL mode.
+ *
+ * @since 3.6.0
  */
 public class JapaneseTokenizerFactory extends TokenizerFactory implements ResourceLoaderAware {
   private static final String MODE = "mode";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikAnalyzer.java b/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikAnalyzer.java
index 4c0c277..3277a10 100644
--- a/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikAnalyzer.java
+++ b/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikAnalyzer.java
@@ -28,6 +28,8 @@ import org.apache.lucene.analysis.standard.StandardTokenizer;
 /**
  * {@link org.apache.lucene.analysis.Analyzer} using Morfologik library.
  * @see <a href="http://morfologik.blogspot.com/">Morfologik project page</a>
+ *
+ * @since 4.0.0
  */
 public class MorfologikAnalyzer extends Analyzer {
   private final Dictionary dictionary;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java b/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java
index df0802e..9b6816b 100644
--- a/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java
+++ b/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/uk/UkrainianMorfologikAnalyzer.java
@@ -39,6 +39,8 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * A dictionary-based {@link Analyzer} for Ukrainian.
+ *
+ * @since 6.2.0
  */
 public final class UkrainianMorfologikAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
index 76023bb..be7acdd 100644
--- a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
@@ -31,6 +31,8 @@ import static org.apache.lucene.analysis.TokenStream.DEFAULT_TOKEN_ATTRIBUTE_FAC
  * Analyzer for Korean that uses morphological analysis.
  * @see KoreanTokenizer
  * @lucene.experimental
+ *
+ * @since 7.4.0
  */
 public class KoreanAnalyzer extends Analyzer {
   private final UserDictionary userDict;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
index 70bcef6..b79ecac 100644
--- a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
@@ -41,6 +41,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *   <li>tags: List of stop tags. if not specified, {@link KoreanPartOfSpeechStopFilter#DEFAULT_STOP_TAGS} is used.</li>
  * </ul>
  * @lucene.experimental
+ *
+ * @since 7.4.0
  */
 public class KoreanPartOfSpeechStopFilterFactory extends TokenFilterFactory {
   private Set<POS.Tag> stopTags;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
index ce2779b..2225a1d 100644
--- a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * &lt;/fieldType&gt;
  * </pre>
  * @lucene.experimental
+ *
+ * @since 7.4.0
  */
 public class KoreanReadingFormFilterFactory extends TokenFilterFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
index f2fed27..780dcb1 100644
--- a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
@@ -58,6 +58,8 @@ import org.apache.lucene.analysis.ko.KoreanTokenizer.DecompoundMode;
  *   <li>outputUnknownUnigrams: If true outputs unigrams for unknown words.</li>
  * </ul>
  * @lucene.experimental
+ *
+ * @since 7.4.0
  */
 public class KoreanTokenizerFactory extends TokenizerFactory implements ResourceLoaderAware {
   private static final String USER_DICT_PATH = "userDictionary";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java
index fdaecb2..3a74dee 100644
--- a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java
+++ b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterFactory.java
@@ -32,6 +32,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *     &lt;filter class="solr.DoubleMetaphoneFilterFactory" inject="true" maxCodeLength="4"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ *
+ * @since 3.1
  */
 public class DoubleMetaphoneFilterFactory extends TokenFilterFactory
 {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
index fc69a61..d5d3b0e 100644
--- a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
+++ b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
@@ -62,6 +62,8 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  * &lt;/fieldType&gt;</pre>
  * 
  * @see PhoneticFilter
+ *
+ * @since 3.1
  */
 public class PhoneticFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   /** parameter name: either a short name or a full class name */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java
index 43cb822..5d30459 100644
--- a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java
+++ b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java
@@ -30,6 +30,8 @@ import org.apache.lucene.util.AttributeFactory;
  * SmartChinese stoplist with a StopFilterFactory via:
  * <code>words="org/apache/lucene/analysis/cn/smart/stopwords.txt"</code>
  * @lucene.experimental
+ *
+ * @since 4.10.0
  */
 public final class HMMChineseTokenizerFactory extends TokenizerFactory {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
index f604d4b..c1e7daf 100644
--- a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
+++ b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
@@ -49,6 +49,8 @@ import org.apache.lucene.util.IOUtils;
  * Thanks to ICTCLAS for their hard work, and for contributing the data under the Apache 2 License!
  * </p>
  * @lucene.experimental
+ *
+ * @since 3.1
  */
 public final class SmartChineseAnalyzer extends Analyzer {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java b/lucene/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
index e449c1e..a58bc78 100644
--- a/lucene/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
+++ b/lucene/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
@@ -38,6 +38,8 @@ import org.egothor.stemmer.Trie;
 
 /**
  * {@link Analyzer} for Polish.
+ *
+ * @since 3.1
  */
 public final class PolishAnalyzer extends StopwordAnalyzerBase {
   private final CharArraySet stemExclusionSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java b/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
index 46338dc..b9a798b 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
@@ -79,6 +79,8 @@ import org.apache.lucene.util.Version;
  *   <li><a href="{@docRoot}/../analyzers-stempel/overview-summary.html">Stempel</a>:
  *       Algorithmic Stemmer for the Polish Language.
  * </ul>
+ *
+ * @since 3.1
  */
 public abstract class Analyzer implements Closeable {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java b/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
index d23d004..0e5297e 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
@@ -42,6 +42,8 @@ import org.apache.lucene.util.AttributeFactory;
  * as superclass!
  *
  * @see DelegatingAnalyzerWrapper
+ *
+ * @since 4.0.0
  */
 public abstract class AnalyzerWrapper extends Analyzer {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/core/src/java/org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java b/lucene/core/src/java/org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java
index edf5b2b..b1a9378 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/DelegatingAnalyzerWrapper.java
@@ -35,6 +35,8 @@ import java.io.Reader;
  * which is returned by {@link #getReuseStrategy()}. This strategy is used when
  * delegating. If you wrap this analyzer again and reuse this strategy, no
  * delegation is done and the given fallback is used.
+ *
+ * @since 4.10.0
  */
 public abstract class DelegatingAnalyzerWrapper extends AnalyzerWrapper {
   
@@ -99,4 +101,4 @@ public abstract class DelegatingAnalyzerWrapper extends AnalyzerWrapper {
     }
   };
   
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/core/src/java/org/apache/lucene/analysis/StopwordAnalyzerBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/StopwordAnalyzerBase.java b/lucene/core/src/java/org/apache/lucene/analysis/StopwordAnalyzerBase.java
index c35e715..0de83e5 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/StopwordAnalyzerBase.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/StopwordAnalyzerBase.java
@@ -27,7 +27,9 @@ import org.apache.lucene.util.IOUtils;
 
 /**
  * Base class for Analyzers that need to make use of stopword sets. 
- * 
+ *
+ *
+ * @since 3.1
  */
 public abstract class StopwordAnalyzerBase extends Analyzer {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/core/src/java/org/apache/lucene/analysis/standard/StandardAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/standard/StandardAnalyzer.java b/lucene/core/src/java/org/apache/lucene/analysis/standard/StandardAnalyzer.java
index 76c958b..3366611 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/standard/StandardAnalyzer.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/standard/StandardAnalyzer.java
@@ -30,6 +30,8 @@ import org.apache.lucene.analysis.WordlistLoader;
 /**
  * Filters {@link StandardTokenizer} with {@link LowerCaseFilter} and
  * {@link StopFilter}, using a configurable list of stop words.
+ *
+ * @since 3.1
  */
 public final class StandardAnalyzer extends StopwordAnalyzerBase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
index 631a67a..35cf5e8 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
@@ -34,6 +34,8 @@ import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
  * and {@link #preserveSep()}
  *
  * @lucene.experimental
+ *
+ * @since 5.1.0
  */
 public final class CompletionAnalyzer extends AnalyzerWrapper {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 14a9f08..979677d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -119,6 +119,8 @@ Other Changes
 * SOLR-12746: Simplify the Ref Guide HTML structure and use semantic HTML tags where possible. Adds new template files
   for Asciidoctor HTML conversion. Building the HTML version now requires the Slim gem. (Cassandra Targett)
 
+* SOLR-12956: Add Javadoc @since tag to Analyzer component classes (Alexandre Rafalovitch)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilterFactory.java b/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilterFactory.java
index e64957f..a1917b8 100644
--- a/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilterFactory.java
+++ b/solr/core/src/java/org/apache/solr/analysis/ReversedWildcardFilterFactory.java
@@ -60,6 +60,7 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
  *
+ * @since 3.1
  */
 public class ReversedWildcardFilterFactory extends TokenFilterFactory {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java b/solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java
index 38d0d17..10c1b7c 100644
--- a/solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java
+++ b/solr/core/src/java/org/apache/solr/analysis/SolrAnalyzer.java
@@ -22,6 +22,7 @@ import java.io.Reader;
 
 /**
  *
+ * @since 3.1
  */
 public abstract class SolrAnalyzer extends Analyzer {
   int posIncGap = 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java b/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
index af7e812..d27e946 100644
--- a/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
+++ b/solr/core/src/java/org/apache/solr/analysis/TokenizerChain.java
@@ -28,6 +28,8 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
 /**
  * An analyzer that uses a tokenizer and a list of token filters to
  * create a TokenStream.
+ *
+ * @since 3.1
  */
 public final class TokenizerChain extends SolrAnalyzer {
   private static final CharFilterFactory[] EMPTY_CHAR_FITLERS = new CharFilterFactory[0];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c07df196/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 41eb460..ae73e09 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -80,7 +80,7 @@ import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MAT
 /**
  * Base class for all field types used by an index schema.
  *
- *
+ * @since 3.1
  */
 public abstract class FieldType extends FieldProperties {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());


[13/50] [abbrv] lucene-solr:jira/http2: SOLR-12971: Add pivot Stream Evaluator to pivot facet co-occurrence counts into a matrix

Posted by da...@apache.org.
SOLR-12971: Add pivot Stream Evaluator to pivot facet co-occurrence counts into a matrix


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

Branch: refs/heads/jira/http2
Commit: 80efc239995b1f882052834e6657f4204e8cb669
Parents: 936ecf1
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 08:12:53 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 08:14:17 2018 -0500

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/io/Lang.java   |   2 +
 .../client/solrj/io/eval/PivotEvaluator.java    | 111 +++++++++++++++++++
 .../apache/solr/client/solrj/io/TestLang.java   |   2 +-
 .../solrj/io/stream/MathExpressionTest.java     |  44 ++++++++
 4 files changed, 158 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/80efc239/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
index 1777467..200f07b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -273,6 +273,8 @@ public class Lang {
         .withFunctionName("getSupportPoints", GetSupportPointsEvaluator.class)
         .withFunctionName("pairSort", PairSortEvaluator.class)
         .withFunctionName("recip", RecipEvaluator.class)
+        .withFunctionName("pivot", PivotEvaluator.class)
+
         // Boolean Stream Evaluators
 
         .withFunctionName("and", AndEvaluator.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/80efc239/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PivotEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PivotEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PivotEvaluator.java
new file mode 100644
index 0000000..919db14
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PivotEvaluator.java
@@ -0,0 +1,111 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+
+import java.util.ArrayList;
+import java.util.Locale;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.Map;
+import java.util.HashMap;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.client.solrj.io.Tuple;
+
+public class PivotEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public  PivotEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(4 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 4 values but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object[] values) throws IOException {
+    if(values.length != 4) {
+      throw new IOException("The pivot function requires four parameters.");
+    }
+
+    Object value1 = values[0];
+    Object value2 = values[1];
+    Object value3 = values[2];
+    Object value4 = values[3];
+
+    if(value1 instanceof List) {
+      List<Tuple> tuples = (List<Tuple>)value1;
+      String x = (String)value2;
+      x = x.replace("\"", "");
+      String y = (String)value3;
+      y = y.replace("\"", "");
+
+      String vlabel = (String)value4;
+      vlabel = vlabel.replace("\"", "");
+
+      Set<String> xset = new TreeSet();
+      Set<String> yset = new TreeSet();
+
+      for(int i=0; i<tuples.size(); i++) {
+        Tuple tuple = tuples.get(i);
+        xset.add(tuple.getString(x));
+        yset.add(tuple.getString(y));
+      }
+
+      double[][] data = new double[xset.size()][yset.size()];
+
+      List<String> xlabels = new ArrayList(xset.size());
+      Map<String, Integer> xindexes = new HashMap();
+      int xindex = 0;
+      for (String xlabel :xset) {
+        xlabels.add(xlabel);
+        xindexes.put(xlabel, xindex);
+        ++xindex;
+      }
+
+      List<String> ylabels = new ArrayList(yset.size());
+      Map<String, Integer> yindexes = new HashMap();
+      int yindex = 0;
+      for (String ylabel : yset) {
+        ylabels.add(ylabel);
+        yindexes.put(ylabel, yindex);
+        ++yindex;
+      }
+
+      for(Tuple tuple : tuples) {
+        String xlabel = tuple.getString(x);
+        String ylabel = tuple.getString(y);
+        int xi = xindexes.get(xlabel);
+        int yi = yindexes.get(ylabel);
+        double val = tuple.getDouble(vlabel);
+        data[xi][yi] = val;
+      }
+
+      Matrix matrix = new Matrix(data);
+      matrix.setRowLabels(xlabels);
+      matrix.setColumnLabels(ylabels);
+      return matrix;
+    } else {
+      throw new IOException("The getValue function expects a list of tuples as the first parameter");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/80efc239/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
index 960eb50..98a6d45 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -73,7 +73,7 @@ public class TestLang extends LuceneTestCase {
       "outliers", "stream", "getCache", "putCache", "listCache", "removeCache", "zscores", "latlonVectors",
       "convexHull", "getVertices", "getBaryCenter", "getArea", "getBoundarySize","oscillate",
       "getAmplitude", "getPhase", "getAngularFrequency", "enclosingDisk", "getCenter", "getRadius",
-      "getSupportPoints", "pairSort", "log10", "plist", "recip"};
+      "getSupportPoints", "pairSort", "log10", "plist", "recip", "pivot"};
 
   @Test
   public void testLang() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/80efc239/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index 4642388..9c93ced 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -2164,6 +2164,50 @@ public class MathExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testPivot() throws Exception {
+    String cexpr = "let(echo=true," +
+        "               a=list(tuple(fx=x1, fy=f1, fv=add(1,1)), " +
+        "                      tuple(fx=x1, fy=f2, fv=add(1,3)), " +
+        "                      tuple(fx=x2, fy=f1, fv=add(1,7)), " +
+        "                      tuple(fx=x3, fy=f1, fv=add(1,4))," +
+        "                      tuple(fx=x3, fy=f3, fv=add(1,7)))," +
+                   "    b=pivot(a, fx, fy, fv)," +
+        "               c=getRowLabels(b)," +
+        "               d=getColumnLabels(b))";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertEquals(tuples.size(), 1);
+    List<List<Number>> matrix = (List<List<Number>>)tuples.get(0).get("b");
+    List<Number> row1 = matrix.get(0);
+    assertEquals(row1.get(0).doubleValue(), 2.0,0);
+    assertEquals(row1.get(1).doubleValue(), 4.0,0);
+    assertEquals(row1.get(2).doubleValue(), 0,0);
+    List<Number> row2 = matrix.get(1);
+    assertEquals(row2.get(0).doubleValue(), 8.0,0);
+    assertEquals(row2.get(1).doubleValue(), 0,0);
+    assertEquals(row2.get(2).doubleValue(), 0,0);
+    List<Number> row3 = matrix.get(2);
+    assertEquals(row3.get(0).doubleValue(), 5.0,0);
+    assertEquals(row3.get(1).doubleValue(), 0,0);
+    assertEquals(row3.get(2).doubleValue(), 8.0,0);
+
+    List<String> rowLabels = (List<String>)tuples.get(0).get("c");
+    assertEquals(rowLabels.get(0), "x1");
+    assertEquals(rowLabels.get(1), "x2");
+    assertEquals(rowLabels.get(2), "x3");
+    List<String> columnLabels = (List<String>)tuples.get(0).get("d");
+    assertEquals(columnLabels.get(0), "f1");
+    assertEquals(columnLabels.get(1), "f2");
+    assertEquals(columnLabels.get(2), "f3");
+  }
+
+  @Test
   public void testEbeSubtract() throws Exception {
     String cexpr = "let(echo=true," +
         "               a=array(2, 4, 6, 8, 10, 12)," +


[46/50] [abbrv] lucene-solr:jira/http2: SOLR-12977: Autoscaling tries to fetch metrics from dead nodes

Posted by da...@apache.org.
SOLR-12977: Autoscaling tries to fetch metrics from dead nodes


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

Branch: refs/heads/jira/http2
Commit: 605c3f6f1a8d14ad3933d2ea225ec5ee66a631d9
Parents: 52998fa
Author: Noble Paul <no...@apache.org>
Authored: Mon Nov 12 18:39:05 2018 +1100
Committer: Noble Paul <no...@apache.org>
Committed: Mon Nov 12 18:39:05 2018 +1100

----------------------------------------------------------------------
 solr/CHANGES.txt                                             | 2 ++
 .../solr/client/solrj/impl/SolrClientNodeStateProvider.java  | 8 +++++++-
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/605c3f6f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a13e5cb..2e8e97a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -212,6 +212,8 @@ New Features
   to set this to false (using docValues as needed) to reduce the risk of large fluxuations in heap
   size due to unexpected attempts to sort/facet/function on non-docValue fields. (hossman)
 
+* SOLR-12977: Autoscaling tries to fetch metrics from dead nodes (noble)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/605c3f6f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index 5a9d5b0..fc3564f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -118,7 +118,6 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
 
   @Override
   public void writeMap(EntryWriter ew) throws IOException {
-//    ew.put("liveNodes", liveNodes);
     ew.put("replicaInfo", Utils.getDeepCopy(nodeVsCollectionVsShardVsReplicaInfo, 5));
     ew.put("nodeValues", nodeVsTags);
   }
@@ -189,6 +188,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
   }
 
   static void fetchReplicaMetrics(String solrNode, ClientSnitchCtx ctx, Map<String, Object> metricsKeyVsTag) {
+    if (!ctx.isNodeAlive(solrNode)) return;
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add("key", metricsKeyVsTag.keySet().toArray(new String[0]));
     try {
@@ -216,6 +216,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
   static class AutoScalingSnitch extends ImplicitSnitch {
     @Override
     protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
+      if (!((ClientSnitchCtx)ctx).isNodeAlive(solrNode)) return;
       ClientSnitchCtx snitchContext = (ClientSnitchCtx) ctx;
       Map<String, Object> metricsKeyVsTag = new HashMap<>();
       for (String tag : requestedTags) {
@@ -314,6 +315,11 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     ZkClientClusterStateProvider zkClientClusterStateProvider;
     CloudSolrClient solrClient;
 
+    public boolean isNodeAlive(String node) {
+      if (zkClientClusterStateProvider != null && zkClientClusterStateProvider.getLiveNodes().contains(node))
+        return true;
+      return true;
+    }
     public ClientSnitchCtx(SnitchInfo perSnitch,
                            String node, Map<String, Object> session,
                            CloudSolrClient solrClient) {


[04/50] [abbrv] lucene-solr:jira/http2: SOLR-12966: Add Javadoc @since tag to URP classes

Posted by da...@apache.org.
SOLR-12966: Add Javadoc @since tag to URP classes


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

Branch: refs/heads/jira/http2
Commit: 0ddbc4bf953e65ff7af5140b95f8d9edcc245875
Parents: c07df19
Author: Alexandre Rafalovitch <ar...@apache.org>
Authored: Tue Nov 6 08:05:07 2018 -0500
Committer: Alexandre Rafalovitch <ar...@apache.org>
Committed: Tue Nov 6 08:05:07 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                                  | 2 ++
 .../update/processor/OpenNLPLangDetectUpdateProcessorFactory.java | 2 ++
 .../solr/update/processor/SignatureUpdateProcessorFactory.java    | 3 +++
 3 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ddbc4bf/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 979677d..17e918b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -121,6 +121,8 @@ Other Changes
 
 * SOLR-12956: Add Javadoc @since tag to Analyzer component classes (Alexandre Rafalovitch)
 
+* SOLR-12966: Add Javadoc @since tag to URP classes (Alexandre Rafalovitch)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ddbc4bf/solr/contrib/langid/src/java/org/apache/solr/update/processor/OpenNLPLangDetectUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/langid/src/java/org/apache/solr/update/processor/OpenNLPLangDetectUpdateProcessorFactory.java b/solr/contrib/langid/src/java/org/apache/solr/update/processor/OpenNLPLangDetectUpdateProcessorFactory.java
index ffe11aa..b4e0253 100644
--- a/solr/contrib/langid/src/java/org/apache/solr/update/processor/OpenNLPLangDetectUpdateProcessorFactory.java
+++ b/solr/contrib/langid/src/java/org/apache/solr/update/processor/OpenNLPLangDetectUpdateProcessorFactory.java
@@ -46,6 +46,8 @@ import opennlp.tools.langdetect.LanguageDetectorModel;
  * &lt;/processor&gt;
  * </pre>
  * See <a href="http://wiki.apache.org/solr/LanguageDetection">http://wiki.apache.org/solr/LanguageDetection</a>
+ *
+ * @since 7.3.0
  */
 public class OpenNLPLangDetectUpdateProcessorFactory extends UpdateRequestProcessorFactory
   implements SolrCoreAware {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ddbc4bf/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java
index 7257fd7..7ea0219 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java
@@ -36,6 +36,9 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
+/**
+ * @since 3.1
+ **/
 public class SignatureUpdateProcessorFactory 
   extends UpdateRequestProcessorFactory 
   implements SolrCoreAware {


[44/50] [abbrv] lucene-solr:jira/http2: SOLR-12965: Add facet support to JsonQueryRequest

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
new file mode 100644
index 0000000..be4f0cc
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DirectJsonQueryRequestFacetingIntegrationTest.java
@@ -0,0 +1,615 @@
+/*
+ * 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.request.json;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION_NAME = "techproducts";
+  private static final String CONFIG_NAME = "techproducts_config";
+  private static final int NUM_TECHPRODUCTS_DOCS = 32;
+  private static final int NUM_IN_STOCK = 17;
+  private static final int NUM_ELECTRONICS = 12;
+  private static final int NUM_CURRENCY = 4;
+  private static final int NUM_MEMORY = 3;
+  private static final int NUM_CORSAIR = 3;
+  private static final int NUM_BELKIN = 2;
+  private static final int NUM_CANON = 2;
+
+
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig(CONFIG_NAME, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+        .configure();
+
+    final List<String> solrUrls = new ArrayList<>();
+    solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
+
+    ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
+    up.setParam("collection", COLLECTION_NAME);
+    up.addFile(getFile("solrj/techproducts.xml"), "application/xml");
+    up.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
+    UpdateResponse updateResponse = up.process(cluster.getSolrClient());
+    assertEquals(0, updateResponse.getStatus());
+  }
+  @Test
+  public void testSingleTermsFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testMultiTermsFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "    },",
+        "    'top_manufacturers': {",
+        "      'type': 'terms',",
+        "      'field': 'manu_id_s',",
+        "      'limit': 3",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    assertHasFacetWithBucketValues(rawResponse,"top_manufacturers", new FacetBucket("corsair",NUM_CORSAIR),
+        new FacetBucket("belkin", NUM_BELKIN), new FacetBucket("canon", NUM_CANON));
+  }
+
+  @Test
+  public void testSingleRangeFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'prices': {",
+        "      'type': 'range',",
+        "      'field': 'price',",
+        "      'start': 0,",
+        "      'end': 100,",
+        "      'gap': 20",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"prices",
+        new FacetBucket(0.0f, 5),
+        new FacetBucket(20.0f, 0),
+        new FacetBucket(40.0f, 0),
+        new FacetBucket(60.0f, 1),
+        new FacetBucket(80.0f, 1));
+  }
+
+  @Test
+  public void testMultiRangeFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'prices': {",
+        "      'type': 'range',",
+        "      'field': 'price',",
+        "      'start': 0,",
+        "      'end': 100,",
+        "      'gap': 20",
+        "    },",
+        "    'shipping_weights': {",
+        "      'type': 'range',",
+        "      'field': 'weight',",
+        "      'start': 0,",
+        "      'end': 200,",
+        "      'gap': 50",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"prices",
+        new FacetBucket(0.0f, 5),
+        new FacetBucket(20.0f, 0),
+        new FacetBucket(40.0f, 0),
+        new FacetBucket(60.0f, 1),
+        new FacetBucket(80.0f, 1));
+    assertHasFacetWithBucketValues(rawResponse, "shipping_weights",
+        new FacetBucket(0.0f, 6),
+        new FacetBucket(50.0f, 0),
+        new FacetBucket(100.0f, 0),
+        new FacetBucket(150.0f,1));
+  }
+
+  @Test
+  public void testSingleStatFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'sum_price': 'sum(price)'",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+  }
+
+  @Test
+  public void testMultiStatFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'sum_price': 'sum(price)',",
+        "    'avg_price': 'avg(price)'",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+  }
+
+  @Test
+  public void testMultiFacetsMixedTypes() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'avg_price': 'avg(price)',",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testNestedTermsFacet() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "      'facet': {",
+        "        'top_manufacturers_for_cat': {",
+        "          'type': 'terms',",
+        "          'field': 'manu_id_s',",
+        "          'limit': 1",
+        "        }",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+
+    // Test subfacet values for each top-level facet bucket
+    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
+    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
+    assertFacetResponseHasFacetWithBuckets(electronicsSubFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
+    assertFacetResponseHasFacetWithBuckets(currencySubfacet, "top_manufacturers_for_cat", new FacetBucket("boa", 1));
+    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
+    assertFacetResponseHasFacetWithBuckets(memorySubfacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+  }
+
+  @Test
+  public void testNestedFacetsOfMixedTypes() throws Exception {
+    final String subfacetName = "avg_price_for_cat";
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "      'facet': {",
+        "        'avg_price_for_cat': 'avg(price)'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+
+    // Test subfacet values for each top-level facet bucket
+    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
+    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
+    assertFacetResponseHasStatFacetWithValue(electronicsSubFacet, subfacetName, 252.02909261530095);
+    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
+    assertFacetResponseHasStatFacetWithValue(currencySubfacet, subfacetName, 0.0);
+    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
+    assertFacetResponseHasStatFacetWithValue(memorySubfacet, subfacetName, 129.99499893188477);
+  }
+
+  @Test
+  public void testFacetWithDomainFilteredBySimpleQueryString() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_popular_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "      'domain': {",
+        "        'filter': 'popularity:[5 TO 10]'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+  }
+
+  @Test
+  public void testFacetWithDomainFilteredByLocalParamsQueryString() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'facet': {",
+        "    'top_popular_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "      'domain': {",
+        "        'filter': '{!lucene df=\"popularity\" v=\"[5 TO 10]\"}'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+  }
+
+  @Test
+  public void testFacetWithArbitraryDomainFromQueryString() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': 'cat:electronics',",
+        "  'facet': {",
+        "    'top_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 3",
+        "      'domain': {",
+        "        'query': '*:*'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testFacetWithArbitraryDomainFromLocalParamsQuery() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': 'cat:electronics',",
+        "  'facet': {",
+        "    'largest_search_cats': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'domain': {",
+        "        'query': '{!lucene df=\"cat\" v=\"search\"}'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    assertHasFacetWithBucketValues(rawResponse,"largest_search_cats",
+        new FacetBucket("search",2),
+        new FacetBucket("software", 2));
+  }
+
+  /*
+   * Multiple query clauses are effectively AND'd together
+   */
+  public void testFacetWithMultipleSimpleQueryClausesInArbitraryDomain() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': 'cat:electronics',",
+        "  'facet': {",
+        "    'cats_matching_solr': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'domain': {",
+        "        'query': ['cat:search', 'name:Solr']",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+        new FacetBucket("search",1),
+        new FacetBucket("software", 1));
+  }
+
+  public void testFacetWithMultipleLocalParamsQueryClausesInArbitraryDomain() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': 'cat:electronics',",
+        "  'facet': {",
+        "    'cats_matching_solr': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'domain': {",
+        "        'query': ['{!lucene df=\"cat\" v=\"search\"}', '{!lucene df=\"name\" v=\"Solr\"}']",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+        new FacetBucket("search",1),
+        new FacetBucket("software", 1));
+  }
+
+  @Test
+  public void testFacetWithDomainWidenedUsingExcludeTagsToIgnoreFilters() throws Exception {
+    final String jsonBody = String.join("\n","{",
+        "  'query': '*:*',",
+        "  'filter': {'#on_shelf': 'inStock:true'},",
+        "  'facet': {",
+        "    'in_stock_only': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 2",
+        "    }",
+        "    'all': {",
+        "      'type': 'terms',",
+        "      'field': 'cat',",
+        "      'limit': 2,",
+        "      'domain': {",
+        "        'excludeTags': 'on_shelf'",
+        "      }",
+        "    }",
+        "  }",
+        "}");
+    final DirectJsonQueryRequest request = new DirectJsonQueryRequest(jsonBody);
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_IN_STOCK, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+
+    assertHasFacetWithBucketValues(rawResponse,"in_stock_only",
+        new FacetBucket("electronics",8),
+        new FacetBucket("currency", 4));
+    assertHasFacetWithBucketValues(rawResponse,"all",
+        new FacetBucket("electronics",12),
+        new FacetBucket("currency", 4));
+  }
+
+  private class FacetBucket {
+    private final Object val;
+    private final int count;
+    FacetBucket(Object val, int count) {
+      this.val = val;
+      this.count = count;
+    }
+
+    public Object getVal() { return val; }
+    public int getCount() { return count; }
+  }
+
+  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
+    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
+  }
+
+  private void assertHasStatFacetWithValue(NamedList<Object> rawResponse, String expectedFacetName, Double expectedStatValue) {
+    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
+    assertFacetResponseHasStatFacetWithValue(facetsTopLevel, expectedFacetName, expectedStatValue);
+  }
+
+  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
+    Object o = topLevelResponse.get("facets");
+    if (o == null) fail("Response has no top-level 'facets' property as expected");
+    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
+
+    return (NamedList<Object>) o;
+  }
+
+  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    Object o = facetResponse.get(expectedFacetName);
+    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
+    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
+
+    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
+    o = expectedFacetTopLevel.get("buckets");
+    if (o == null) fail("Response has no 'buckets' property under 'facets'");
+    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
+
+    final List<NamedList> bucketList = (List<NamedList>) o;
+    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
+        expectedBuckets.length, bucketList.size());
+    for (int i = 0; i < expectedBuckets.length; i++) {
+      final FacetBucket expectedBucket = expectedBuckets[i];
+      final NamedList<Object> actualBucket = bucketList.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
+      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+    }
+  }
+
+  private void assertFacetResponseHasStatFacetWithValue(NamedList<Object> facetResponse, String expectedFacetName, Double expectedStatValue) {
+    Object o = facetResponse.get(expectedFacetName);
+    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
+    if (!(o instanceof Number)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a Number");
+
+    final Number actualStatValueAsNumber = (Number) o;
+    final Double actualStatValueAsDouble = ((Number) o).doubleValue();
+    assertEquals(expectedStatValue, actualStatValueAsDouble, 0.5);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DomainMapTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DomainMapTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DomainMapTest.java
new file mode 100644
index 0000000..d437d14
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/DomainMapTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.request.json;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+
+public class DomainMapTest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testRejectsInvalidFilters() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .withFilter(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresFilterWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .withFilter("name:Solr");
+    final List<String> filterList = (List<String>) domain.get("filter");
+
+    assertTrue("Expected filter list to contain provided filter", filterList.contains("name:Solr"));
+  }
+
+  @Test
+  public void testStoresMultipleFilters() {
+    final DomainMap domain = new DomainMap()
+        .withFilter("name:Solr")
+        .withFilter("cat:search");
+    final List<String> filterList = (List<String>) domain.get("filter");
+
+    assertTrue("Expected filter list to contain 1st provided filter", filterList.contains("name:Solr"));
+    assertTrue("Expected filter list to contain 2nd provided filter", filterList.contains("cat:search"));
+  }
+
+  @Test
+  public void testRejectsInvalidQueries() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .withQuery(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresQueryWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .withQuery("name:Solr");
+    final List<String> queryList = (List<String>) domain.get("query");
+
+    assertTrue("Expected query list to contain provided query", queryList.contains("name:Solr"));
+  }
+
+  @Test
+  public void testStoresMultipleQueries() {
+    final DomainMap domain = new DomainMap()
+        .withQuery("name:Solr")
+        .withQuery("cat:search");
+    final List<String> queryList = (List<String>) domain.get("query");
+
+    assertTrue("Expected query list to contain 1st provided query", queryList.contains("name:Solr"));
+    assertTrue("Expected query list to contain 2nd provided query", queryList.contains("cat:search"));
+  }
+
+  @Test
+  public void testRejectsInvalidTagsToExclude() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .withTagsToExclude(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresTagsToExcludeWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .withTagsToExclude("BRAND");
+    final List<String> exclusionList = (List<String>) domain.get("excludeTags");
+
+    assertTrue("Expected tag-exclusion list to contain provided tag", exclusionList.contains("BRAND"));
+  }
+
+  @Test
+  public void testStoresMultipleTagExclusionStrings() {
+    final DomainMap domain = new DomainMap()
+        .withTagsToExclude("BRAND")
+        .withTagsToExclude("COLOR");
+    final List<String> exclusionList = (List<String>) domain.get("excludeTags");
+
+    assertTrue("Expected tag-exclusion list to contain provided 1st tag", exclusionList.contains("BRAND"));
+    assertTrue("Expected tag-exclusion list to contain provided 2nd tag", exclusionList.contains("COLOR"));
+  }
+
+  @Test
+  public void testRejectsInvalidBlockParentQuery() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .setBlockParentQuery(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresBlockParentQueryWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .setBlockParentQuery("content_type:product");
+    assertEquals("content_type:product", domain.get("blockParent"));
+  }
+
+  @Test
+  public void testRejectsInvalidBlockChildrenQuery() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .setBlockChildQuery(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresBlockChildrenQueryWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .setBlockChildQuery("content_type:productColors");
+    assertEquals("content_type:productColors", domain.get("blockChildren"));
+  }
+
+  @Test
+  public void testRejectsInvalidJoinFromParam() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .setJoinTransformation(null, "valid-to-field");
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidJoinToParam() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new DomainMap()
+          .setJoinTransformation("valid-from-field", null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresJoinValuesWithCorrectKey() {
+    final DomainMap domain = new DomainMap()
+        .setJoinTransformation("any-from-field", "any-to-field");
+
+    assertTrue(domain.containsKey("join"));
+    final Map<String, Object> joinParams = (Map<String, Object>) domain.get("join");
+    assertEquals("any-from-field", joinParams.get("from"));
+    assertEquals("any-to-field", joinParams.get("to"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/HeatmapFacetMapTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/HeatmapFacetMapTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/HeatmapFacetMapTest.java
new file mode 100644
index 0000000..9063714
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/HeatmapFacetMapTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.request.json;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+
+
+public class HeatmapFacetMapTest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testRejectsInvalidFieldName() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresFieldNameWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME");
+    assertEquals("ANY_FIELD_NAME", heatmapFacet.get("field"));
+  }
+
+  @Test
+  public void testDoesntSupportSubfacets() {
+    final Throwable thrown = expectThrows(UnsupportedOperationException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .withSubFacet("ANY_NAME", new TermsFacetMap("ANY_OTHER_FIELD_NAME"));
+    });
+    assertThat(thrown.getMessage(), containsString("doesn't currently support subfacets"));
+  }
+
+  @Test
+  public void testRejectsInvalidRegionQueries() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .setRegionQuery(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresRegionQueryWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME")
+        .setRegionQuery("[-120,-35 TO 50,60]");
+    assertEquals("[-120,-35 TO 50,60]", heatmapFacet.get("geom"));
+  }
+
+  @Test
+  public void testRejectsInvalidCellSize() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .setGridLevel(0);
+    });
+    assertThat(thrown.getMessage(), containsString("must be a positive integer"));
+  }
+
+  @Test
+  public void testStoresCellSizeWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME")
+        .setGridLevel(42);
+    assertEquals(42, heatmapFacet.get("gridLevel"));
+  }
+
+  @Test
+  public void testRejectsInvalidDistanceError() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .setDistErr(-1.0);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-negative"));
+  }
+
+  @Test
+  public void testStoresDistanceErrorWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME")
+        .setDistErr(4.5);
+    assertEquals(4.5, heatmapFacet.get("distErr"));
+  }
+
+  @Test
+  public void testRejectsInvalidDistanceErrorPercentageWithCorrectKey() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .setDistErrPct(2.0);
+    });
+    assertThat(thrown.getMessage(), containsString("must be between 0.0 and 1.0"));
+  }
+
+  @Test
+  public void testStoresDistanceErrorPercentageWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME")
+        .setDistErrPct(0.45);
+    assertEquals(0.45, heatmapFacet.get("distErrPct"));
+  }
+
+  @Test
+  public void testRejectsInvalidHeatmapFormat() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new HeatmapFacetMap("ANY_FIELD_NAME")
+          .setHeatmapFormat(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresHeatmapFormatWithCorrectKey() {
+    final HeatmapFacetMap heatmapFacet = new HeatmapFacetMap("ANY_FIELD_NAME")
+        .setHeatmapFormat(HeatmapFacetMap.HeatmapFormat.PNG);
+    assertEquals("png", heatmapFacet.get("format"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
new file mode 100644
index 0000000..7717f7f
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestFacetingIntegrationTest.java
@@ -0,0 +1,530 @@
+/*
+ * 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.request.json;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION_NAME = "techproducts";
+  private static final String CONFIG_NAME = "techproducts_config";
+  private static final int NUM_TECHPRODUCTS_DOCS = 32;
+  private static final int NUM_IN_STOCK = 17;
+  private static final int NUM_ELECTRONICS = 12;
+  private static final int NUM_CURRENCY = 4;
+  private static final int NUM_MEMORY = 3;
+  private static final int NUM_CORSAIR = 3;
+  private static final int NUM_BELKIN = 2;
+  private static final int NUM_CANON = 2;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig(CONFIG_NAME, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+        .configure();
+
+    final List<String> solrUrls = new ArrayList<>();
+    solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
+
+    ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
+    up.setParam("collection", COLLECTION_NAME);
+    up.addFile(getFile("solrj/techproducts.xml"), "application/xml");
+    up.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
+    UpdateResponse updateResponse = up.process(cluster.getSolrClient());
+    assertEquals(0, updateResponse.getStatus());
+  }
+
+  @Test
+  public void testSingleTermsFacet() throws Exception {
+    final TermsFacetMap categoriesFacetMap = new TermsFacetMap("cat")
+        .setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacetMap);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testFacetCanBeRepresentedByMapWriter() throws Exception {
+    final MapWriter categoriesFacet = new MapWriter() {
+      @Override
+      public void writeMap(EntryWriter ew) throws IOException {
+        ew.put("type", "terms");
+        ew.put("field", "cat");
+        ew.put("limit", 3);
+      }
+    };
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testMultiTermsFacet() throws Exception {
+    final TermsFacetMap categoriesFacetMap = new TermsFacetMap("cat")
+        .setLimit(3);
+    final TermsFacetMap manufacturersFacetMap = new TermsFacetMap("manu_id_s")
+        .setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacetMap)
+        .withFacet("top_manufacturers", manufacturersFacetMap);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_cats",
+        new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY),
+        new FacetBucket("memory", NUM_MEMORY));
+    assertHasFacetWithBucketValues(rawResponse,"top_manufacturers",
+        new FacetBucket("corsair",NUM_CORSAIR),
+        new FacetBucket("belkin", NUM_BELKIN),
+        new FacetBucket("canon", NUM_CANON));
+  }
+
+  @Test
+  public void testSingleRangeFacet() throws Exception {
+    final RangeFacetMap pricesFacet = new RangeFacetMap("price", 0, 100, 20);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("prices", pricesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"prices",
+        new FacetBucket(0.0f, 5),
+        new FacetBucket(20.0f, 0),
+        new FacetBucket(40.0f, 0),
+        new FacetBucket(60.0f, 1),
+        new FacetBucket(80.0f, 1));
+  }
+
+  @Test
+  public void testMultiRangeFacet() throws Exception {
+    final RangeFacetMap pricesFacet = new RangeFacetMap("price", 0, 100, 20);
+    final RangeFacetMap shippingWeightFacet = new RangeFacetMap("weight", 0, 200, 50);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("prices", pricesFacet)
+        .withFacet("shipping_weights", shippingWeightFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"prices",
+        new FacetBucket(0.0f, 5),
+        new FacetBucket(20.0f, 0),
+        new FacetBucket(40.0f, 0),
+        new FacetBucket(60.0f, 1),
+        new FacetBucket(80.0f, 1));
+    assertHasFacetWithBucketValues(rawResponse, "shipping_weights",
+        new FacetBucket(0.0f, 6),
+        new FacetBucket(50.0f, 0),
+        new FacetBucket(100.0f, 0),
+        new FacetBucket(150.0f,1));
+  }
+
+  @Test
+  public void testSingleStatFacet() throws Exception {
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withStatFacet("sum_price", "sum(price)");
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+  }
+
+  @Test
+  public void testMultiStatFacet() throws Exception {
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withStatFacet("sum_price", "sum(price)")
+        .withStatFacet("avg_price", "avg(price)");
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"sum_price", 5251.270030975342);
+    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+  }
+
+  @Test
+  public void testMultiFacetsMixedTypes() throws Exception {
+    final TermsFacetMap categoryFacet = new TermsFacetMap("cat")
+        .setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withStatFacet("avg_price", "avg(price)")
+        .withFacet("top_cats", categoryFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasStatFacetWithValue(rawResponse,"avg_price", 328.20437693595886);
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testNestedTermsFacet() throws Exception {
+    final TermsFacetMap categoriesFacet = new TermsFacetMap("cat")
+        .setLimit(3)
+        .withSubFacet("top_manufacturers_for_cat", new TermsFacetMap("manu_id_s").setLimit(1));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    // Test subfacet values for each top-level facet bucket
+    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
+    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
+    assertFacetResponseHasFacetWithBuckets(electronicsSubFacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
+    assertFacetResponseHasFacetWithBuckets(currencySubfacet, "top_manufacturers_for_cat", new FacetBucket("boa", 1));
+    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
+    assertFacetResponseHasFacetWithBuckets(memorySubfacet, "top_manufacturers_for_cat", new FacetBucket("corsair", 3));
+  }
+
+  @Test
+  public void testNestedFacetsOfMixedTypes() throws Exception {
+    final String subfacetName = "avg_price_for_cat";
+
+    final TermsFacetMap categoriesFacet = new TermsFacetMap("cat")
+        .setLimit(3)
+        .withStatSubFacet(subfacetName, "avg(price)");
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_cats", categoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    // Test top level facets
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+    // Test subfacet values for each top-level facet bucket
+    final List<NamedList<Object>> topLevelFacetResponse = (List<NamedList<Object>>) rawResponse.findRecursive("facets", "top_cats", "buckets");
+    final NamedList<Object> electronicsSubFacet = topLevelFacetResponse.get(0);
+    assertFacetResponseHasStatFacetWithValue(electronicsSubFacet, subfacetName, 252.02909261530095);
+    final NamedList<Object> currencySubfacet = topLevelFacetResponse.get(1);
+    assertFacetResponseHasStatFacetWithValue(currencySubfacet, subfacetName, 0.0);
+    final NamedList<Object> memorySubfacet = topLevelFacetResponse.get(2);
+    assertFacetResponseHasStatFacetWithValue(memorySubfacet, subfacetName, 129.99499893188477);
+  }
+
+  @Test
+  public void testFacetWithDomainFilteredBySimpleQueryString() throws Exception {
+    final TermsFacetMap popularCategoriesFacet = new TermsFacetMap("cat")
+        .setLimit(3)
+        .withDomain(new DomainMap()
+            .withFilter("popularity:[5 TO 10]"));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_popular_cats", popularCategoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+  }
+
+  @Test
+  public void testFacetWithDomainFilteredByLocalParamsQueryString() throws Exception {
+    final TermsFacetMap popularCategoriesFacet = new TermsFacetMap("cat")
+        .setLimit(3)
+        .withDomain(new DomainMap()
+            .withFilter("{!lucene df=\"popularity\" v=\"[5 TO 10]\"}"));
+
+    JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("top_popular_cats", popularCategoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_TECHPRODUCTS_DOCS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_popular_cats", new FacetBucket("electronics",9),
+        new FacetBucket("graphics card", 2), new FacetBucket("hard drive", 2));
+  }
+
+  @Test
+  public void testFacetWithArbitraryDomainFromQueryString() throws Exception {
+    final TermsFacetMap categoriesFacet = new TermsFacetMap("cat")
+        .setLimit(3)
+        .withDomain(new DomainMap()
+            .withQuery("*:*"));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("cat:electronics")
+        .withFacet("top_cats", categoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"top_cats", new FacetBucket("electronics",NUM_ELECTRONICS),
+        new FacetBucket("currency", NUM_CURRENCY), new FacetBucket("memory", NUM_MEMORY));
+  }
+
+  @Test
+  public void testFacetWithArbitraryDomainFromLocalParamsQuery() throws Exception {
+    final TermsFacetMap searchCategoriesFacet = new TermsFacetMap("cat")
+        .withDomain(new DomainMap()
+            .withQuery("{!lucene df=\"cat\" v=\"search\"}"));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("cat:electronics")
+        .withFacet("largest_search_cats", searchCategoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"largest_search_cats",
+        new FacetBucket("search",2),
+        new FacetBucket("software", 2));
+  }
+
+  public void testFacetWithMultipleSimpleQueryClausesInArbitraryDomain() throws Exception {
+    final TermsFacetMap solrCategoriesFacet = new TermsFacetMap("cat")
+        .withDomain(new DomainMap()
+            .withQuery("cat:search")
+            .withQuery("name:Solr"));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("cat:electronics")
+        .withFacet("cats_matching_solr", solrCategoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+        new FacetBucket("search",1),
+        new FacetBucket("software", 1));
+  }
+
+  public void testFacetWithMultipleLocalParamsQueryClausesInArbitraryDomain() throws Exception {
+    final TermsFacetMap solrCategoriesFacet = new TermsFacetMap("cat")
+        .withDomain(new DomainMap()
+            .withQuery("{!lucene df=\"cat\" v=\"search\"}")
+            .withQuery("{!lucene df=\"name\" v=\"Solr\"}"));
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("cat:electronics")
+        .withFacet("cats_matching_solr", solrCategoriesFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_ELECTRONICS, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"cats_matching_solr",
+        new FacetBucket("search",1),
+        new FacetBucket("software", 1));
+  }
+
+  @Test
+  public void testFacetWithDomainWidenedUsingExcludeTagsToIgnoreFilters() throws Exception {
+    final TermsFacetMap inStockFacet = new TermsFacetMap("cat")
+        .setLimit(2);
+    final TermsFacetMap allProductsFacet = new TermsFacetMap("cat")
+        .setLimit(2).withDomain(new DomainMap().withTagsToExclude("on_shelf"));
+    final Map<String, Object> taggedFilterMap = new HashMap<>();
+    taggedFilterMap.put("#on_shelf", "inStock:true");
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFilter(taggedFilterMap)
+        .withFacet("in_stock_only", inStockFacet)
+        .withFacet("all", allProductsFacet);
+
+    QueryResponse response = request.process(cluster.getSolrClient(), COLLECTION_NAME);
+
+    assertEquals(0, response.getStatus());
+    final SolrDocumentList returnedDocs = response.getResults();
+    assertEquals(NUM_IN_STOCK, returnedDocs.getNumFound());
+    assertEquals(10, returnedDocs.size());
+    final NamedList<Object> rawResponse = response.getResponse();
+    assertHasFacetWithBucketValues(rawResponse,"in_stock_only",
+        new FacetBucket("electronics",8),
+        new FacetBucket("currency", 4));
+    assertHasFacetWithBucketValues(rawResponse,"all",
+        new FacetBucket("electronics",12),
+        new FacetBucket("currency", 4));
+  }
+
+  private class FacetBucket {
+    private final Object val;
+    private final int count;
+    FacetBucket(Object val, int count) {
+      this.val = val;
+      this.count = count;
+    }
+
+    public Object getVal() { return val; }
+    public int getCount() { return count; }
+  }
+
+  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
+    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
+  }
+
+  private void assertHasStatFacetWithValue(NamedList<Object> rawResponse, String expectedFacetName, Double expectedStatValue) {
+    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
+    assertFacetResponseHasStatFacetWithValue(facetsTopLevel, expectedFacetName, expectedStatValue);
+  }
+
+  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
+    Object o = topLevelResponse.get("facets");
+    if (o == null) fail("Response has no top-level 'facets' property as expected");
+    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
+
+    return (NamedList<Object>) o;
+  }
+
+  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    Object o = facetResponse.get(expectedFacetName);
+    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
+    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
+
+    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
+    o = expectedFacetTopLevel.get("buckets");
+    if (o == null) fail("Response has no 'buckets' property under 'facets'");
+    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
+
+    final List<NamedList> bucketList = (List<NamedList>) o;
+    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
+        expectedBuckets.length, bucketList.size());
+    for (int i = 0; i < expectedBuckets.length; i++) {
+      final FacetBucket expectedBucket = expectedBuckets[i];
+      final NamedList<Object> actualBucket = bucketList.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
+      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+    }
+  }
+
+  private void assertFacetResponseHasStatFacetWithValue(NamedList<Object> facetResponse, String expectedFacetName, Double expectedStatValue) {
+    Object o = facetResponse.get(expectedFacetName);
+    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
+    if (!(o instanceof Number)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a Number");
+
+    final Number actualStatValueAsNumber = (Number) o;
+    final Double actualStatValueAsDouble = ((Number) o).doubleValue();
+    assertEquals(expectedStatValue, actualStatValueAsDouble, 0.5);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
index 84ef956..6e8c647 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/JsonQueryRequestUnitTest.java
@@ -25,7 +25,6 @@ import java.util.Map;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.request.RequestWriter;
-import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.MapWriter;
 import org.junit.Test;
@@ -98,6 +97,91 @@ public class JsonQueryRequestUnitTest extends LuceneTestCase {
   }
 
   @Test
+  public void testRejectsInvalidFacetName() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withFacet(null, new HashMap<>());
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+
+    thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withStatFacet(null, "avg(price)");
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidFacetMap() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withFacet("anyFacetName", (Map<String, Object>)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsNullFacetMapWriter() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withFacet("anyFacetName", (MapWriter)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidStatFacetString() {
+    Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new JsonQueryRequest().withStatFacet("anyFacetName", (String)null);
+    });
+    assertThat(thrown.getMessage(),containsString("must be non-null"));
+  }
+
+  @Test
+  public void testWritesProvidedFacetMapToJsonCorrectly() {
+    final Map<String, Object> categoryFacetMap = new HashMap<>();
+    categoryFacetMap.put("type", "terms");
+    categoryFacetMap.put("field", "category");
+    final JsonQueryRequest request = new JsonQueryRequest().withFacet("top_categories", categoryFacetMap);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"facet\":{\"top_categories\":{\"field\":\"category\",\"type\":\"terms\"}}"));
+  }
+
+  @Test
+  public void testWritesProvidedFacetMapWriterToJsonCorrectly() {
+    final MapWriter facetWriter = new MapWriter() {
+      @Override
+      public void writeMap(EntryWriter ew) throws IOException {
+        ew.put("type", "terms");
+        ew.put("field", "category");
+      }
+    };
+    final JsonQueryRequest request = new JsonQueryRequest().withFacet("top_categories", facetWriter);
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"facet\":{\"top_categories\":{\"type\":\"terms\",\"field\":\"category\"}}"));
+  }
+
+  @Test
+  public void testWritesProvidedStatFacetToJsonCorrectly() {
+    final JsonQueryRequest request = new JsonQueryRequest().withStatFacet("avg_price", "avg(price)");
+    final String requestBody = writeRequestToJson(request);
+    assertThat(requestBody, containsString("\"facet\":{\"avg_price\":\"avg(price)\"}"));
+  }
+
+  @Test
+  public void testWritesMultipleFacetMapsToJsonCorrectly() {
+    final Map<String, Object> facetMap1 = new HashMap<>();
+    facetMap1.put("type", "terms");
+    facetMap1.put("field", "a");
+    final Map<String, Object> facetMap2 = new HashMap<>();
+    facetMap2.put("type", "terms");
+    facetMap2.put("field", "b");
+    final JsonQueryRequest request = new JsonQueryRequest();
+
+    request.withFacet("facet1", facetMap1);
+    request.withFacet("facet2", facetMap2);
+    final String requestBody = writeRequestToJson(request);
+
+    assertThat(requestBody, containsString("\"facet\":{\"facet2\":{\"field\":\"b\",\"type\":\"terms\"},\"facet1\":{\"field\":\"a\",\"type\":\"terms\"}}"));
+  }
+
+  @Test
   public void testRejectsInvalidLimit() {
     Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
       new JsonQueryRequest().setLimit(-1);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/QueryFacetMapTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/QueryFacetMapTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/QueryFacetMapTest.java
new file mode 100644
index 0000000..8d07a4f
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/QueryFacetMapTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.request.json;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+
+public class QueryFacetMapTest extends SolrTestCaseJ4 {
+  @Test
+  public void testSetsFacetTypeToQuery() {
+    final QueryFacetMap queryFacet = new QueryFacetMap("any:query");
+    assertEquals("query", queryFacet.get("type"));
+  }
+
+  @Test
+  public void testRejectsInvalidQueryString() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final QueryFacetMap queryFacet = new QueryFacetMap(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testSetsQueryWithCorrectKey() {
+    final QueryFacetMap queryFacet = new QueryFacetMap("any:query");
+    assertEquals("any:query", queryFacet.get("q"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/RangeFacetMapTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/RangeFacetMapTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/RangeFacetMapTest.java
new file mode 100644
index 0000000..43bb5f5
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/RangeFacetMapTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.request.json;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+
+/**
+ * Unit tests for {@link RangeFacetMap}
+ */
+public class RangeFacetMapTest extends SolrTestCaseJ4 {
+  @Test
+  public void testRejectsInvalidFieldName() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new RangeFacetMap(null, 1, 2, 3);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testRejectsInvalidStartEndBounds() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new RangeFacetMap("ANY_FIELD_NAME", 1, -1, 3);
+    });
+    assertThat(thrown.getMessage(), containsString("'end' must be greater than parameter 'start'"));
+  }
+
+  @Test
+  public void testRejectsInvalidGap() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new RangeFacetMap("ANY_FIELD_NAME", 1, 2, -1);
+    });
+    assertThat(thrown.getMessage(), containsString("must be a positive integer"));
+  }
+
+  @Test
+  public void testStoresRequiredValuesWithCorrectKeys() {
+    final RangeFacetMap rangeFacet = new RangeFacetMap("ANY_FIELD_NAME", 1, 2, 3);
+    assertEquals("ANY_FIELD_NAME", rangeFacet.get("field"));
+    assertEquals(1L, rangeFacet.get("start"));
+    assertEquals(2L, rangeFacet.get("end"));
+    assertEquals(3L, rangeFacet.get("gap"));
+  }
+
+  @Test
+  public void testStoresHardEndWithCorrectKey() {
+    final RangeFacetMap rangeFacet = new RangeFacetMap("ANY_FIELD_NAME", 1, 2, 3)
+        .setHardEnd(true);
+    assertEquals(true, rangeFacet.get("hardend"));
+  }
+
+  @Test
+  public void testRejectsInvalidOtherBuckets() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      new RangeFacetMap("ANY_FIELD_NAME", 1, 2, 3)
+          .setOtherBuckets(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresOtherBucketsValueWithCorrectKey() {
+    final RangeFacetMap rangeFacet = new RangeFacetMap("ANY_FIELD_NAME", 1, 2, 3)
+        .setOtherBuckets(RangeFacetMap.OtherBuckets.BETWEEN);
+    assertEquals("between", rangeFacet.get("other"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/TermsFacetMapTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/TermsFacetMapTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/TermsFacetMapTest.java
new file mode 100644
index 0000000..58f807d
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/json/TermsFacetMapTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.request.json;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+import static org.junit.internal.matchers.StringContains.containsString;
+
+
+public class TermsFacetMapTest extends SolrTestCaseJ4 {
+  private static final String ANY_FIELD_NAME = "ANY_FIELD_NAME";
+
+  @Test
+  public void testSetsFacetTypeToTerm() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME);
+    assertEquals("terms", termsFacet.get("type"));
+  }
+
+  @Test
+  public void testStoresFieldWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME);
+    assertEquals(ANY_FIELD_NAME, termsFacet.get("field"));
+  }
+
+  @Test
+  public void testRejectsNegativeBucketOffset() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setBucketOffset(-1);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-negative"));
+  }
+
+  @Test
+  public void testStoresBucketOffsetWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setBucketOffset(2);
+    assertEquals(2, termsFacet.get("offset"));
+
+  }
+
+  @Test
+  public void testRejectsNegativeBucketLimit() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setLimit(-1);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-negative"));
+  }
+
+  @Test
+  public void testStoresBucketLimitWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setLimit(3);
+    assertEquals(3, termsFacet.get("limit"));
+  }
+
+  @Test
+  public void testRejectsInvalidSortString() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setSort(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresSortWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setSort("price asc");
+    assertEquals("price asc", termsFacet.get("sort"));
+  }
+
+  @Test
+  public void testRejectInvalidOverRequestBuckets() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setOverRequest(-2);
+    });
+    assertThat(thrown.getMessage(), containsString("must be >= -1"));
+  }
+
+  @Test
+  public void testStoresOverRequestBucketsWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setOverRequest(4);
+    assertEquals(4, termsFacet.get("overrequest"));
+  }
+
+  @Test
+  public void testStoresRefinementFlagWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .useDistributedFacetRefining(true);
+    assertEquals(true, termsFacet.get("refine"));
+  }
+
+  @Test
+  public void testRejectInvalidOverRefineBuckets() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setOverRefine(-2);
+    });
+    assertThat(thrown.getMessage(), containsString("must be >= -1"));
+  }
+
+  @Test
+  public void testStoresOverRefineBucketsWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setOverRefine(5);
+    assertEquals(5, termsFacet.get("overrefine"));
+  }
+
+  @Test
+  public void testRejectInvalidMinCount() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setMinCount(0);
+    });
+    assertThat(thrown.getMessage(), containsString("must be a positive integer"));
+  }
+
+  @Test
+  public void testStoresMinCountWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setMinCount(6);
+    assertEquals(6, termsFacet.get("mincount"));
+  }
+
+  @Test
+  public void testStoresNumBucketsFlagWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .includeTotalNumBuckets(true);
+    assertEquals(true, termsFacet.get("numBuckets"));
+  }
+
+  @Test
+  public void testStoresAllBucketsFlagWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .includeAllBucketsUnionBucket(true);
+    assertEquals(true, termsFacet.get("allBuckets"));
+  }
+
+  @Test
+  public void testRejectInvalidTermPrefix() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setTermPrefix(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresTermPrefixWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setTermPrefix("ANY_PREF");
+    assertEquals("ANY_PREF", termsFacet.get("prefix"));
+  }
+
+  @Test
+  public void testRejectsInvalidMethod() {
+    final Throwable thrown = expectThrows(IllegalArgumentException.class, () -> {
+      final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+          .setFacetMethod(null);
+    });
+    assertThat(thrown.getMessage(), containsString("must be non-null"));
+  }
+
+  @Test
+  public void testStoresMethodWithCorrectKey() {
+    final TermsFacetMap termsFacet = new TermsFacetMap(ANY_FIELD_NAME)
+        .setFacetMethod(TermsFacetMap.FacetMethod.STREAM);
+    assertEquals("stream", termsFacet.get("method"));
+  }
+}


[43/50] [abbrv] lucene-solr:jira/http2: LUCENE-8560: TestByteBuffersDirectory.testSeekPastEOF() failures with ByteArrayIndexInput. ByteArrayIndexInput removed entirely, without a replacement.

Posted by da...@apache.org.
LUCENE-8560: TestByteBuffersDirectory.testSeekPastEOF() failures with ByteArrayIndexInput. ByteArrayIndexInput removed entirely, without a replacement.


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

Branch: refs/heads/jira/http2
Commit: 4e2481b04b31ee0e5fb368fb69b47bb3da389030
Parents: 397b88a
Author: Dawid Weiss <dw...@apache.org>
Authored: Sat Nov 10 16:54:28 2018 +0100
Committer: Dawid Weiss <dw...@apache.org>
Committed: Sat Nov 10 16:54:28 2018 +0100

----------------------------------------------------------------------
 .../lucene/store/ByteArrayIndexInput.java       | 208 -------------------
 .../lucene/store/ByteBuffersDirectory.java      |  17 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |  16 +-
 .../lucene/store/BaseDirectoryTestCase.java     |  13 +-
 4 files changed, 27 insertions(+), 227 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e2481b0/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
deleted file mode 100644
index 9bf5ab2..0000000
--- a/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.store;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.Locale;
-
-/**
- * A {@link IndexInput} backed by a byte array.
- * 
- * @lucene.experimental
- */
-public final class ByteArrayIndexInput extends IndexInput implements RandomAccessInput {
-  private byte[] bytes;
-
-  private final int offset;
-  private final int length;
-
-  private int pos;
-
-  public ByteArrayIndexInput(String description, byte[] bytes) {
-    this(description, bytes, 0, bytes.length);
-  }
-  
-  public ByteArrayIndexInput(String description, byte[] bytes, int offs, int length) {
-    super(description);
-    this.offset = offs;
-    this.bytes = bytes;
-    this.length = length;
-    this.pos = offs;
-  }
-
-  public long getFilePointer() {
-    return pos - offset;
-  }
-  
-  public void seek(long pos) throws EOFException {
-    int newPos = Math.toIntExact(pos + offset);
-    try {
-      if (pos < 0 || pos > length) {
-        throw new EOFException();
-      }
-    } finally {
-      this.pos = newPos;
-    }
-  }
-
-  @Override
-  public long length() {
-    return length;
-  }
-
-  @Override
-  public short readShort() {
-    return (short) (((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF));
-  }
- 
-  @Override
-  public int readInt() {
-    return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
-      | ((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF);
-  }
- 
-  @Override
-  public long readLong() {
-    final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
-      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
-    final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
-      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
-    return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
-  }
-
-  @Override
-  public int readVInt() {
-    byte b = bytes[pos++];
-    if (b >= 0) return b;
-    int i = b & 0x7F;
-    b = bytes[pos++];
-    i |= (b & 0x7F) << 7;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7F) << 14;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7F) << 21;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    // Warning: the next ands use 0x0F / 0xF0 - beware copy/paste errors:
-    i |= (b & 0x0F) << 28;
-    if ((b & 0xF0) == 0) return i;
-    throw new RuntimeException("Invalid vInt detected (too many bits)");
-  }
- 
-  @Override
-  public long readVLong() {
-    byte b = bytes[pos++];
-    if (b >= 0) return b;
-    long i = b & 0x7FL;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 7;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 14;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 21;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 28;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 35;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 42;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 49;
-    if (b >= 0) return i;
-    b = bytes[pos++];
-    i |= (b & 0x7FL) << 56;
-    if (b >= 0) return i;
-    throw new RuntimeException("Invalid vLong detected (negative values disallowed)");
-  }
-
-  // NOTE: AIOOBE not EOF if you read too much
-  @Override
-  public byte readByte() {
-    return bytes[pos++];
-  }
-
-  // NOTE: AIOOBE not EOF if you read too much
-  @Override
-  public void readBytes(byte[] b, int offset, int len) {
-    System.arraycopy(bytes, pos, b, offset, len);
-    pos += len;
-  }
-
-  @Override
-  public void close() {
-    bytes = null;
-  }
-
-  @Override
-  public IndexInput clone() {
-    ByteArrayIndexInput slice = slice("(cloned)" + toString(), 0, length());
-    try {
-      slice.seek(getFilePointer());
-    } catch (EOFException e) {
-      throw new UncheckedIOException(e);
-    }
-    return slice;
-  }
-
-  public ByteArrayIndexInput slice(String sliceDescription, long offset, long length) {
-    if (offset < 0 || length < 0 || offset + length > this.length) {
-      throw new IllegalArgumentException(String.format(Locale.ROOT,
-          "slice(offset=%s, length=%s) is out of bounds: %s",
-          offset, length, this));
-    }
-
-    return new ByteArrayIndexInput(sliceDescription, this.bytes, Math.toIntExact(this.offset + offset), 
-        Math.toIntExact(length));
-  }
-
-  @Override
-  public byte readByte(long pos) throws IOException {
-    return bytes[Math.toIntExact(offset + pos)];
-  }
-
-  @Override
-  public short readShort(long pos) throws IOException {
-    int i = Math.toIntExact(offset + pos);
-    return (short) (((bytes[i]     & 0xFF) << 8) |
-                     (bytes[i + 1] & 0xFF));
-  }
-
-  @Override
-  public int readInt(long pos) throws IOException {
-    int i = Math.toIntExact(offset + pos);
-    return ((bytes[i]     & 0xFF) << 24) | 
-           ((bytes[i + 1] & 0xFF) << 16) | 
-           ((bytes[i + 2] & 0xFF) <<  8) | 
-            (bytes[i + 3] & 0xFF);
-  }
-
-  @Override
-  public long readLong(long pos) throws IOException {
-    return (((long) readInt(pos)) << 32) | 
-             (readInt(pos + 4) & 0xFFFFFFFFL);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e2481b0/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java
index 50fbc02..d1302dc 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java
@@ -68,21 +68,14 @@ public final class ByteBuffersDirectory extends BaseDirectory {
         return new ByteBuffersIndexInput(dataInput, inputName);
       };
 
-  public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY = 
-      (fileName, output) -> {
-        byte[] array = output.toArrayCopy();
-        String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
-            ByteArrayIndexInput.class.getSimpleName(),
-            fileName,
-            array.length);
-        return new ByteArrayIndexInput(inputName, array, 0, array.length);
-      };
+  public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY = OUTPUT_AS_ONE_BUFFER;
 
   public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE = 
       (fileName, output) -> {
         List<ByteBuffer> bufferList = output.toBufferList();
-        int chunkSizePower;
         bufferList.add(ByteBuffer.allocate(0));
+
+        int chunkSizePower;
         int blockSize = ByteBuffersDataInput.determineBlockPage(bufferList);
         if (blockSize == 0) {
           chunkSizePower = 30;
@@ -95,8 +88,8 @@ public final class ByteBuffersDirectory extends BaseDirectory {
             fileName);
 
         ByteBufferGuard guard = new ByteBufferGuard("none", (String resourceDescription, ByteBuffer b) -> {});
-        return ByteBufferIndexInput.newInstance(inputName, 
-            bufferList.toArray(new ByteBuffer [bufferList.size()]), 
+        return ByteBufferIndexInput.newInstance(inputName,
+            bufferList.toArray(new ByteBuffer [bufferList.size()]),
             output.size(), chunkSizePower, guard);
       };
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e2481b0/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index 29eb817..06a8a0c 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -19,6 +19,7 @@ package org.apache.lucene.replicator.nrt;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -42,7 +43,9 @@ import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
-import org.apache.lucene.store.ByteArrayIndexInput;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -242,7 +245,7 @@ public abstract class ReplicaNode extends Node {
         byte[] infosBytes = job.getCopyState().infosBytes;
 
         SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
-                                                         new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
+                                                         toIndexInput(job.getCopyState().infosBytes),
                                                          job.getCopyState().gen);
 
         // Must always commit to a larger generation than what's currently in the index:
@@ -382,7 +385,7 @@ public abstract class ReplicaNode extends Node {
       // Turn byte[] back to SegmentInfos:
       byte[] infosBytes = copyState.infosBytes;
       SegmentInfos infos = SegmentInfos.readCommit(dir,
-                                                   new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
+                                                   toIndexInput(copyState.infosBytes),
                                                    copyState.gen);
       assert infos.getVersion() == copyState.version;
 
@@ -439,6 +442,13 @@ public abstract class ReplicaNode extends Node {
                           markerCount));
   }
 
+  private ChecksumIndexInput toIndexInput(byte[] input) {
+    return new BufferedChecksumIndexInput(
+        new ByteBuffersIndexInput(
+            new ByteBuffersDataInput(
+                Arrays.asList(ByteBuffer.wrap(input))), "SegmentInfos"));
+  }
+
   /** Start a background copying job, to copy the specified files from the current primary node.  If files is null then the latest copy
    *  state should be copied.  If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
    *  2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e2481b0/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
index ab2921b..e49000b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
@@ -513,10 +513,15 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
       o.writeBytes(b, 0, len);
       o.close();
       IndexInput i = dir.openInput("out", newIOContext(random()));
-      expectThrows(EOFException.class, () -> {
-        i.seek(len + random().nextInt(2048));
-        i.readByte();
-      });
+
+      // Seeking past EOF should always throw EOFException
+      expectThrows(EOFException.class, () -> i.seek(len + RandomizedTest.randomIntBetween(1, 2048)));
+
+      // Seeking exactly to EOF should never throw any exception.
+      i.seek(len);
+
+      // But any read following the seek(len) should throw an EOFException.
+      expectThrows(EOFException.class, i::readByte);
 
       i.close();
     }


[26/50] [abbrv] lucene-solr:jira/http2: SOLR-12975: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-12975: 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/65dc312d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/65dc312d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/65dc312d

Branch: refs/heads/jira/http2
Commit: 65dc312daf495e5e55402d6024ea86b31ce8e4ff
Parents: 2d0e565
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 21:03:01 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 21:03:01 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/65dc312d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6f46bde..2f04c03 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -189,6 +189,8 @@ New Features
 
 * SOLR-12829: Add plist (parallel list) Streaming Expression (Joel Bernstein)
 
+* SOLR-12975: Add ltrim and rtrim Stream Evaluators (Joel Bernstein)
+
 Other Changes
 ----------------------
 


[06/50] [abbrv] lucene-solr:jira/http2: SOLR-12795: Fix precommit

Posted by da...@apache.org.
SOLR-12795: Fix precommit


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

Branch: refs/heads/jira/http2
Commit: b230543b47df4f9ff3de4414f4f787fc3286d60d
Parents: 3d94213
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Nov 6 11:53:00 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Nov 6 11:53:00 2018 -0500

----------------------------------------------------------------------
 .../client/solrj/io/stream/StreamExpressionToExpessionTest.java    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b230543b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
index 289b925..0b1c764 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.client.solrj.io.stream;
 
-import java.io.IOException;
-
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;


[42/50] [abbrv] lucene-solr:jira/http2: SOLR-12313: No need to wait for in-sync with leader in RecoveryAfterSoftCommitTest since we only care about recovery

Posted by da...@apache.org.
SOLR-12313: No need to wait for in-sync with leader in RecoveryAfterSoftCommitTest since we only care about recovery


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

Branch: refs/heads/jira/http2
Commit: 397b88aefa39d66d1310dfdea6b6d344ce1c9ce5
Parents: 42ee966
Author: Cao Manh Dat <da...@apache.org>
Authored: Sat Nov 10 06:01:19 2018 +0000
Committer: Cao Manh Dat <da...@apache.org>
Committed: Sat Nov 10 06:01:19 2018 +0000

----------------------------------------------------------------------
 .../test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java   | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/397b88ae/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
index 8290e12..a5cc04c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
@@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.util.TestInjection;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,6 +51,7 @@ public class RecoveryAfterSoftCommitTest extends AbstractFullDistribZkTestBase {
     System.setProperty("solr.ulog.numRecordsToKeep", String.valueOf(ULOG_NUM_RECORDS_TO_KEEP));
     // avoid creating too many files, see SOLR-7421
     System.setProperty("useCompoundFile", "true");
+    TestInjection.waitForReplicasInSync = null;
   }
 
   @AfterClass
@@ -57,6 +59,7 @@ public class RecoveryAfterSoftCommitTest extends AbstractFullDistribZkTestBase {
     System.clearProperty("solr.tests.maxBufferedDocs");
     System.clearProperty("solr.ulog.numRecordsToKeep");
     System.clearProperty("useCompoundFile");
+    TestInjection.reset();
   }
 
   /**


[29/50] [abbrv] lucene-solr:jira/http2: SOLR-12913: Add ltrim and rtrim to the Math Expressions User Guide

Posted by da...@apache.org.
SOLR-12913: Add ltrim and rtrim to the Math Expressions User Guide


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

Branch: refs/heads/jira/http2
Commit: 01397c1b88883593d741e2c744d316dc033a995a
Parents: 243a8a6
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Nov 8 15:29:15 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Nov 8 15:29:15 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/vector-math.adoc   | 35 +++++++++++++++++++++++++
 solr/solr-ref-guide/src/vectorization.adoc |  8 +++---
 2 files changed, 39 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/01397c1b/solr/solr-ref-guide/src/vector-math.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/vector-math.adoc b/solr/solr-ref-guide/src/vector-math.adoc
index 6171d77..cb4ef1d 100644
--- a/solr/solr-ref-guide/src/vector-math.adoc
+++ b/solr/solr-ref-guide/src/vector-math.adoc
@@ -143,6 +143,41 @@ When this expression is sent to the `/stream` handler it responds with:
 }
 ----
 
+Elements of an array can be trimmed using the `ltrim` (left trim) and `rtrim` (right trim) functions.
+The `ltrim` and `rtrim` functions remove a specific number of elements from the left or right of an array.
+
+The example below shows the `lrtim` function trimming the first 2 elements of an array:
+
+[source,text]
+----
+ltrim(array(0,1,2,3,4,5,6))
+----
+
+When this expression is sent to the `/stream` handler it responds with:
+
+[source,json]
+----
+{
+  "result-set": {
+    "docs": [
+      {
+        "return-value": [
+          2,
+          3,
+          4,
+          5,
+          6,
+        ]
+      },
+      {
+        "EOF": true,
+        "RESPONSE_TIME": 1
+      }
+    ]
+  }
+}
+----
+
 == Vector Sorting
 
 An array can be sorted in natural ascending order with the `asc` function.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/01397c1b/solr/solr-ref-guide/src/vectorization.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/vectorization.adoc b/solr/solr-ref-guide/src/vectorization.adoc
index 9483cc3..5c08a58 100644
--- a/solr/solr-ref-guide/src/vectorization.adoc
+++ b/solr/solr-ref-guide/src/vectorization.adoc
@@ -250,15 +250,15 @@ When this expression is sent to the `/stream` handler it responds with:
 
 == Facet Co-occurrence Matrices
 
-The `facet` function can be used to quickly perform mulit-dimension aggregations of categorical data from
+The `facet` function can be used to quickly perform multi-dimension aggregations of categorical data from
 records stored in a Solr Cloud collection. These multi-dimension aggregations can represent co-occurrence
 counts for the values in the dimensions. The `pivot` function can be used to move two dimensional
 aggregations into a co-occurrence matrix. The co-occurrence matrix can then be clustered or analyzed for
 correlations to learn about the hidden connections within the data.
 
-In the example below th `facet` expression is used to generate a two dimensional faceted aggregation.
+In the example below the `facet` expression is used to generate a two dimensional faceted aggregation.
 The first dimension is the US State that a car was purchased in and the second dimension is the car model.
-The two dimensional facet generates the co-occurrence counts for the number of times a particular car model
+This two dimensional facet generates the co-occurrence counts for the number of times a particular car model
 was purchased in a particular state.
 
 
@@ -311,7 +311,7 @@ When this expression is sent to the `/stream` handler it responds with:
 The `pivot` function can be used to move the facet results into a co-occurrence matrix. In the example below
 The `pivot` function is used to create a matrix where the rows of the matrix are the US States (state) and the
 columns of the matrix are the car models (model). The values in the matrix are the co-occurrence counts (count(*))
- from facet results.  Once the co-occurrence matrix has been created the US States can be clustered
+ from the facet results.  Once the co-occurrence matrix has been created the US States can be clustered
 by car model, or the matrix can be transposed and car models can be clustered by the US States
 where they were bought.
 


[18/50] [abbrv] lucene-solr:jira/http2: SOLR-12936: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-12936: 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/a1979047
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a1979047
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a1979047

Branch: refs/heads/jira/http2
Commit: a197904743a2e729c8327c2d651e24f5e42d5e14
Parents: 7bde5e2
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 09:43:41 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 09:43:41 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1979047/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 35bd675..00588bb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -185,6 +185,7 @@ New Features
 
 * SOLR-11572: Add recip Stream Evaluator to support reciprocal transformations (Joel Bernstein)
 
+* SOLR-12936: Allow percentiles Stream Evaluator to accept an array of percentiles to calculate (Joel bernstein)
 
 Other Changes
 ----------------------


[21/50] [abbrv] lucene-solr:jira/http2: SOLR-12913: Add new facet expression and pivot docs

Posted by da...@apache.org.
SOLR-12913: Add new facet expression and pivot docs


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

Branch: refs/heads/jira/http2
Commit: 531b16633acc8c398a20ca2a52b7ded3901702e6
Parents: ff1df8a
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 15:07:21 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 15:07:46 2018 -0500

----------------------------------------------------------------------
 .../src/stream-source-reference.adoc            | 14 +++-
 solr/solr-ref-guide/src/vectorization.adoc      | 80 ++++++++++++++++++++
 2 files changed, 90 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/531b1663/solr/solr-ref-guide/src/stream-source-reference.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/stream-source-reference.adoc b/solr/solr-ref-guide/src/stream-source-reference.adoc
index c31639a..c83991e 100644
--- a/solr/solr-ref-guide/src/stream-source-reference.adoc
+++ b/solr/solr-ref-guide/src/stream-source-reference.adoc
@@ -130,8 +130,12 @@ The `facet` function provides aggregations that are rolled up over buckets. Unde
 * `collection`: (Mandatory) Collection the facets will be aggregated from.
 * `q`: (Mandatory) The query to build the aggregations from.
 * `buckets`: (Mandatory) Comma separated list of fields to rollup over. The comma separated list represents the dimensions in a multi-dimensional rollup.
-* `bucketSorts`: Comma separated list of sorts to apply to each dimension in the buckets parameters. Sorts can be on the computed metrics or on the bucket values.
-* `bucketSizeLimit`: The number of buckets to include. This value is applied to each dimension. '-1' will fetch all the buckets.
+* `bucketSorts`: (Mandatory) Comma separated list of sorts to apply to each dimension in the buckets parameters. Sorts can be on the computed metrics or on the bucket values.
+* `rows`: (Default 10) The number of rows to return. '-1' will return all rows.
+* `offset`:(Default 0) The offset in the result set to start from.
+* `overfetch`: (Default 150) Over-fetching is used to provide accurate aggregations over high cardinality fields.
+* `method`: The JSON facet API aggregation method.
+* `bucketSizeLimit`: Sets the absolute number of rows to fetch. This is incompatible with rows, offset and overfetch. This value is applied to each dimension. '-1' will fetch all the buckets.
 * `metrics`: List of metrics to compute for the buckets. Currently supported metrics are `sum(col)`, `avg(col)`, `min(col)`, `max(col)`, `count(*)`.
 
 === facet Syntax
@@ -144,7 +148,7 @@ facet(collection1,
       q="*:*",
       buckets="a_s",
       bucketSorts="sum(a_i) desc",
-      bucketSizeLimit=100,
+      rows=100,
       sum(a_i),
       sum(a_f),
       min(a_i),
@@ -166,7 +170,8 @@ facet(collection1,
       q="*:*",
       buckets="year_i, month_i, day_i",
       bucketSorts="year_i desc, month_i desc, day_i desc",
-      bucketSizeLimit=100,
+      rows=10,
+      offset=20,
       sum(a_i),
       sum(a_f),
       min(a_i),
@@ -179,6 +184,7 @@ facet(collection1,
 ----
 
 The example above shows a facet function with rollups over three buckets, where the buckets are returned in descending order by bucket value.
+The rows param returns 10 rows and the offset param starts returning rows from the 20th row.
 
 == features
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/531b1663/solr/solr-ref-guide/src/vectorization.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/vectorization.adoc b/solr/solr-ref-guide/src/vectorization.adoc
index 5fdfadc..acd56ec 100644
--- a/solr/solr-ref-guide/src/vectorization.adoc
+++ b/solr/solr-ref-guide/src/vectorization.adoc
@@ -31,6 +31,12 @@ to vectorize and analyze the results sets.
 
 Below are some of the key stream sources:
 
+* *`facet`*: Multi-dimensional aggregations are a powerful tool for generating
+co-occurrence counts for categorical data. The `facet` function uses the JSON facet API
+under the covers to provide fast, distributed, multi-dimension aggregations. With math expressions
+the aggregated results can be pivoted into a co-occurance matrix which can be mined for
+correlations and hidden similarities within the data.
+
 * *`random`*: Random sampling is widely used in statistics, probability and machine learning.
 The `random` function returns a random sample of search results that match a
 query. The random samples can be vectorized and operated on by math expressions and the results
@@ -242,6 +248,80 @@ When this expression is sent to the `/stream` handler it responds with:
 }
 ----
 
+== Facet Co-Occurrence Matrices
+
+The `facet` function can be used to quickly perform mulit-dimension aggregations of categorical data from
+records stored in a Solr Cloud collection. These multi-dimension aggregations can represent co-occurrence
+counts for the values in the dimensions. The `pivot` function can be used to move two dimensional
+aggregations into a co-occurrence matrix. The co-occurrence matrix can then be clustered or analyzed for
+correlations to learn about the hidden connections within the data.
+
+In the example below th `facet` expression is used to generate a two dimensional faceted aggregation.
+The first dimension is the US State that a car was purchased in and the second dimension is the car model.
+The two dimensional facet generates the co-occurrence counts for the number of times a particular car model
+was purchased in a particular state.
+
+
+[source,text]
+----
+facet(collection1, q="*:*", buckets="state, model", bucketSorts="count(*) desc", rows=5, count(*))
+----
+
+When this expression is sent to the `/stream` handler it responds with:
+
+[source,json]
+----
+{
+  "result-set": {
+    "docs": [
+      {
+        "state": "NY",
+        "model": "camry",
+        "count(*)": 13342
+      },
+      {
+        "state": "NJ",
+        "model": "accord",
+        "count(*)": 13002
+      },
+      {
+        "state": "NY",
+        "model": "civic",
+        "count(*)": 12901
+      },
+      {
+        "state": "CA",
+        "model": "focus",
+        "count(*)": 12892
+      },
+      {
+        "state": "TX",
+        "model": "f150",
+        "count(*)": 12871
+      },
+      {
+        "EOF": true,
+        "RESPONSE_TIME": 171
+      }
+    ]
+  }
+}
+----
+
+The `pivot` function can be used to move the facet results into a co-occurrence matrix. In the example below
+The `pivot` function is used to create a matrix where the rows of the matrix are the US States (state) and the
+columns of the matrix are the car models (model). The values in the matrix are the co-occurrence counts (count(*))
+ from facet results.  Once the co-occurrence matrix has been created the US States can be clustered
+by car model, or the matrix can be transposed and car models can be clustered by the US States
+where they were bought.
+
+[source,text]
+----
+let(a=facet(collection1, q="*:*", buckets="state, model", bucketSorts="count(*) desc", rows="-1", count(*)),
+    b=pivot(a, state, model, count(*)),
+    c=kmeans(b, 7))
+----
+
 == Latitude / Longitude Vectors
 
 The `latlonVectors` function wraps a list of tuples and parses a lat/lon location field into


[16/50] [abbrv] lucene-solr:jira/http2: SOLR-12795: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-12795: 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/bc88d78c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/bc88d78c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/bc88d78c

Branch: refs/heads/jira/http2
Commit: bc88d78c84db42ee971f4a8d449ba3b037817d5a
Parents: e1bdb9a
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 09:28:47 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 09:28:47 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc88d78c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index bd4570d..1ddb471 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -181,6 +181,9 @@ New Features
 
 * SOLR-12971: Add pivot Stream Evaluator to pivot facet co-occurrence counts into a matrix (Joel Bernstein)
 
+* SOLR-12795: Introduce 'rows' and 'offset' parameter in FacetStream (Joel Bernstein, Amrit Sarkar, Varun Thacker)
+
+
 Other Changes
 ----------------------
 


[39/50] [abbrv] lucene-solr:jira/http2: Add 7.7.0 version

Posted by da...@apache.org.
Add 7.7.0 version


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

Branch: refs/heads/jira/http2
Commit: 6d9c714052a1a34eee8b56fd5b6599accdbe7423
Parents: 0330372
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Nov 9 15:20:33 2018 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Fri Nov 9 15:20:33 2018 -0600

----------------------------------------------------------------------
 lucene/CHANGES.txt                                  |  3 +++
 .../src/java/org/apache/lucene/util/Version.java    |  7 +++++++
 solr/CHANGES.txt                                    | 16 ++++++++++++++++
 3 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6d9c7140/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c3ff482..1a00644 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -192,6 +192,9 @@ Optimizations
 * LUCENE-8507: TopFieldCollector can now update the minimum competitive score if the primary sort
   is by relevancy and the total hit count is not required. (Jim Ferenczi)
 
+======================= Lucene 7.7.0 =======================
+(No Changes)
+
 ======================= Lucene 7.6.0 =======================
 
 Build:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6d9c7140/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 808e56b..9a43d4c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -104,6 +104,13 @@ public final class Version {
   public static final Version LUCENE_7_6_0 = new Version(7, 6, 0);
 
   /**
+   * Match settings and bugs in Lucene's 7.7.0 release.
+   * @deprecated Use latest
+   */
+  @Deprecated
+  public static final Version LUCENE_7_7_0 = new Version(7, 7, 0);
+
+  /**
    * Match settings and bugs in Lucene's 8.0.0 release.
    * <p>
    * Use this to get the latest &amp; greatest settings, bug

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6d9c7140/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3405384..a13e5cb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -98,6 +98,22 @@ Other Changes
 
 * SOLR-12620: Remove the Admin UI Cloud -> Graph (Radial) view (janhoy)
 
+==================  7.7.0 ==================
+
+Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+
+Versions of Major Components
+---------------------
+Apache Tika 1.19.1
+Carrot2 3.16.0
+Velocity 1.7 and Velocity Tools 2.0
+Apache ZooKeeper 3.4.11
+Jetty 9.4.11.v20180605
+
+
+(No Changes)
+
+
 ==================  7.6.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.


[23/50] [abbrv] lucene-solr:jira/http2: LUCENE-8558: add changes entry

Posted by da...@apache.org.
LUCENE-8558: add changes entry


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

Branch: refs/heads/jira/http2
Commit: 4794a1617c1d82fdafa8e6a3a5c1291d387f1add
Parents: 9594ee3
Author: Simon Willnauer <si...@apache.org>
Authored: Wed Nov 7 22:16:13 2018 +0100
Committer: Simon Willnauer <si...@apache.org>
Committed: Wed Nov 7 22:16:47 2018 +0100

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4794a161/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index dadc53d..586ad20 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -259,6 +259,8 @@ Improvements:
   the same cached instance.  MemoryIndex's impl now pre-creates the FieldInfos instead of
   re-calculating a new instance each time.  (Tim Underwood, David Smiley)
 
+* LUCENE-8558: Replace O(N) lookup with O(1) lookup in PerFieldMergeState#FilterFieldInfos.
+  (Kranthi via Simon Willnauer)
 Other:
 
 * LUCENE-8523: Correct typo in JapaneseNumberFilterFactory javadocs (Ankush Jhalani


[40/50] [abbrv] lucene-solr:jira/http2: LUCENE-8537: ant test command fails under lucene/tools

Posted by da...@apache.org.
LUCENE-8537: ant test command fails under lucene/tools


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

Branch: refs/heads/jira/http2
Commit: efd3f17f9a98aa9544e8af5126ae892fbc14728c
Parents: 6d9c714
Author: Uwe Schindler <us...@apache.org>
Authored: Fri Nov 9 23:53:14 2018 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Fri Nov 9 23:53:14 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt     | 4 ++++
 lucene/tools/build.xml | 4 +++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/efd3f17f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 1a00644..54fda0b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -195,6 +195,10 @@ Optimizations
 ======================= Lucene 7.7.0 =======================
 (No Changes)
 
+Build:
+
+* LUCENE-8537: ant test command fails under lucene/tools (Peter Somogyi)
+
 ======================= Lucene 7.6.0 =======================
 
 Build:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/efd3f17f/lucene/tools/build.xml
----------------------------------------------------------------------
diff --git a/lucene/tools/build.xml b/lucene/tools/build.xml
index 4736ea5..b245dce 100644
--- a/lucene/tools/build.xml
+++ b/lucene/tools/build.xml
@@ -33,8 +33,10 @@
 
   <path id="test.classpath"/>
 
-  <!-- redefine the test compilation, so it's just a no-op -->
+  <!-- redefine the test compilation, -test and -check-totals, so these are just no-ops -->
   <target name="compile-test"/>
+  <target name="-test"/>
+  <target name="-check-totals"/>
   
   <!-- redefine the forbidden apis to be no-ops -->
   <target name="-check-forbidden-tests"/>


[33/50] [abbrv] lucene-solr:jira/http2: SOLR-12880: Show the FacetProcessor class name instead of the FacetRequest in the JSON Facets debug-trace output

Posted by da...@apache.org.
SOLR-12880: Show the FacetProcessor class name instead of the
 FacetRequest in the JSON Facets debug-trace output


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

Branch: refs/heads/jira/http2
Commit: fbb987282e764aded31d7867e0d8f1fd6e2f02d4
Parents: 780305c
Author: Tim Underwood <ti...@gmail.com>
Authored: Thu Nov 8 23:32:26 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Nov 8 23:32:26 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/search/facet/FacetHeatmap.java  | 103 ++++++++++---------
 .../apache/solr/search/facet/FacetRequest.java  |   2 +-
 3 files changed, 59 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fbb98728/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b8f9f18..3c58d35 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -296,6 +296,9 @@ Improvements
 * SOLR-12964: Json Facets: use DocValuesIterator advanceExact() instead of advance() in FacetFieldProcessorByHashDV and
   UniqueSinglevaluedSlotAcc. (Tim Underwood)
 
+* SOLR-12880: Json Facets: Show the FacetProcessor class name instead of the FacetRequest in the JSON Facets debug-trace
+  output (Tim Underwood)
+
 ==================  7.5.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/fbb98728/solr/core/src/java/org/apache/solr/search/facet/FacetHeatmap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetHeatmap.java b/solr/core/src/java/org/apache/solr/search/facet/FacetHeatmap.java
index b5e481a..c3386e2 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetHeatmap.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetHeatmap.java
@@ -208,62 +208,69 @@ public class FacetHeatmap extends FacetRequest {
 
   @Override
   public FacetProcessor createFacetProcessor(FacetContext fcontext) {
-    return new FacetProcessor(fcontext, this) {
-      @Override
-      public void process() throws IOException {
-        super.process(); // handles domain changes
-
-        //Compute!
-        final HeatmapFacetCounter.Heatmap heatmap;
-        try {
-          heatmap = HeatmapFacetCounter.calcFacets(
-              strategy,
-              fcontext.searcher.getTopReaderContext(),
-              getTopAcceptDocs(fcontext.base, fcontext.searcher), // turn DocSet into Bits
-              boundsShape,
-              gridLevel,
-              maxCells);
-        } catch (IllegalArgumentException e) {//e.g. too many cells
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e.toString(), e);
-        }
-
-        //Populate response
-        response = new SimpleOrderedMap();
-        response.add("gridLevel", gridLevel);
-        response.add("columns", heatmap.columns);
-        response.add("rows", heatmap.rows);
-        response.add("minX", heatmap.region.getMinX());
-        response.add("maxX", heatmap.region.getMaxX());
-        response.add("minY", heatmap.region.getMinY());
-        response.add("maxY", heatmap.region.getMaxY());
+    return new FacetHeatmapProcessor(fcontext);
+  }
 
-        //A shard request will always be a PNG
-        String format = fcontext.isShard() ? FORMAT_PNG : FacetHeatmap.this.format;
+  // don't use an anonymous class since the getSimpleName() isn't friendly in debug output
+  private class FacetHeatmapProcessor extends FacetProcessor {
+    public FacetHeatmapProcessor(FacetContext fcontext) {
+      super(fcontext, FacetHeatmap.this);
+    }
 
-        response.add("counts_" + format, formatCountsVal(format, heatmap.columns, heatmap.rows, heatmap.counts, fcontext.getDebugInfo()));
+    @Override
+    public void process() throws IOException {
+      super.process(); // handles domain changes
 
-        // note: we do not call processStats or processSubs as it's not supported yet
+      //Compute!
+      final HeatmapFacetCounter.Heatmap heatmap;
+      try {
+        heatmap = HeatmapFacetCounter.calcFacets(
+            strategy,
+            fcontext.searcher.getTopReaderContext(),
+            getTopAcceptDocs(fcontext.base, fcontext.searcher), // turn DocSet into Bits
+            boundsShape,
+            gridLevel,
+            maxCells);
+      } catch (IllegalArgumentException e) {//e.g. too many cells
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e.toString(), e);
       }
 
-      //TODO this is a general utility that should go elsewhere?  DocSetUtil?  Then should DocSetBase.getBits go away?
-      private Bits getTopAcceptDocs(DocSet docSet, SolrIndexSearcher searcher) throws IOException {
-        if (docSet.size() == searcher.numDocs()) {
-          return null; // means match everything (all live docs). This can speedup things a lot.
-        } else if (docSet.size() == 0) {
-          return new Bits.MatchNoBits(searcher.maxDoc()); // can speedup things a lot
-        } else if (docSet instanceof BitDocSet) {
-          return ((BitDocSet) docSet).getBits();
-        } else {
-          // TODO DocSetBase.getBits ought to be at DocSet level?  Though it doesn't know maxDoc but it could?
-          FixedBitSet bits = new FixedBitSet(searcher.maxDoc());
-          for (DocIterator iter = docSet.iterator(); iter.hasNext();) {
-            bits.set(iter.nextDoc());
-          }
-          return bits;
+      //Populate response
+      response = new SimpleOrderedMap();
+      response.add("gridLevel", gridLevel);
+      response.add("columns", heatmap.columns);
+      response.add("rows", heatmap.rows);
+      response.add("minX", heatmap.region.getMinX());
+      response.add("maxX", heatmap.region.getMaxX());
+      response.add("minY", heatmap.region.getMinY());
+      response.add("maxY", heatmap.region.getMaxY());
+
+      //A shard request will always be a PNG
+      String format = fcontext.isShard() ? FORMAT_PNG : FacetHeatmap.this.format;
+
+      response.add("counts_" + format, formatCountsVal(format, heatmap.columns, heatmap.rows, heatmap.counts, fcontext.getDebugInfo()));
+
+      // note: we do not call processStats or processSubs as it's not supported yet
+    }
+
+    //TODO this is a general utility that should go elsewhere?  DocSetUtil?  Then should DocSetBase.getBits go away?
+    private Bits getTopAcceptDocs(DocSet docSet, SolrIndexSearcher searcher) throws IOException {
+      if (docSet.size() == searcher.numDocs()) {
+        return null; // means match everything (all live docs). This can speedup things a lot.
+      } else if (docSet.size() == 0) {
+        return new Bits.MatchNoBits(searcher.maxDoc()); // can speedup things a lot
+      } else if (docSet instanceof BitDocSet) {
+        return ((BitDocSet) docSet).getBits();
+      } else {
+        // TODO DocSetBase.getBits ought to be at DocSet level?  Though it doesn't know maxDoc but it could?
+        FixedBitSet bits = new FixedBitSet(searcher.maxDoc());
+        for (DocIterator iter = docSet.iterator(); iter.hasNext();) {
+          bits.set(iter.nextDoc());
         }
+        return bits;
       }
+    }
 
-    };
   }
 
   private static Object formatCountsVal(String format, int columns, int rows, int[] counts, FacetDebugInfo debugInfo) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fbb98728/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
index 658e0fc..4135c87 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
@@ -371,7 +371,7 @@ public abstract class FacetRequest {
         debugInfo.setFilter(fcontext.filter.toString());
       }
       debugInfo.setReqDescription(getFacetDescription());
-      debugInfo.setProcessor(getClass().getSimpleName());
+      debugInfo.setProcessor(facetProcessor.getClass().getSimpleName());
       debugInfo.putInfoItem("domainSize", (long) fcontext.base.size());
       RTimer timer = new RTimer();
       facetProcessor.process();


[49/50] [abbrv] lucene-solr:jira/http2: SOLR-12969: Inconsistency with leader when PeerSync return ALREADY_IN_SYNC

Posted by da...@apache.org.
SOLR-12969: Inconsistency with leader when PeerSync return ALREADY_IN_SYNC


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

Branch: refs/heads/jira/http2
Commit: f357c06276139defa26d0569fe5903cfd3d66cdb
Parents: cd1e829
Author: Cao Manh Dat <da...@apache.org>
Authored: Mon Nov 12 10:10:22 2018 +0000
Committer: Cao Manh Dat <da...@apache.org>
Committed: Mon Nov 12 10:10:22 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/solr/update/PeerSync.java   | 158 ++++++++++---------
 .../apache/solr/update/PeerSyncWithLeader.java  |  59 ++++++-
 .../apache/solr/cloud/HttpPartitionTest.java    |   2 +-
 .../apache/solr/cloud/TestCloudRecovery2.java   | 143 +++++++++++++++++
 .../org/apache/solr/update/PeerSyncTest.java    |  50 +++---
 .../solr/update/PeerSyncWithLeaderTest.java     |  18 +++
 6 files changed, 325 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/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 cb7b918..8fd3bef 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -692,33 +692,18 @@ public class PeerSync implements SolrMetricProducer {
     }
   }
 
-  /**
-   * Helper class for doing comparison ourUpdates and other replicas's updates to find the updates that we missed
-   */
-  public static class MissedUpdatesFinder {
-    private List<Long> ourUpdates;
+  static abstract class MissedUpdatesFinderBase {
     private Set<Long> ourUpdateSet;
-    private Set<Long> requestedUpdateSet;
+    private Set<Long> requestedUpdateSet = new HashSet<>();
 
-    private long ourLowThreshold;  // 20th percentile
-    private long ourHighThreshold; // 80th percentile
-    private long ourHighest;  // currently just used for logging/debugging purposes
-    private String logPrefix;
-    private long nUpdates;
+    long ourLowThreshold;  // 20th percentile
+    List<Long> ourUpdates;
 
-    MissedUpdatesFinder(List<Long> ourUpdates, String logPrefix, long nUpdates,
-                        long ourLowThreshold, long ourHighThreshold) {
+    MissedUpdatesFinderBase(List<Long> ourUpdates, long ourLowThreshold) {
       assert sorted(ourUpdates);
-
-      this.logPrefix = logPrefix;
       this.ourUpdates = ourUpdates;
-      this.ourLowThreshold = ourLowThreshold;
-      this.ourHighThreshold = ourHighThreshold;
-      this.ourHighest = ourUpdates.get(0);
-      this.nUpdates = nUpdates;
-
       this.ourUpdateSet = new HashSet<>(ourUpdates);
-      this.requestedUpdateSet = new HashSet<>();
+      this.ourLowThreshold = ourLowThreshold;
     }
 
     private boolean sorted(List<Long> list) {
@@ -730,61 +715,7 @@ public class PeerSync implements SolrMetricProducer {
       return true;
     }
 
-    public MissedUpdatesRequest find(List<Long> otherVersions, Object updateFrom, Supplier<Boolean> canHandleVersionRanges) {
-      otherVersions.sort(absComparator);
-      if (debug) {
-        log.debug("{} sorted versions from {} = {}", logPrefix, otherVersions, updateFrom);
-      }
-
-      long otherHigh = percentile(otherVersions, .2f);
-      long otherLow = percentile(otherVersions, .8f);
-      long otherHighest = otherVersions.get(0);
-
-      if (ourHighThreshold < otherLow) {
-        // Small overlap between version windows and ours is older
-        // This means that we might miss updates if we attempted to use this method.
-        // Since there exists just one replica that is so much newer, we must
-        // fail the sync.
-        log.info("{} Our versions are too old. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
-            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
-        return MissedUpdatesRequest.UNABLE_TO_SYNC;
-      }
-
-      if (ourLowThreshold > otherHigh) {
-        // Small overlap between windows and ours is newer.
-        // Using this list to sync would result in requesting/replaying results we don't need
-        // and possibly bringing deleted docs back to life.
-        log.info("{} Our versions are newer. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
-            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
-
-        // Because our versions are newer, IndexFingerprint with the remote would not match us.
-        // We return true on our side, but the remote peersync with us should fail.
-        return MissedUpdatesRequest.ALREADY_IN_SYNC;
-      }
-
-      boolean completeList = otherVersions.size() < nUpdates;
-
-      MissedUpdatesRequest updatesRequest;
-      if (canHandleVersionRanges.get()) {
-        updatesRequest = handleVersionsWithRanges(otherVersions, completeList);
-      } else {
-        updatesRequest = handleIndividualVersions(otherVersions, completeList);
-      }
-
-      if (updatesRequest.totalRequestedUpdates > nUpdates) {
-        log.info("{} PeerSync will fail because number of missed updates is more than:{}", logPrefix, nUpdates);
-        return MissedUpdatesRequest.UNABLE_TO_SYNC;
-      }
-
-      if (updatesRequest == MissedUpdatesRequest.EMPTY) {
-        log.info("{} No additional versions requested. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
-            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
-      }
-
-      return updatesRequest;
-    }
-
-    private MissedUpdatesRequest handleVersionsWithRanges(List<Long> otherVersions, boolean completeList) {
+    MissedUpdatesRequest handleVersionsWithRanges(List<Long> otherVersions, boolean completeList) {
       // we may endup asking for updates for too many versions, causing 2MB post payload limit. Construct a range of
       // versions to request instead of asking individual versions
       List<String> rangesToRequest = new ArrayList<>();
@@ -829,7 +760,7 @@ public class PeerSync implements SolrMetricProducer {
       return MissedUpdatesRequest.of(rangesToRequestStr, totalRequestedVersions);
     }
 
-    private MissedUpdatesRequest handleIndividualVersions(List<Long> otherVersions, boolean completeList) {
+    MissedUpdatesRequest handleIndividualVersions(List<Long> otherVersions, boolean completeList) {
       List<Long> toRequest = new ArrayList<>();
       for (Long otherVersion : otherVersions) {
         // stop when the entries get old enough that reorders may lead us to see updates we don't need
@@ -848,7 +779,80 @@ public class PeerSync implements SolrMetricProducer {
 
       return MissedUpdatesRequest.of(StrUtils.join(toRequest, ','), toRequest.size());
     }
+  }
+
+  /**
+   * Helper class for doing comparison ourUpdates and other replicas's updates to find the updates that we missed
+   */
+  public static class MissedUpdatesFinder extends MissedUpdatesFinderBase {
+    private long ourHighThreshold; // 80th percentile
+    private long ourHighest;  // currently just used for logging/debugging purposes
+    private String logPrefix;
+    private long nUpdates;
+
+    MissedUpdatesFinder(List<Long> ourUpdates, String logPrefix, long nUpdates,
+                        long ourLowThreshold, long ourHighThreshold) {
+      super(ourUpdates, ourLowThreshold);
+
+      this.logPrefix = logPrefix;
+      this.ourHighThreshold = ourHighThreshold;
+      this.ourHighest = ourUpdates.get(0);
+      this.nUpdates = nUpdates;
+    }
+
+    public MissedUpdatesRequest find(List<Long> otherVersions, Object updateFrom, Supplier<Boolean> canHandleVersionRanges) {
+      otherVersions.sort(absComparator);
+      if (debug) {
+        log.debug("{} sorted versions from {} = {}", logPrefix, otherVersions, updateFrom);
+      }
+
+      long otherHigh = percentile(otherVersions, .2f);
+      long otherLow = percentile(otherVersions, .8f);
+      long otherHighest = otherVersions.get(0);
+
+      if (ourHighThreshold < otherLow) {
+        // Small overlap between version windows and ours is older
+        // This means that we might miss updates if we attempted to use this method.
+        // Since there exists just one replica that is so much newer, we must
+        // fail the sync.
+        log.info("{} Our versions are too old. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
+            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
+        return MissedUpdatesRequest.UNABLE_TO_SYNC;
+      }
+
+      if (ourLowThreshold > otherHigh && ourHighest >= otherHighest) {
+        // Small overlap between windows and ours is newer.
+        // Using this list to sync would result in requesting/replaying results we don't need
+        // and possibly bringing deleted docs back to life.
+        log.info("{} Our versions are newer. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
+            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
 
+        // Because our versions are newer, IndexFingerprint with the remote would not match us.
+        // We return true on our side, but the remote peersync with us should fail.
+        return MissedUpdatesRequest.ALREADY_IN_SYNC;
+      }
+
+      boolean completeList = otherVersions.size() < nUpdates;
+
+      MissedUpdatesRequest updatesRequest;
+      if (canHandleVersionRanges.get()) {
+        updatesRequest = handleVersionsWithRanges(otherVersions, completeList);
+      } else {
+        updatesRequest = handleIndividualVersions(otherVersions, completeList);
+      }
+
+      if (updatesRequest.totalRequestedUpdates > nUpdates) {
+        log.info("{} PeerSync will fail because number of missed updates is more than:{}", logPrefix, nUpdates);
+        return MissedUpdatesRequest.UNABLE_TO_SYNC;
+      }
+
+      if (updatesRequest == MissedUpdatesRequest.EMPTY) {
+        log.info("{} No additional versions requested. ourHighThreshold={} otherLowThreshold={} ourHighest={} otherHighest={}",
+            logPrefix, ourHighThreshold, otherLow, ourHighest, otherHighest);
+      }
+
+      return updatesRequest;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
index b485727..ae58662 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Timer;
@@ -43,9 +44,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.DISTRIB;
+import static org.apache.solr.update.PeerSync.MissedUpdatesRequest;
 import static org.apache.solr.update.PeerSync.absComparator;
 import static org.apache.solr.update.PeerSync.percentile;
-import static org.apache.solr.update.PeerSync.MissedUpdatesRequest;
 
 public class PeerSyncWithLeader implements SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -62,7 +63,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
 
   private SolrCore core;
   private PeerSync.Updater updater;
-  private PeerSync.MissedUpdatesFinder missedUpdatesFinder;
+  private MissedUpdatesFinder missedUpdatesFinder;
   private Set<Long> bufferedUpdates;
 
   // metrics
@@ -203,7 +204,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
       log.info("Leader fingerprint {}", leaderFingerprint);
     }
 
-    missedUpdatesFinder = new PeerSync.MissedUpdatesFinder(ourUpdates, msg(), nUpdates, ourLowThreshold, ourHighThreshold);
+    missedUpdatesFinder = new MissedUpdatesFinder(ourUpdates, msg(), nUpdates, ourLowThreshold);
     MissedUpdatesRequest missedUpdates = buildMissedUpdatesRequest(leaderVersionsAndFingerprint);
     if (missedUpdates == MissedUpdatesRequest.ALREADY_IN_SYNC) return true;
     if (missedUpdates != MissedUpdatesRequest.UNABLE_TO_SYNC) {
@@ -369,4 +370,56 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
     }
     return false;
   }
+
+  /**
+   * Helper class for doing comparison ourUpdates and other replicas's updates to find the updates that we missed
+   */
+  public static class MissedUpdatesFinder extends PeerSync.MissedUpdatesFinderBase {
+    private long ourHighest;
+    private String logPrefix;
+    private long nUpdates;
+
+    MissedUpdatesFinder(List<Long> ourUpdates, String logPrefix, long nUpdates,
+                        long ourLowThreshold) {
+      super(ourUpdates, ourLowThreshold);
+
+      this.logPrefix = logPrefix;
+      this.ourHighest = ourUpdates.get(0);
+      this.nUpdates = nUpdates;
+    }
+
+    public MissedUpdatesRequest find(List<Long> leaderVersions, Object updateFrom, Supplier<Boolean> canHandleVersionRanges) {
+      leaderVersions.sort(absComparator);
+      log.debug("{} sorted versions from {} = {}", logPrefix, leaderVersions, updateFrom);
+
+      long leaderLowest = leaderVersions.get(leaderVersions.size() - 1);
+      if (Math.abs(ourHighest) < Math.abs(leaderLowest)) {
+        log.info("{} Our versions are too old comparing to leader, ourHighest={} otherLowest={}", logPrefix, ourHighest, leaderLowest);
+        return MissedUpdatesRequest.UNABLE_TO_SYNC;
+      }
+      // we don't have to check the case we ahead of the leader.
+      // (maybe we are the old leader and we contain some updates that no one have)
+      // In that case, we will fail on compute fingerprint with the current leader and start segments replication
+
+      boolean completeList = leaderVersions.size() < nUpdates;
+      MissedUpdatesRequest updatesRequest;
+      if (canHandleVersionRanges.get()) {
+        updatesRequest = handleVersionsWithRanges(leaderVersions, completeList);
+      } else {
+        updatesRequest = handleIndividualVersions(leaderVersions, completeList);
+      }
+
+      if (updatesRequest.totalRequestedUpdates > nUpdates) {
+        log.info("{} PeerSync will fail because number of missed updates is more than:{}", logPrefix, nUpdates);
+        return MissedUpdatesRequest.UNABLE_TO_SYNC;
+      }
+
+      if (updatesRequest == MissedUpdatesRequest.EMPTY) {
+        log.info("{} No additional versions requested", logPrefix);
+      }
+
+      return updatesRequest;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 7f77d57..b0ce886 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -252,7 +252,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     log.info("Looked up max version bucket seed "+maxVersionBefore+" for core "+coreName);
 
     // now up the stakes and do more docs
-    int numDocs = TEST_NIGHTLY ? 1000 : 100;
+    int numDocs = TEST_NIGHTLY ? 1000 : 105;
     boolean hasPartition = false;
     for (int d = 0; d < numDocs; d++) {
       // create / restore partition every 100 docs

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
new file mode 100644
index 0000000..ae5e769
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
@@ -0,0 +1,143 @@
+/*
+ * 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.cloud;
+
+import java.lang.invoke.MethodHandles;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.cloud.Replica;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestCloudRecovery2 extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final String COLLECTION = "collection1";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
+    System.setProperty("solr.ulog.numRecordsToKeep", "1000");
+
+    configureCluster(2)
+        .addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+
+    CollectionAdminRequest
+        .createCollection(COLLECTION, "config", 1,2)
+        .setMaxShardsPerNode(2)
+        .process(cluster.getSolrClient());
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
+        false, true, 30);
+  }
+
+  @Test
+  public void test() throws Exception {
+    JettySolrRunner node1 = cluster.getJettySolrRunner(0);
+    JettySolrRunner node2 = cluster.getJettySolrRunner(1);
+    try (HttpSolrClient client1 = getHttpSolrClient(node1.getBaseUrl().toString())) {
+
+      node2.stop();
+      waitForState("", COLLECTION, (liveNodes, collectionState) -> liveNodes.size() == 1);
+
+      UpdateRequest req = new UpdateRequest();
+      for (int i = 0; i < 100; i++) {
+        req = req.add("id", i+"", "num", i+"");
+      }
+      req.commit(client1, COLLECTION);
+
+      node2.start();
+      waitForState("", COLLECTION, clusterShape(1, 2));
+
+      try (HttpSolrClient client = getHttpSolrClient(node2.getBaseUrl().toString())) {
+        long numFound = client.query(COLLECTION, new SolrQuery("q","*:*", "distrib", "false")).getResults().getNumFound();
+        assertEquals(100, numFound);
+      }
+      long numFound = client1.query(COLLECTION, new SolrQuery("q","*:*", "distrib", "false")).getResults().getNumFound();
+      assertEquals(100, numFound);
+
+      new UpdateRequest().add("id", "1", "num", "10")
+          .commit(client1, COLLECTION);
+
+      try (HttpSolrClient client = getHttpSolrClient(node2.getBaseUrl().toString())) {
+        Object v = client.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+        assertEquals("10", v.toString());
+      }
+      Object v = client1.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("10", v.toString());
+
+      //
+      node2.stop();
+      waitForState("", COLLECTION, (liveNodes, collectionState) -> liveNodes.size() == 1);
+
+      new UpdateRequest().add("id", "1", "num", "20")
+          .commit(client1, COLLECTION);
+      v = client1.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("20", v.toString());
+
+      node2.start();
+      waitForState("", COLLECTION, clusterShape(1, 2));
+      try (HttpSolrClient client = getHttpSolrClient(node2.getBaseUrl().toString())) {
+        v = client.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+        assertEquals("20", v.toString());
+      }
+
+      node2.stop();
+      waitForState("", COLLECTION, (liveNodes, collectionState) -> liveNodes.size() == 1);
+
+      new UpdateRequest().add("id", "1", "num", "30")
+          .commit(client1, COLLECTION);
+      v = client1.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("30", v.toString());
+
+      node2.start();
+      waitForState("", COLLECTION, clusterShape(1, 2));
+
+      try (HttpSolrClient client = getHttpSolrClient(node2.getBaseUrl().toString())) {
+        v = client.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+        assertEquals("30", v.toString());
+      }
+      v = client1.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("30", v.toString());
+    }
+
+    node1.stop();
+    waitForState("", COLLECTION, (liveNodes, collectionState) -> {
+      Replica leader = collectionState.getLeader("shard1");
+      return leader != null && leader.getNodeName().equals(node2.getNodeName());
+    });
+
+    node1.start();
+    waitForState("", COLLECTION, clusterShape(1, 2));
+    try (HttpSolrClient client = getHttpSolrClient(node1.getBaseUrl().toString())) {
+      Object v = client.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("30", v.toString());
+    }
+    try (HttpSolrClient client = getHttpSolrClient(node2.getBaseUrl().toString())) {
+      Object v = client.query(COLLECTION, new SolrQuery("q","id:1", "distrib", "false")).getResults().get(0).get("num");
+      assertEquals("30", v.toString());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/solr/core/src/test/org/apache/solr/update/PeerSyncTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/PeerSyncTest.java b/solr/core/src/test/org/apache/solr/update/PeerSyncTest.java
index 848d1bc..001c727 100644
--- a/solr/core/src/test/org/apache/solr/update/PeerSyncTest.java
+++ b/solr/core/src/test/org/apache/solr/update/PeerSyncTest.java
@@ -42,10 +42,9 @@ import static org.junit.internal.matchers.StringContains.containsString;
 
 @SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
 public class PeerSyncTest extends BaseDistributedSearchTestCase {
-  private static int numVersions = 100;  // number of versions to use when syncing
-  private final String FROM_LEADER = DistribPhase.FROMLEADER.toString();
-
-  private ModifiableSolrParams seenLeader = 
+  protected static int numVersions = 100;  // number of versions to use when syncing
+  protected static final String FROM_LEADER = DistribPhase.FROMLEADER.toString();
+  protected static final ModifiableSolrParams seenLeader =
     params(DISTRIB_UPDATE_PARAM, FROM_LEADER);
   
   public PeerSyncTest() {
@@ -117,24 +116,7 @@ public class PeerSyncTest extends BaseDistributedSearchTestCase {
 
     validateDocs(docsAdded, client0, client1);
 
-    int toAdd = (int)(numVersions *.95);
-    for (int i=0; i<toAdd; i++) {
-      add(client0, seenLeader, sdoc("id",Integer.toString(i+11),"_version_",v+i+1));
-      docsAdded.add(i+11);
-    }
-
-    // sync should fail since there's not enough overlap to give us confidence
-    assertSync(client1, numVersions, false, shardsArr[0]);
-
-    // add some of the docs that were missing... just enough to give enough overlap
-    int toAdd2 = (int)(numVersions * .25);
-    for (int i=0; i<toAdd2; i++) {
-      add(client1, seenLeader, sdoc("id",Integer.toString(i+11),"_version_",v+i+1));
-    }
-
-    assertSync(client1, numVersions, true, shardsArr[0]);
-    validateDocs(docsAdded, client0, client1);
-
+    testOverlap(docsAdded, client0, client1, v);
     // test delete and deleteByQuery
     v=1000;
     SolrInputDocument doc = sdoc("id","1000","_version_",++v);
@@ -201,7 +183,7 @@ public class PeerSyncTest extends BaseDistributedSearchTestCase {
     v = 4000;
     add(client0, seenLeader, sdoc("id",Integer.toString((int)v),"_version_",v));
     docsAdded.add(4000);
-    toAdd = numVersions+10;
+    int toAdd = numVersions+10;
     for (int i=0; i<toAdd; i++) {
       add(client0, seenLeader, sdoc("id",Integer.toString((int)v+i+1),"_version_",v+i+1));
       add(client1, seenLeader, sdoc("id",Integer.toString((int)v+i+1),"_version_",v+i+1));
@@ -326,7 +308,27 @@ public class PeerSyncTest extends BaseDistributedSearchTestCase {
 
   }
 
-  private void validateDocs(Set<Integer> docsAdded, SolrClient client0, SolrClient client1) throws SolrServerException, IOException {
+  protected void testOverlap(Set<Integer> docsAdded, SolrClient client0, SolrClient client1, long v) throws IOException, SolrServerException {
+    int toAdd = (int)(numVersions *.95);
+    for (int i=0; i<toAdd; i++) {
+      add(client0, seenLeader, sdoc("id",Integer.toString(i+11),"_version_",v+i+1));
+      docsAdded.add(i+11);
+    }
+
+    // sync should fail since there's not enough overlap to give us confidence
+    assertSync(client1, numVersions, false, shardsArr[0]);
+
+    // add some of the docs that were missing... just enough to give enough overlap
+    int toAdd2 = (int)(numVersions * .25);
+    for (int i=0; i<toAdd2; i++) {
+      add(client1, seenLeader, sdoc("id",Integer.toString(i+11),"_version_",v+i+1));
+    }
+
+    assertSync(client1, numVersions, true, shardsArr[0]);
+    validateDocs(docsAdded, client0, client1);
+  }
+
+  protected void validateDocs(Set<Integer> docsAdded, SolrClient client0, SolrClient client1) throws SolrServerException, IOException {
     client0.commit();
     client1.commit();
     QueryResponse qacResponse;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f357c062/solr/core/src/test/org/apache/solr/update/PeerSyncWithLeaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/PeerSyncWithLeaderTest.java b/solr/core/src/test/org/apache/solr/update/PeerSyncWithLeaderTest.java
index 4ca343a..f1c7f69 100644
--- a/solr/core/src/test/org/apache/solr/update/PeerSyncWithLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/update/PeerSyncWithLeaderTest.java
@@ -19,6 +19,7 @@ package org.apache.solr.update;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Set;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
@@ -31,6 +32,23 @@ import org.apache.solr.common.util.StrUtils;
 public class PeerSyncWithLeaderTest extends PeerSyncTest {
 
   @Override
+  protected void testOverlap(Set<Integer> docsAdded, SolrClient client0, SolrClient client1, long v) throws IOException, SolrServerException {
+    for (int i=0; i<numVersions; i++) {
+      add(client0, seenLeader, sdoc("id",Integer.toString(i+11),"_version_",v+i+1));
+      docsAdded.add(i+11);
+    }
+
+    // sync should fail since we are too far with the leader
+    assertSync(client1, numVersions, false, shardsArr[0]);
+
+    // add a doc that was missing... just enough to give enough overlap
+    add(client1, seenLeader, sdoc("id",Integer.toString(11),"_version_",v+1));
+
+    assertSync(client1, numVersions, true, shardsArr[0]);
+    validateDocs(docsAdded, client0, client1);
+  }
+
+  @Override
   void assertSync(SolrClient client, int numVersions, boolean expectedResult, String... syncWith) throws IOException, SolrServerException {
     QueryRequest qr = new QueryRequest(params("qt","/get", "getVersions",Integer.toString(numVersions), "syncWithLeader", StrUtils.join(Arrays.asList(syncWith), ',')));
     NamedList rsp = client.request(qr);


[32/50] [abbrv] lucene-solr:jira/http2: SOLR-12913: Fix typo in User Guide

Posted by da...@apache.org.
SOLR-12913: Fix typo in User Guide


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

Branch: refs/heads/jira/http2
Commit: 780305cf44933db260ac7f0d4ed02e8cfc4691b9
Parents: dc6019c
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Nov 8 20:06:05 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Nov 8 20:06:05 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/vector-math.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/780305cf/solr/solr-ref-guide/src/vector-math.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/vector-math.adoc b/solr/solr-ref-guide/src/vector-math.adoc
index cb4ef1d..f820ac1 100644
--- a/solr/solr-ref-guide/src/vector-math.adoc
+++ b/solr/solr-ref-guide/src/vector-math.adoc
@@ -150,7 +150,7 @@ The example below shows the `lrtim` function trimming the first 2 elements of an
 
 [source,text]
 ----
-ltrim(array(0,1,2,3,4,5,6))
+ltrim(array(0,1,2,3,4,5,6), 2)
 ----
 
 When this expression is sent to the `/stream` handler it responds with:


[25/50] [abbrv] lucene-solr:jira/http2: SOLR-12975: Add ltrim and rtrim Stream Evaluators

Posted by da...@apache.org.
SOLR-12975: Add ltrim and rtrim Stream Evaluators


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

Branch: refs/heads/jira/http2
Commit: 2d0e565c62f09174bf1c123f0483d3b75d66aa41
Parents: 53482e5
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 20:45:13 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 20:47:07 2018 -0500

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/io/Lang.java   |  2 +
 .../solrj/io/eval/LeftShiftEvaluator.java       | 53 +++++++++++++++++++
 .../solrj/io/eval/RightShiftEvaluator.java      | 54 ++++++++++++++++++++
 .../apache/solr/client/solrj/io/TestLang.java   |  2 +-
 .../solrj/io/stream/MathExpressionTest.java     | 41 +++++++++++++++
 5 files changed, 151 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d0e565c/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
index 200f07b..13c91ec 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -274,6 +274,8 @@ public class Lang {
         .withFunctionName("pairSort", PairSortEvaluator.class)
         .withFunctionName("recip", RecipEvaluator.class)
         .withFunctionName("pivot", PivotEvaluator.class)
+        .withFunctionName("ltrim", LeftShiftEvaluator.class)
+        .withFunctionName("rtrim", RightShiftEvaluator.class)
 
         // Boolean Stream Evaluators
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d0e565c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LeftShiftEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LeftShiftEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LeftShiftEvaluator.java
new file mode 100644
index 0000000..2e86212
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LeftShiftEvaluator.java
@@ -0,0 +1,53 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LeftShiftEvaluator extends RecursiveObjectEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public LeftShiftEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(2 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 2 value but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object value1, Object value2) throws IOException {
+    if (value1 instanceof List && value2 instanceof Number) {
+      List<?> actual = (List<?>) value1;
+      int val = ((Number) value2).intValue();
+
+      List<Object> shifted = new ArrayList<>();
+      for (int i = val; i < actual.size(); i++) {
+        shifted.add(actual.get(i));
+      }
+      return shifted;
+    } else {
+      throw new IOException("The ltrim function takes two parameters: an array and number to trim it by");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d0e565c/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RightShiftEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RightShiftEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RightShiftEvaluator.java
new file mode 100644
index 0000000..1dce384
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RightShiftEvaluator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class RightShiftEvaluator extends RecursiveObjectEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public RightShiftEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(2 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 2 value but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object value1, Object value2) throws IOException {
+    if (value1 instanceof List && value2 instanceof Number) {
+      List<?> actual = (List<?>) value1;
+      int val = ((Number) value2).intValue();
+
+      List<Object> shifted = new ArrayList<>();
+      for (int i = 0; i < actual.size()-val; i++) {
+        shifted.add(actual.get(i));
+      }
+      return shifted;
+    } else {
+      throw new IOException("The rtrim function takes two parameters: an array and number to trim it by");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d0e565c/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
index 98a6d45..3adac54 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -73,7 +73,7 @@ public class TestLang extends LuceneTestCase {
       "outliers", "stream", "getCache", "putCache", "listCache", "removeCache", "zscores", "latlonVectors",
       "convexHull", "getVertices", "getBaryCenter", "getArea", "getBoundarySize","oscillate",
       "getAmplitude", "getPhase", "getAngularFrequency", "enclosingDisk", "getCenter", "getRadius",
-      "getSupportPoints", "pairSort", "log10", "plist", "recip", "pivot"};
+      "getSupportPoints", "pairSort", "log10", "plist", "recip", "pivot", "ltrim", "rtrim"};
 
   @Test
   public void testLang() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d0e565c/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index 9c93ced..0ccd691 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -1226,6 +1226,47 @@ public class MathExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testLtrim() throws Exception {
+    String cexpr = "ltrim(array(1,2,3,4,5,6), 2)";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    List<Number> out = (List<Number>)tuples.get(0).get("return-value");
+    assertEquals(out.size(), 4);
+    assertEquals(out.get(0).intValue(), 3);
+    assertEquals(out.get(1).intValue(), 4);
+    assertEquals(out.get(2).intValue(), 5);
+    assertEquals(out.get(3).intValue(), 6);
+  }
+
+  @Test
+  public void testRtrim() throws Exception {
+    String cexpr = "rtrim(array(1,2,3,4,5,6), 2)";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    List<Number> out = (List<Number>)tuples.get(0).get("return-value");
+    assertEquals(out.size(), 4);
+    assertEquals(out.get(0).intValue(), 1);
+    assertEquals(out.get(1).intValue(), 2);
+    assertEquals(out.get(2).intValue(), 3);
+    assertEquals(out.get(3).intValue(), 4);
+  }
+
+
+  @Test
   public void testZeros() throws Exception {
     String cexpr = "zeros(6)";
     ModifiableSolrParams paramsLoc = new ModifiableSolrParams();


[41/50] [abbrv] lucene-solr:jira/http2: fix merge relict

Posted by da...@apache.org.
fix merge relict


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

Branch: refs/heads/jira/http2
Commit: 42ee96642c01fb6280c5af161761ff28be6ee914
Parents: efd3f17
Author: Uwe Schindler <us...@apache.org>
Authored: Fri Nov 9 23:56:56 2018 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Fri Nov 9 23:58:01 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42ee9664/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 54fda0b..8ceb4ac 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -193,7 +193,6 @@ Optimizations
   is by relevancy and the total hit count is not required. (Jim Ferenczi)
 
 ======================= Lucene 7.7.0 =======================
-(No Changes)
 
 Build:
 


[30/50] [abbrv] lucene-solr:jira/http2: Ref Guide: fix pages missing header license info

Posted by da...@apache.org.
Ref Guide: fix pages missing header license info


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

Branch: refs/heads/jira/http2
Commit: 73005d482d814b5b56551ddaff734b5a79054449
Parents: 01397c1
Author: Cassandra Targett <ct...@apache.org>
Authored: Thu Nov 8 15:01:55 2018 -0600
Committer: Cassandra Targett <ct...@apache.org>
Committed: Thu Nov 8 15:02:33 2018 -0600

----------------------------------------------------------------------
 solr/solr-ref-guide/src/json-facet-api.adoc      | 16 ++++++++++++++++
 solr/solr-ref-guide/src/solr-tutorial.adoc       | 16 ++++++++++++++++
 solr/solr-ref-guide/src/the-tagger-handler.adoc  | 16 ++++++++++++++++
 solr/solr-ref-guide/src/time-routed-aliases.adoc | 16 ++++++++++++++++
 4 files changed, 64 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/73005d48/solr/solr-ref-guide/src/json-facet-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/json-facet-api.adoc b/solr/solr-ref-guide/src/json-facet-api.adoc
index b0f6e37..1e52034 100644
--- a/solr/solr-ref-guide/src/json-facet-api.adoc
+++ b/solr/solr-ref-guide/src/json-facet-api.adoc
@@ -1,5 +1,21 @@
 = JSON Facet API
 :page-tocclass: right
+// 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.
 
 == Facet & Analytics Module
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/73005d48/solr/solr-ref-guide/src/solr-tutorial.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solr-tutorial.adoc b/solr/solr-ref-guide/src/solr-tutorial.adoc
index 577fc5c..fb0acbc 100644
--- a/solr/solr-ref-guide/src/solr-tutorial.adoc
+++ b/solr/solr-ref-guide/src/solr-tutorial.adoc
@@ -1,6 +1,22 @@
 = Solr Tutorial
 :page-tocclass: right
 :experimental:
+// 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.
 
 This tutorial covers getting Solr up and running, ingesting a variety of data sources into Solr collections,
 and getting a feel for the Solr administrative and search interfaces.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/73005d48/solr/solr-ref-guide/src/the-tagger-handler.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/the-tagger-handler.adoc b/solr/solr-ref-guide/src/the-tagger-handler.adoc
index 4536eb0..6a7b303 100644
--- a/solr/solr-ref-guide/src/the-tagger-handler.adoc
+++ b/solr/solr-ref-guide/src/the-tagger-handler.adoc
@@ -1,4 +1,20 @@
 = The Tagger Handler
+// 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.
 
 The "Tagger" Request Handler, AKA the "SolrTextTagger" is a "text tagger".
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/73005d48/solr/solr-ref-guide/src/time-routed-aliases.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/time-routed-aliases.adoc b/solr/solr-ref-guide/src/time-routed-aliases.adoc
index a97d051..3ef0e19 100644
--- a/solr/solr-ref-guide/src/time-routed-aliases.adoc
+++ b/solr/solr-ref-guide/src/time-routed-aliases.adoc
@@ -1,4 +1,20 @@
 = Time Routed Aliases
+// 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.
 
 Time Routed Aliases (TRAs) is a SolrCloud feature that manages an alias and a time sequential series of collections.
 


[14/50] [abbrv] lucene-solr:jira/http2: SOLR-12795: Require bucketSorts

Posted by da...@apache.org.
SOLR-12795: Require bucketSorts


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

Branch: refs/heads/jira/http2
Commit: 259925d12fd9a693a8d8314e4060055e7734b1f0
Parents: 80efc23
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 08:58:57 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 08:58:57 2018 -0500

----------------------------------------------------------------------
 .../client/solrj/io/stream/FacetStream.java     |  2 +-
 .../stream/StreamExpressionToExpessionTest.java | 28 --------------------
 2 files changed, 1 insertion(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/259925d1/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
index 6923a3b..126df81 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
@@ -163,7 +163,7 @@ public class FacetStream extends TupleStream implements Expressible  {
     String bucketSortString = null;
 
     if(bucketSortExpression == null) {
-      bucketSortString = "count(*) desc";
+      throw new IOException("The bucketSorts parameter is required for the facet function.");
     } else {
       bucketSortString = ((StreamExpressionValue)bucketSortExpression.getParameter()).getValue();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/259925d1/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
index 0b1c764..3dd4261 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
@@ -316,34 +316,6 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
         +   "collection1, "
         +   "q=\"*:*\", "
         +   "buckets=\"a_s\", "
-        +   "bucketSizeLimit=100, "
-        +   "sum(a_i), sum(a_f), "
-        +   "min(a_i), min(a_f), "
-        +   "max(a_i), max(a_f), "
-        +   "avg(a_i), avg(a_f), "
-        +   "count(*)"
-        + ")"), factory)){
-      expressionString = stream.toExpression(factory).toString();
-      assertTrue(expressionString.contains("facet(collection1"));
-      assertTrue(expressionString.contains("q=\"*:*\""));
-      assertTrue(expressionString.contains("buckets=a_s"));
-      assertTrue(expressionString.contains("bucketSorts=\"count(*) desc\""));
-      assertTrue(expressionString.contains("bucketSizeLimit=100"));
-      assertTrue(expressionString.contains("sum(a_i)"));
-      assertTrue(expressionString.contains("sum(a_f)"));
-      assertTrue(expressionString.contains("min(a_i)"));
-      assertTrue(expressionString.contains("min(a_f)"));
-      assertTrue(expressionString.contains("max(a_i)"));
-      assertTrue(expressionString.contains("max(a_f)"));
-      assertTrue(expressionString.contains("avg(a_i,false)"));
-      assertTrue(expressionString.contains("avg(a_f,false)"));
-      assertTrue(expressionString.contains("count(*)"));
-    }
-
-    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
-        +   "collection1, "
-        +   "q=\"*:*\", "
-        +   "buckets=\"a_s\", "
         +   "bucketSorts=\"sum(a_i) asc\", "
         +   "rows=10, method=dvhash, "
         +   "sum(a_i), sum(a_f), "


[47/50] [abbrv] lucene-solr:jira/http2: SOLR-12977: Autoscaling tries to fetch metrics from dead nodes

Posted by da...@apache.org.
SOLR-12977: Autoscaling tries to fetch metrics from dead nodes


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

Branch: refs/heads/jira/http2
Commit: e6e6ad2c833591028ca9f504571cf26e9585fdda
Parents: 605c3f6
Author: Noble Paul <no...@apache.org>
Authored: Mon Nov 12 18:43:09 2018 +1100
Committer: Noble Paul <no...@apache.org>
Committed: Mon Nov 12 18:44:26 2018 +1100

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6e6ad2c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2e8e97a..6fca62d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -212,7 +212,6 @@ New Features
   to set this to false (using docValues as needed) to reduce the risk of large fluxuations in heap
   size due to unexpected attempts to sort/facet/function on non-docValue fields. (hossman)
 
-* SOLR-12977: Autoscaling tries to fetch metrics from dead nodes (noble)
 
 Other Changes
 ----------------------
@@ -289,6 +288,8 @@ used with DVHASH method in json.facet. (Tim Underwood via Mikhail Khludnev)
 * SOLR-12243: Edismax missing phrase queries when phrases contain multiterm synonyms
   (Elizabeth Haubert, Alessandro Benedetti, Uwe Schindler, Steve Rowe)
 
+* SOLR-12977: Autoscaling tries to fetch metrics from dead nodes (noble)
+
 Improvements
 ----------------------
 


[07/50] [abbrv] lucene-solr:jira/http2: LUCENE-8557: LeafReader.getFieldInfos should always return the same instance MemoryIndex: compute/cache up-front Solr Collapse/Expand with top_fc: compute/cache up-front Json Facets numerics / hash DV: use the cach

Posted by da...@apache.org.
LUCENE-8557: LeafReader.getFieldInfos should always return the same instance
MemoryIndex: compute/cache up-front
Solr Collapse/Expand with top_fc: compute/cache up-front
Json Facets numerics / hash DV: use the cached fieldInfos on SolrIndexSearcher
SolrIndexSearcher: move the cached FieldInfos to SlowCompositeReaderWrapper


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

Branch: refs/heads/jira/http2
Commit: d0cd4245bdb8363e9adf3812817b9989ce4f506c
Parents: b230543
Author: David Smiley <ds...@apache.org>
Authored: Tue Nov 6 14:45:32 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Tue Nov 6 14:45:32 2018 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +
 .../org/apache/lucene/index/FieldInfos.java     |  4 +
 .../org/apache/lucene/index/LeafReader.java     |  4 +
 .../apache/lucene/index/TestPendingDeletes.java |  2 +-
 .../lucene/index/TestTermVectorsReader.java     |  2 +-
 .../apache/lucene/index/memory/MemoryIndex.java | 15 ++--
 .../org/apache/lucene/search/QueryUtils.java    |  3 +-
 .../java/org/apache/lucene/util/TestUtil.java   |  5 ++
 solr/CHANGES.txt                                |  4 +
 .../solr/handler/component/ExpandComponent.java | 77 +----------------
 .../solr/index/SlowCompositeReaderWrapper.java  |  9 +-
 .../solr/search/CollapsingQParserPlugin.java    | 89 +++++++++++---------
 .../apache/solr/search/SolrIndexSearcher.java   |  7 +-
 .../facet/FacetFieldProcessorByHashDV.java      |  2 +-
 .../test/org/apache/solr/search/TestDocSet.java |  3 +-
 15 files changed, 95 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 59d81d0..313d7bf 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -252,6 +252,10 @@ Improvements:
   first 4 are index dimensions defining the bounding box of the Triangle and the
   remaining 3 data dimensions define the vertices of the triangle. (Nick Knize)
 
+* LUCENE-8557: LeafReader.getFieldInfos is now documented and tested that it ought to return
+  the same cached instance.  MemoryIndex's impl now pre-creates the FieldInfos instead of
+  re-calculating a new instance each time.  (Tim Underwood, David Smiley)
+
 Other:
 
 * LUCENE-8523: Correct typo in JapaneseNumberFilterFactory javadocs (Ankush Jhalani

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
index 580c50c..5cd9639 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
@@ -38,6 +38,10 @@ import org.apache.lucene.util.ArrayUtil;
  *  @lucene.experimental
  */
 public class FieldInfos implements Iterable<FieldInfo> {
+
+  /** An instance without any fields. */
+  public final static FieldInfos EMPTY = new FieldInfos(new FieldInfo[0]);
+
   private final boolean hasFreq;
   private final boolean hasProx;
   private final boolean hasPayloads;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
index faea32d..1d09742 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
@@ -206,6 +206,10 @@ public abstract class LeafReader extends IndexReader {
   /**
    * Get the {@link FieldInfos} describing all fields in
    * this reader.
+   *
+   * Note: Implementations should cache the FieldInfos
+   * instance returned by this method such that subsequent
+   * calls to this method return the same instance.
    * @lucene.experimental
    */
   public abstract FieldInfos getFieldInfos();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
index d453034..e82ec10 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
@@ -132,7 +132,7 @@ public class TestPendingDeletes extends LuceneTestCase {
     SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
         Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
     SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
-    FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]);
+    FieldInfos fieldInfos = FieldInfos.EMPTY;
     si.getCodec().fieldInfosFormat().write(dir, si, "", fieldInfos, IOContext.DEFAULT);
     PendingDeletes deletes = newPendingDeletes(commitInfo);
     for (int i = 0; i < 3; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
index 76947dd..e2a3f20 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
@@ -47,7 +47,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
   private int[][] positions = new int[testTerms.length][];
   private Directory dir;
   private SegmentCommitInfo seg;
-  private FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]);
+  private FieldInfos fieldInfos = FieldInfos.EMPTY;
   private static int TERM_FREQ = 3;
 
   private static class TestToken implements Comparable<TestToken> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 0078c3f..fde9438 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -1142,12 +1142,20 @@ public class MemoryIndex {
   private final class MemoryIndexReader extends LeafReader {
 
     private final MemoryFields memoryFields = new MemoryFields(fields);
+    private final FieldInfos fieldInfos;
 
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
+
+      FieldInfo[] fieldInfosArr = new FieldInfo[fields.size()];
+
+      int i = 0;
       for (Info info : fields.values()) {
         info.prepareDocValuesAndPointValues();
+        fieldInfosArr[i++] = info.fieldInfo;
       }
+
+      fieldInfos = new FieldInfos(fieldInfosArr);
     }
 
     private Info getInfoForExpectedDocValuesType(String fieldName, DocValuesType expectedType) {
@@ -1171,12 +1179,7 @@ public class MemoryIndex {
     
     @Override
     public FieldInfos getFieldInfos() {
-      FieldInfo[] fieldInfos = new FieldInfo[fields.size()];
-      int i = 0;
-      for (Info info : fields.values()) {
-        fieldInfos[i++] = info.fieldInfo;
-      }
-      return new FieldInfos(fieldInfos);
+      return fieldInfos;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
index d7f63cb..840e78e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
@@ -22,7 +22,6 @@ import java.util.Random;
 
 import junit.framework.Assert;
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
@@ -207,7 +206,7 @@ public class QueryUtils {
 
       @Override
       public FieldInfos getFieldInfos() {
-        return new FieldInfos(new FieldInfo[0]);
+        return FieldInfos.EMPTY;
       }
 
       final Bits liveDocs = new Bits.MatchNoBits(maxDoc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index fee918f..64287a0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -335,6 +335,11 @@ public final class TestUtil {
       }
       assert Accountables.toString(sr) != null;
     }
+
+    // FieldInfos should be cached at the reader and always return the same instance
+    if (reader.getFieldInfos() != reader.getFieldInfos()) {
+      throw new RuntimeException("getFieldInfos() returned different instances for class: "+reader.getClass());
+    }
   }
   
   // used by TestUtil.checkReader to check some things really unrelated to the index,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 17e918b..2850b30 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -30,6 +30,7 @@ Jetty 9.4.11.v20180605
 
 Upgrade Notes
 ----------------------
+
 * SOLR-12767: The min_rf parameter is no longer needed, Solr will always return the achieved replication factor (rf)
   in the response header.
 
@@ -187,6 +188,9 @@ Improvements
 * SOLR-12699: Make contrib/ltr LTRScoringModel immutable and cache its hashCode.
   (Stanislav Livotov, Edward Ribeiro, Christine Poerschke)
 
+* LUCENE-8557: Some internal LeafReader.getFieldInfos implementations were being re-computed on-demand instead of
+  once up front leading to some slowdowns in places like JSON Facets and field collapsing. (Tim Underwood, David Smiley)
+
 ==================  7.5.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/d0cd4245/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 2cbe703..bda643b 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -19,7 +19,6 @@ package org.apache.solr.handler.component;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -33,10 +32,6 @@ import com.carrotsearch.hppc.cursors.LongCursor;
 import com.carrotsearch.hppc.cursors.LongObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -84,7 +79,6 @@ import org.apache.solr.search.DocSlice;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SortSpecParsing;
-import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
@@ -214,9 +208,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     if(fieldType instanceof StrField) {
       //Get The Top Level SortedDocValues
       if(CollapsingQParserPlugin.HINT_TOP_FC.equals(hint)) {
-        @SuppressWarnings("resource") LeafReader uninvertingReader = UninvertingReader.wrap(
-            new ReaderWrapper(searcher.getSlowAtomicReader(), field),
-            Collections.singletonMap(field, UninvertingReader.Type.SORTED)::get);
+        @SuppressWarnings("resource")
+        LeafReader uninvertingReader = CollapsingQParserPlugin.getTopFieldCacheReader(searcher, field);
         values = uninvertingReader.getSortedDocValues(field);
       } else {
         values = DocValues.getSorted(reader, field);
@@ -384,9 +377,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     if(values != null) {
       //Get The Top Level SortedDocValues again so we can re-iterate:
       if(CollapsingQParserPlugin.HINT_TOP_FC.equals(hint)) {
-        @SuppressWarnings("resource") LeafReader uninvertingReader = UninvertingReader.wrap(
-            new ReaderWrapper(searcher.getSlowAtomicReader(), field),
-            Collections.singletonMap(field, UninvertingReader.Type.SORTED)::get);
+        @SuppressWarnings("resource")
+        LeafReader uninvertingReader = CollapsingQParserPlugin.getTopFieldCacheReader(searcher, field);
         values = uninvertingReader.getSortedDocValues(field);
       } else {
         values = DocValues.getSorted(reader, field);
@@ -764,65 +756,4 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     return Category.QUERY;
   }
 
-  // this reader alters the content of the given reader so it should not
-  // delegate the caching stuff
-  private static class ReaderWrapper extends FilterLeafReader {
-
-    private String field;
-
-    public ReaderWrapper(LeafReader leafReader, String field) {
-      super(leafReader);
-      this.field = field;
-    }
-
-    public SortedDocValues getSortedDocValues(String field) {
-      return null;
-    }
-
-    public FieldInfos getFieldInfos() {
-      Iterator<FieldInfo> it = in.getFieldInfos().iterator();
-      List<FieldInfo> newInfos = new ArrayList<>();
-      while(it.hasNext()) {
-        FieldInfo fieldInfo = it.next();
-
-        if(fieldInfo.name.equals(field)) {
-          FieldInfo f = new FieldInfo(fieldInfo.name,
-              fieldInfo.number,
-              fieldInfo.hasVectors(),
-              fieldInfo.hasNorms(),
-              fieldInfo.hasPayloads(),
-              fieldInfo.getIndexOptions(),
-              DocValuesType.NONE,
-              fieldInfo.getDocValuesGen(),
-              fieldInfo.attributes(),
-              fieldInfo.getPointDataDimensionCount(),
-              fieldInfo.getPointIndexDimensionCount(),
-              fieldInfo.getPointNumBytes(),
-              fieldInfo.isSoftDeletesField());
-          newInfos.add(f);
-
-        } else {
-          newInfos.add(fieldInfo);
-        }
-      }
-      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
-      return infos;
-    }
-
-    // NOTE: delegating the caches is wrong here as we are altering the content
-    // of the reader, this should ONLY be used under an uninvertingreader which
-    // will restore doc values back using uninversion, otherwise all sorts of
-    // crazy things could happen.
-
-    @Override
-    public CacheHelper getCoreCacheHelper() {
-      return in.getCoreCacheHelper();
-    }
-
-    @Override
-    public CacheHelper getReaderCacheHelper() {
-      return in.getReaderCacheHelper();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
index 441abae..11299e4 100644
--- a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
+++ b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
@@ -48,6 +48,11 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
   private final CompositeReader in;
   private final LeafMetaData metaData;
 
+  // Cached copy of FieldInfos to prevent it from being re-created on each
+  // getFieldInfos call.  Most (if not all) other LeafReader implementations
+  // also have a cached FieldInfos instance so this is consistent. SOLR-12878
+  private final FieldInfos fieldInfos;
+
   final Map<String,Terms> cachedTerms = new ConcurrentHashMap<>();
 
   // TODO: consider ConcurrentHashMap ?
@@ -86,6 +91,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
       }
       metaData = new LeafMetaData(reader.leaves().get(0).reader().getMetaData().getCreatedVersionMajor(), minVersion, null);
     }
+    fieldInfos = FieldInfos.getMergedFieldInfos(in);
   }
 
   @Override
@@ -273,8 +279,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
 
   @Override
   public FieldInfos getFieldInfos() {
-    ensureOpen();
-    return FieldInfos.getMergedFieldInfos(in); // TODO cache?
+    return fieldInfos;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index 57d3d43..e2c86a6 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -387,13 +387,55 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
   }
 
+  /**
+   * This forces the use of the top level field cache for String fields.
+   * This is VERY fast at query time but slower to warm and causes insanity.
+   */
+  public static LeafReader getTopFieldCacheReader(SolrIndexSearcher searcher, String collapseField) {
+    return UninvertingReader.wrap(
+        new ReaderWrapper(searcher.getSlowAtomicReader(), collapseField),
+        Collections.singletonMap(collapseField, UninvertingReader.Type.SORTED)::get);
+  }
+
   private static class ReaderWrapper extends FilterLeafReader {
 
-    private String field;
+    private final FieldInfos fieldInfos;
 
-    public ReaderWrapper(LeafReader leafReader, String field) {
+    ReaderWrapper(LeafReader leafReader, String field) {
       super(leafReader);
-      this.field = field;
+
+      // TODO can we just do "field" and not bother with the other fields?
+      List<FieldInfo> newInfos = new ArrayList<>(in.getFieldInfos().size());
+      for (FieldInfo fieldInfo : in.getFieldInfos()) {
+        if (fieldInfo.name.equals(field)) {
+          FieldInfo f = new FieldInfo(fieldInfo.name,
+              fieldInfo.number,
+              fieldInfo.hasVectors(),
+              fieldInfo.hasNorms(),
+              fieldInfo.hasPayloads(),
+              fieldInfo.getIndexOptions(),
+              DocValuesType.NONE,
+              fieldInfo.getDocValuesGen(),
+              fieldInfo.attributes(),
+              fieldInfo.getPointDataDimensionCount(),
+              fieldInfo.getPointIndexDimensionCount(),
+              fieldInfo.getPointNumBytes(),
+              fieldInfo.isSoftDeletesField());
+          newInfos.add(f);
+        } else {
+          newInfos.add(fieldInfo);
+        }
+      }
+      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
+      this.fieldInfos = infos;
+    }
+
+    public FieldInfos getFieldInfos() {
+      return fieldInfos;
+    }
+
+    public SortedDocValues getSortedDocValues(String field) {
+      return null;
     }
 
     // NOTE: delegating the caches is wrong here as we are altering the content
@@ -410,37 +452,6 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     public CacheHelper getReaderCacheHelper() {
       return in.getReaderCacheHelper();
     }
-
-    public SortedDocValues getSortedDocValues(String field) {
-      return null;
-    }
-
-    public FieldInfos getFieldInfos() {
-      Iterator<FieldInfo> it = in.getFieldInfos().iterator();
-      List<FieldInfo> newInfos = new ArrayList();
-      while(it.hasNext()) {
-        FieldInfo fieldInfo = it.next();
-
-        if(fieldInfo.name.equals(field)) {
-          FieldInfo f = new FieldInfo(fieldInfo.name,
-                                      fieldInfo.number,
-                                      fieldInfo.hasVectors(),
-                                      fieldInfo.hasNorms(),
-                                      fieldInfo.hasPayloads(),
-                                      fieldInfo.getIndexOptions(),
-                                      DocValuesType.NONE,
-                                      fieldInfo.getDocValuesGen(),
-                                      fieldInfo.attributes(),
-                                      0, 0, 0, fieldInfo.isSoftDeletesField());
-          newInfos.add(f);
-
-        } else {
-          newInfos.add(fieldInfo);
-        }
-      }
-      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
-      return infos;
-    }
   }
 
 
@@ -1255,14 +1266,8 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
       if(collapseFieldType instanceof StrField) {
         if(HINT_TOP_FC.equals(hint)) {
-
-          /*
-          * This hint forces the use of the top level field cache for String fields.
-          * This is VERY fast at query time but slower to warm and causes insanity.
-          */
-          @SuppressWarnings("resource") final LeafReader uninvertingReader = UninvertingReader.wrap(
-                  new ReaderWrapper(searcher.getSlowAtomicReader(), collapseField),
-                  Collections.singletonMap(collapseField, UninvertingReader.Type.SORTED)::get);
+          @SuppressWarnings("resource")
+          final LeafReader uninvertingReader = getTopFieldCacheReader(searcher, collapseField);
 
           docValuesProducer = new EmptyDocValuesProducer() {
               @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 9b36c97..a659d66 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -126,8 +126,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   // list of all caches associated with this searcher.
   private final SolrCache[] cacheList;
 
-  private final FieldInfos fieldInfos;
-
   private DirectoryFactory directoryFactory;
 
   private final LeafReader leafReader;
@@ -263,7 +261,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     this.queryResultMaxDocsCached = solrConfig.queryResultMaxDocsCached;
     this.useFilterForSortedQuery = solrConfig.useFilterForSortedQuery;
 
-    this.fieldInfos = leafReader.getFieldInfos();
     this.docFetcher = new SolrDocumentFetcher(this, solrConfig, enableCache);
 
     this.cachingEnabled = enableCache;
@@ -319,7 +316,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   }
 
   public FieldInfos getFieldInfos() {
-    return fieldInfos;
+    return leafReader.getFieldInfos();
   }
 
   /*
@@ -494,7 +491,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    * Returns a collection of all field names the index reader knows about.
    */
   public Iterable<String> getFieldNames() {
-    return Iterables.transform(fieldInfos, fieldInfo -> fieldInfo.name);
+    return Iterables.transform(getFieldInfos(), fieldInfo -> fieldInfo.name);
   }
 
   public SolrCache<Query,DocSet> getFilterCache() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
index c246c21..5070a97 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
@@ -199,7 +199,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           getClass()+" doesn't support prefix"); // yet, but it could
     }
-    FieldInfo fieldInfo = fcontext.searcher.getSlowAtomicReader().getFieldInfos().fieldInfo(sf.getName());
+    FieldInfo fieldInfo = fcontext.searcher.getFieldInfos().fieldInfo(sf.getName());
     if (fieldInfo != null &&
         fieldInfo.getDocValuesType() != DocValuesType.NUMERIC &&
         fieldInfo.getDocValuesType() != DocValuesType.SORTED &&

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0cd4245/solr/core/src/test/org/apache/solr/search/TestDocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestDocSet.java b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
index b592317..999b080 100644
--- a/solr/core/src/test/org/apache/solr/search/TestDocSet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
@@ -391,7 +390,7 @@ public class TestDocSet extends LuceneTestCase {
 
       @Override
       public FieldInfos getFieldInfos() {
-        return new FieldInfos(new FieldInfo[0]);
+        return FieldInfos.EMPTY;
       }
 
       @Override


[31/50] [abbrv] lucene-solr:jira/http2: fix LatLonShapeBoundingBoxQuery to not round min Y above max Y value

Posted by da...@apache.org.
fix LatLonShapeBoundingBoxQuery to not round min Y above max Y value


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

Branch: refs/heads/jira/http2
Commit: dc6019c54e93d38fe89302e9d42fe3e596440e08
Parents: 73005d4
Author: Nicholas Knize <nk...@gmail.com>
Authored: Thu Nov 8 16:27:37 2018 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Thu Nov 8 16:34:54 2018 -0600

----------------------------------------------------------------------
 .../document/LatLonShapeBoundingBoxQuery.java   |  9 ++++++--
 .../document/BaseLatLonShapeTestCase.java       |  8 ++++++-
 .../document/TestLatLonPointShapeQueries.java   |  9 ++------
 .../apache/lucene/document/TestLatLonShape.java | 22 ++++++++++++++++++++
 4 files changed, 38 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc6019c5/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
index b4f7f4b..ebbdeed 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
@@ -57,8 +57,13 @@ final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
     this.bbox = new byte[4 * LatLonShape.BYTES];
     int minXenc = encodeLongitudeCeil(minLon);
     int maxXenc = encodeLongitude(maxLon);
-    this.minY = encodeLatitudeCeil(minLat);
-    this.maxY = encodeLatitude(maxLat);
+    int minYenc = encodeLatitudeCeil(minLat);
+    int maxYenc = encodeLatitude(maxLat);
+    if (minYenc > maxYenc) {
+      minYenc = maxYenc;
+    }
+    this.minY = minYenc;
+    this.maxY = maxYenc;
 
     if (minLon > maxLon == true) {
       // crossing dateline is split into east/west boxes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc6019c5/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
index 942979b..e8ea640 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
@@ -321,6 +321,8 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
         boolean expected;
         double qMinLon = quantizeLonCeil(rect.minLon);
         double qMaxLon = quantizeLon(rect.maxLon);
+        double qMinLat = quantizeLatCeil(rect.minLat);
+        double qMaxLat = quantizeLat(rect.maxLat);
         if (liveDocs != null && liveDocs.get(docID) == false) {
           // document is deleted
           expected = false;
@@ -333,7 +335,11 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
             // then do not use encodeCeil
             qMinLon = quantizeLon(rect.minLon);
           }
-          expected = getValidator(queryRelation).testBBoxQuery(quantizeLatCeil(rect.minLat), quantizeLat(rect.maxLat), qMinLon, qMaxLon, shapes[id]);
+
+          if (qMinLat > qMaxLat) {
+            qMinLat = quantizeLat(rect.maxLat);
+          }
+          expected = getValidator(queryRelation).testBBoxQuery(qMinLat, qMaxLat, qMinLon, qMaxLon, shapes[id]);
         }
 
         if (hits.get(docID) != expected) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc6019c5/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
index 96b026c..6020617 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
@@ -25,11 +25,6 @@ import org.apache.lucene.geo.Line2D;
 import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
-
 /** random bounding box and polygon query tests for random generated {@code latitude, longitude} points */
 public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
 
@@ -105,8 +100,8 @@ public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     private boolean testPoint(EdgeTree tree, Point p) {
-      double lat = decodeLatitude(encodeLatitude(p.lat));
-      double lon = decodeLongitude(encodeLongitude(p.lon));
+      double lat = quantizeLat(p.lat);
+      double lon = quantizeLon(p.lon);
       // for consistency w/ the query we test the point as a triangle
       Relation r = tree.relateTriangle(lon, lat, lon, lat, lon, lat);
       if (queryRelation == QueryRelation.WITHIN) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc6019c5/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
index 9a125ba..d3ab8a6 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
@@ -223,4 +223,26 @@ public class TestLatLonShape extends LuceneTestCase {
 
     IOUtils.close(reader, dir);
   }
+
+  public void testPointIndexAndQuery() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document document = new Document();
+    BaseLatLonShapeTestCase.Point p = (BaseLatLonShapeTestCase.Point) BaseLatLonShapeTestCase.ShapeType.POINT.nextShape();
+    Field[] fields = LatLonShape.createIndexableFields(FIELDNAME, p.lat, p.lon);
+    for (Field f : fields) {
+      document.add(f);
+    }
+    writer.addDocument(document);
+
+    //// search
+    IndexReader r = writer.getReader();
+    writer.close();
+    IndexSearcher s = newSearcher(r);
+
+    // search by same point
+    Query q = LatLonShape.newBoxQuery(FIELDNAME, QueryRelation.INTERSECTS, p.lat, p.lat, p.lon, p.lon);
+    assertEquals(1, s.count(q));
+    IOUtils.close(r, dir);
+  }
 }


[08/50] [abbrv] lucene-solr:jira/http2: SOLR-12023: correcting wrong git merge

Posted by da...@apache.org.
SOLR-12023: correcting wrong git merge


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

Branch: refs/heads/jira/http2
Commit: 6f6a880ec2126690bb363b2a591bed36c406caee
Parents: d0cd424
Author: Noble Paul <no...@apache.org>
Authored: Wed Nov 7 12:56:00 2018 +1100
Committer: Noble Paul <no...@apache.org>
Committed: Wed Nov 7 12:56:00 2018 +1100

----------------------------------------------------------------------
 solr/CHANGES.txt | 110 ++++++++++++++++++++++++++++++++++++++++++++++----
 1 file changed, 103 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6f6a880e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2850b30..7b60eb0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -16,6 +16,88 @@ In this release, there is an example Solr server including a bundled
 servlet container in the directory named "example".
 See the Solr tutorial at https://lucene.apache.org/solr/guide/solr-tutorial.html
 
+==================  8.0.0 ==================
+
+Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+
+Versions of Major Components
+---------------------
+Apache Tika 1.19.1
+Carrot2 3.16.0
+Velocity 1.7 and Velocity Tools 2.0
+Apache ZooKeeper 3.4.11
+Jetty 9.4.11.v20180605
+
+Upgrade Notes
+----------------------
+
+* LUCENE-7996: The 'func' query parser now returns scores that are equal to 0
+  when a negative value is produced. This change is due to the fact that
+  Lucene now requires scores to be positive. (Adrien Grand)
+
+* SOLR-11882: SolrMetric registries retained references to SolrCores when closed. A
+  change of SolrMetricMAnager.registerGauge and SolrMetricProducer.initializeMetrics
+  method signatures was required to fix it. Third party components that use this API
+  need to be updated. (Eros Taborelli, Erick Erickson, ab)
+
+* LUCENE-8267: Memory codecs have been removed from the codebase (MemoryPostings,
+  MemoryDocValues). If you used postingsFormat="Memory" or docValuesFormat="Memory"
+  then either remove it to use the default or experiment with one of the others. (Dawid Weiss)
+
+* SOLR-12586: The date format patterns used by ParseDateFieldUpdateProcessorFactory (present in "schemaless mode")
+  are now interpreted by Java 8's java.time.DateTimeFormatter instead of Joda Time.  The pattern language is very
+  similar but not the same.  Typically, simply update the pattern by changing an uppercase 'Z' to lowercase 'z' and
+  that's it.  For the current recommended set of patterns in schemaless mode, see "Schemaless Mode" in the ref guide,
+  or simply examine the default configSet.  Also note that the set of patterns (formats) here have
+  expanded from before to subsume those patterns previously handled by the "extract" contrib (Solr Cell / Tika).
+  (David Smiley, Bar Rotstein)
+
+* SOLR-12593: The "extraction" contrib (Solr Cell) no longer does any date parsing, and thus no longer has the
+  "date.formats" configuration.  To ensure date strings are properly parsed, use ParseDateFieldUpdateProcessorFactory
+  (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein)
+
+* SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true.  If there are unforseen highlight problems,
+  this may be the culprit.
+
+New Features
+----------------------
+
+* SOLR-12591: Expand the set of recognized date format patterns of schemaless mode to subsume those handled by the
+  "extract" contrib (Solr Cell / Tika).  This is primarily a change in configuration of the default configSet for more
+  patterns, but also included enabling "lenient" parsing in ParseDateFieldUpdateProcessorFactory.  The default
+  locale was changed from ROOT to en_US since well-known patterns assume this locale.
+  (David Smiley, Bar Rotstein)
+
+* SOLR-12879: MinHash query parser that builds queries providing a measure of Jaccard similarity (Andy Hind via Tommaso Teofili)
+
+* SOLR-12593: The default configSet now includes an "ignored_*" dynamic field.  (David Smiley)
+
+Optimizations
+----------------------
+
+* SOLR-12725: ParseDateFieldUpdateProcessorFactory should reuse ParsePosition. (ab)
+
+Other Changes
+----------------------
+
+* SOLR-12614: Make "Nodes" view the default in AdminUI "Cloud" tab (janhoy)
+
+* SOLR-12586: Upgrade ParseDateFieldUpdateProcessorFactory (present in "schemaless mode") to use Java 8's
+  java.time.DateTimeFormatter instead of Joda time (see upgrade notes).  "Lenient" is enabled.  Removed Joda Time dependency.
+  (David Smiley, Bar Rotstein)
+
+* SOLR-5163: edismax now throws an exception when qf refers to a nonexistent field (Charles Sanders, David Smiley)
+
+* SOLR-12805: Store previous term (generation) of replica when start recovery process (Cao Manh Dat)
+
+* SOLR-12652: Remove SolrMetricManager.overridableRegistryName method (Peter Somogyi via David Smiley)
+
+* LUCENE-8513: SlowCompositeReaderWrapper now uses MultiTerms directly instead of MultiFields (David Smiley)
+
+* SOLR-11812: Remove backward compatibility of old LIR implementation in 8.0 (Cao Manh Dat)
+
+* SOLR-12620: Remove the Admin UI Cloud -> Graph (Radial) view (janhoy)
+
 ==================  7.6.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
@@ -91,6 +173,8 @@ New Features
 
 * SOLR-12862: Add log10 Stream Evaluator and allow the pow Stream Evaluator to accept a vector of exponents (Joel Bernstein)
 
+* SOLR-12942: Add an option in IndexSizeTrigger to select the split shard method. (ab)
+
 * SOLR-12938: Cluster Status returns results for aliases, instead of throwing exceptions (Gus Heck)
 
 * SOLR-11997: Suggestions API/UI should show an entry where a violation could not be resolved (noble)
@@ -161,6 +245,8 @@ Bug Fixes
 * SOLR-12875: fix ArrayIndexOutOfBoundsException when unique(field) or uniqueBlock(_root_) is
 used with DVHASH method in json.facet. (Tim Underwood via Mikhail Khludnev)
 
+* SOLR-12954: fix facet.pivot refinement bugs when using facet.sort=index and facet.mincount>1 (hossman)
+
 * SOLR-12023: Autoscaling policy engine shuffles replicas needlessly (noble)
 
 * SOLR-12243: Edismax missing phrase queries when phrases contain multiterm synonyms
@@ -185,6 +271,8 @@ Improvements
 
 * SOLR-12892: MapWriter to use CharSequence instead of String (noble)
 
+* SOLR-12882: Eliminate excessive lambda allocation in json facets FacetFieldProcessorByHashDV (Tim Underwood)
+
 * SOLR-12699: Make contrib/ltr LTRScoringModel immutable and cache its hashCode.
   (Stanislav Livotov, Edward Ribeiro, Christine Poerschke)
 
@@ -460,13 +548,12 @@ Optimizations
 * SOLR-11654: Time Routed Alias will now route documents to the ideal shard of a collection, thus avoiding a hop.
   Usually documents were already routed well but not always.  (Gus Heck, David Smiley)
 
-* SOLR-12305: When a replica is applying updates, some kind of updates can skip buffering for faster recovery.
-  (Cao Manh Dat)
-
 * SOLR-11598: The export handler does not limit users to 4 sort fields and is now unlimited. However the speed at
   which we can export is directly proportional to the number of sort fields specified. This change also allows streaming
   expressions to group by on more than 4 fields. (Aroop Ganguly, Amrit Sarkar, Varun Thacker)
 
+* SOLR-12305: When a replica is applying updates, some kind of updates can skip buffering for faster recovery.
+  (Cao Manh Dat)
 
 * SOLR-12509: Improve SplitShardCmd performance and reliability. A new method of splitting has been
   introduced (splitMethod=link) which uses hard-linking of index files when possible, resulting in
@@ -516,6 +603,8 @@ Other Changes
 
 * SOLR-12617: Remove Commons BeanUtils as a dependency (Varun Thacker)
 
+* SOLR-11008: Use a lighter config for MetricsHandlerTest and ensure the core is up before the test starts (Varun Thacker)
+
 * SOLR-11766: Move Streaming Expressions section in Ref Guide to be a top-level section. (Cassandra Targett)
 
 * SOLR-12656: ShardSplitTest should extend AbstractFullDistribZkTestBase instead of BasicDistributedZkTest. (shalin)
@@ -646,6 +735,11 @@ New Features
 * SOLR-12328: JSON Facet API: Domain change with graph query.
   (Daniel Meehl, Kevin Watters, yonik)
 
+* SOLR-11453: Configuring slowQueryThresholdMillis logs slow requests to a separate file - solr_slow_requests.log.
+  (Shawn Heisey, Remko Popma, Varun Thacker)
+
+* SOLR-12401: Add getValue() and setValue() Stream Evaluators (Joel Bernstein, janhoy)
+
 * SOLR-11779, SOLR-12438: Basic long-term collection of aggregated metrics. Historical data is
   maintained as multi-resolution time series using round-robin databases in the '.system'
   collection. New /admin/metrics/history API allows retrieval of this data in numeric
@@ -795,11 +889,11 @@ Bug Fixes
 * SOLR-3567: Spellcheck custom parameters not being passed through due to wrong prefix creation.
   (Josh Lucas via shalin)
 
-* SOLR-12294: update processors loaded from runtime jars fail to load if they are specified
-   in an update processor chain (noble)
-
 * SOLR-12358: Autoscaling suggestions fail randomly with sorting (noble)
 
+* SOLR-12294: update processors loaded from runtime jars fail to load if they are specified
+  in an update processor chain (noble)
+
 * SOLR-12314: Use http timeout's defined in solr.xml for creating ConcurrentUpdateSolrClient during
   indexing requests between leader and replica ( Mark Miller, Varun Thacker)
 
@@ -853,7 +947,7 @@ Optimizations
 * SOLR-11880: Avoid creating new exceptions for every request made to MDCAwareThreadPoolExecutor by distributed
   search and update operations. (Varun Thacker, shalin)
 
-* SOLR-12375: Optimize Lucene needsScore / ScoreMode use:
+* SOLR-12375: Optimize Lucene ScoreMode use:
   A non-cached filter query could be told incorrectly that scores were needed.
   The /export (ExportQParserPlugin) would declare incorrectly that scores are needed.
   Expanded docs (expand component) could be told incorrectly that scores are needed.  (David Smiley)
@@ -946,6 +1040,8 @@ Other Changes
 * SOLR-12435: Fix bin/solr help and ref guide text to describe ZK_HOST in solr.in.sh/solr.in.cmd
   as an alternative to -z cmdline param. (Steve Rowe)
 
+* SOLR-12428: Solr LTR jar now included in _default configset's solrconfig.xml (Ishan Chattopadhyaya)
+
 ==================  7.3.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.


[24/50] [abbrv] lucene-solr:jira/http2: SOLR-12938 - fix test case for handling of bogus collection names that was failing when HttpClusterStateProvider is used instead of ZkClusterStateProvider

Posted by da...@apache.org.
SOLR-12938 - fix test case for handling of bogus collection names
that was failing when HttpClusterStateProvider is used instead of
ZkClusterStateProvider


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

Branch: refs/heads/jira/http2
Commit: 53482e510c773fabd80fc0a5a0efa1f137f89def
Parents: 4794a16
Author: Gus Heck <gu...@apache.org>
Authored: Wed Nov 7 20:05:32 2018 -0500
Committer: Gus Heck <gu...@apache.org>
Committed: Wed Nov 7 20:05:32 2018 -0500

----------------------------------------------------------------------
 .../apache/solr/handler/admin/ClusterStatus.java  |  4 +++-
 .../solrj/impl/HttpClusterStateProvider.java      | 18 +++++++++++++-----
 2 files changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53482e51/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
index 4daae31..463c5b6 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
@@ -112,7 +112,9 @@ public class ClusterStatus {
       DocCollection clusterStateCollection = entry.getValue();
       if (clusterStateCollection == null) {
         if (collection != null) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
+          SolrException solrException = new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
+          solrException.setMetadata("CLUSTERSTATUS","NOT_FOUND");
+          throw solrException;
         } else {
           //collection might have got deleted at the same time
           continue;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53482e51/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
index 484eaad..bbab3aa 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
@@ -95,7 +95,13 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
           withHttpClient(httpClient).build()) {
         ClusterState cs = fetchClusterState(client, collection, null);
         return cs.getCollectionRef(collection);
-      } catch (SolrServerException | RemoteSolrException | IOException e) {
+      } catch (SolrServerException | IOException e) {
+        log.warn("Attempt to fetch cluster state from " +
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+      } catch (RemoteSolrException e) {
+        if ("NOT_FOUND".equals(e.getMetadata("CLUSTERSTATUS"))) {
+          return null;
+        }
         log.warn("Attempt to fetch cluster state from " +
             Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       } catch (NotACollectionException e) {
@@ -257,9 +263,9 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
         log.warn("Attempt to fetch cluster state from " +
             Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       } catch (NotACollectionException e) {
-        // Cluster state for the given collection was not found, could be an alias.
-        // Lets fetch/update our aliases:
-        getAliases(true);
+        // not possible! (we passed in null for collection so it can't be an alias)
+        throw new RuntimeException("null should never cause NotACollectionException in " +
+            "fetchClusterState() Please report this as a bug!");
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
@@ -282,7 +288,9 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
         log.warn("Attempt to fetch cluster state from " +
             Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       } catch (NotACollectionException e) {
-        // should be an an alias, don't care
+        // not possible! (we passed in null for collection so it can't be an alias)
+        throw new RuntimeException("null should never cause NotACollectionException in " +
+            "fetchClusterState() Please report this as a bug!");
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes + ". However, "


[11/50] [abbrv] lucene-solr:jira/http2: SOLR-12600: Add missing name in CHANGES Missed it on the last commit

Posted by da...@apache.org.
SOLR-12600: Add missing name in CHANGES
Missed it on the last commit


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

Branch: refs/heads/jira/http2
Commit: 50cf1cca265de83438163bdc4c7830f99a24a11d
Parents: 419d42b
Author: Alexandre Rafalovitch <ar...@apache.org>
Authored: Wed Nov 7 07:11:03 2018 -0500
Committer: Alexandre Rafalovitch <ar...@apache.org>
Committed: Wed Nov 7 07:11:03 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50cf1cca/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 928cd44..77e8665 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -208,7 +208,7 @@ Other Changes
 
 * SOLR-12966: Add Javadoc @since tag to URP classes (Alexandre Rafalovitch)
 
-* SOLR-12600: Fix parameter names in Solr JSON documentation
+* SOLR-12600: Fix parameter names in Solr JSON documentation (Alexandre Rafalovitch)
 
 Bug Fixes
 ----------------------


[17/50] [abbrv] lucene-solr:jira/http2: SOLR-11572: Update CHANGES.txt

Posted by da...@apache.org.
SOLR-11572: 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/7bde5e23
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7bde5e23
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7bde5e23

Branch: refs/heads/jira/http2
Commit: 7bde5e2343f1481ddb3d0fec93a04da93d8dad75
Parents: bc88d78
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Nov 7 09:38:19 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Nov 7 09:38:19 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bde5e23/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1ddb471..35bd675 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -183,6 +183,8 @@ New Features
 
 * SOLR-12795: Introduce 'rows' and 'offset' parameter in FacetStream (Joel Bernstein, Amrit Sarkar, Varun Thacker)
 
+* SOLR-11572: Add recip Stream Evaluator to support reciprocal transformations (Joel Bernstein)
+
 
 Other Changes
 ----------------------


[45/50] [abbrv] lucene-solr:jira/http2: SOLR-12965: Add facet support to JsonQueryRequest

Posted by da...@apache.org.
SOLR-12965: Add facet support to JsonQueryRequest


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

Branch: refs/heads/jira/http2
Commit: 52998fa50e60ce9c7f49167b1ab107347c30d8d6
Parents: 4e2481b
Author: Jason Gerlowski <ge...@apache.org>
Authored: Sat Nov 10 19:48:50 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Sat Nov 10 21:44:25 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/json-facet-api.adoc     |  36 ++
 .../client/solrj/request/json/DomainMap.java    | 139 +++++
 .../solrj/request/json/HeatmapFacetMap.java     | 137 +++++
 .../client/solrj/request/json/JsonFacetMap.java |  62 ++
 .../solrj/request/json/JsonQueryRequest.java    | 109 ++++
 .../solrj/request/json/QueryFacetMap.java       |  39 ++
 .../solrj/request/json/RangeFacetMap.java       | 105 ++++
 .../solrj/request/json/TermsFacetMap.java       | 204 ++++++
 .../solrj/src/test-files/solrj/techproducts.xml | 421 +++++++++++++
 .../ref_guide_examples/JsonRequestApiTest.java  |  98 ++-
 ...JsonQueryRequestFacetingIntegrationTest.java | 615 +++++++++++++++++++
 .../solrj/request/json/DomainMapTest.java       | 177 ++++++
 .../solrj/request/json/HeatmapFacetMapTest.java | 130 ++++
 ...JsonQueryRequestFacetingIntegrationTest.java | 530 ++++++++++++++++
 .../request/json/JsonQueryRequestUnitTest.java  |  86 ++-
 .../solrj/request/json/QueryFacetMapTest.java   |  45 ++
 .../solrj/request/json/RangeFacetMapTest.java   |  84 +++
 .../solrj/request/json/TermsFacetMapTest.java   | 189 ++++++
 18 files changed, 3202 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solr-ref-guide/src/json-facet-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/json-facet-api.adoc b/solr/solr-ref-guide/src/json-facet-api.adoc
index 1e52034..1d1a285 100644
--- a/solr/solr-ref-guide/src/json-facet-api.adoc
+++ b/solr/solr-ref-guide/src/json-facet-api.adoc
@@ -1,5 +1,7 @@
 = JSON Facet API
 :page-tocclass: right
+:solr-root-path: ../../
+:example-source-dir: {solr-root-path}solrj/src/test/org/apache/solr/client/ref_guide_examples/
 // 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
@@ -74,6 +76,11 @@ The response to the facet request above will start with documents matching the r
 
 Here's an example of a bucketing facet, that partitions documents into bucket based on the `cat` field (short for category), and returns the top 3 buckets:
 
+[.dynamic-tabs]
+--
+[example.tab-pane#curljsonsimpletermsfacet]
+====
+[.tab-label]*curl*
 [source,bash]
 ----
 curl http://localhost:8983/solr/techproducts/query -d 'q=*:*&
@@ -85,6 +92,18 @@ json.facet={
   }
 }'
 ----
+====
+
+[example.tab-pane#solrjjsonsimpletermsfacet]
+====
+[.tab-label]*SolrJ*
+
+[source,java,indent=0]
+----
+include::{example-source-dir}JsonRequestApiTest.java[tag=solrj-json-simple-terms-facet]
+----
+====
+--
 
 The response below shows us that 32 documents match the default root domain. and 12 documents have `cat:electronics`, 4 documents have `cat:currency`, etc.
 
@@ -132,6 +151,11 @@ curl http://localhost:8983/solr/techproducts/query -d 'q=*:*&json.facet=
 
 Another option is to use the JSON Request API to provide the entire request in JSON:
 
+[.dynamic-tabs]
+--
+[example.tab-pane#curljsontermsfacet2]
+====
+[.tab-label]*curl*
 [source,bash]
 ----
 curl http://localhost:8983/solr/techproducts/query -d '
@@ -144,6 +168,18 @@ curl http://localhost:8983/solr/techproducts/query -d '
 }
 '
 ----
+====
+
+[example.tab-pane#solrjjsontermsfacet2]
+====
+[.tab-label]*SolrJ*
+
+[source,java,indent=0]
+----
+include::{example-source-dir}JsonRequestApiTest.java[tag=solrj-json-terms-facet2]
+----
+====
+--
 
 === JSON Extensions
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DomainMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DomainMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DomainMap.java
new file mode 100644
index 0000000..c23cee9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/DomainMap.java
@@ -0,0 +1,139 @@
+/*
+ * 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.request.json;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DomainMap extends HashMap<String, Object> {
+
+  /**
+   * Indicates that the domain should be narrowed by the specified filter
+   *
+   * May be called multiple times.  Each added filter is retained and used to narrow the domain.
+   */
+  public DomainMap withFilter(String filter) {
+    if (filter == null) {
+      throw new IllegalArgumentException("Parameter 'filter' must be non-null");
+    }
+
+    if (! containsKey("filter")) {
+      put("filter", new ArrayList<String>());
+    }
+
+    final List<String> filterList = (List<String>) get("filter");
+    filterList.add(filter);
+    return this;
+  }
+
+  /**
+   * Indicates that the domain should be the following query
+   *
+   * May be called multiple times.  Each specified query is retained and included in the domain.
+   */
+  public DomainMap withQuery(String query) {
+    if (query == null) {
+      throw new IllegalArgumentException("Parameter 'query' must be non-null");
+    }
+
+    if (! containsKey("query")) {
+      put("query", new ArrayList<String>());
+    }
+
+    final List<String> queryList = (List<String>) get("query");
+    queryList.add(query);
+    return this;
+  }
+
+  /**
+   * Provide a tag or tags that correspond to filters or queries to exclude from the domain
+   *
+   * May be called multiple times.  Each exclude-string is retained and used for removing queries/filters from the
+   * domain specification.
+   *
+   * @param excludeTagsValue a comma-delimited String containing filter/query tags to exclude
+   */
+  public DomainMap withTagsToExclude(String excludeTagsValue) {
+    if (excludeTagsValue == null) {
+      throw new IllegalArgumentException("Parameter 'excludeTagValue' must be non-null");
+    }
+
+    if (! containsKey("excludeTags")) {
+      put("excludeTags", new ArrayList<String>());
+    }
+
+    final List<String> excludeTagsList = (List<String>) get("excludeTags");
+    excludeTagsList.add(excludeTagsValue);
+    return this;
+  }
+
+  /**
+   * Indicates that the resulting domain will contain all parent documents of the children in the existing domain
+   *
+   * @param allParentsQuery a query used to identify all parent documents in the collection
+   */
+  public DomainMap setBlockParentQuery(String allParentsQuery) {
+    if (allParentsQuery == null) {
+      throw new IllegalArgumentException("Parameter 'allParentsQuery' must be non-null");
+    }
+
+    put("blockParent", allParentsQuery);
+    return this;
+  }
+
+  /**
+   * Indicates that the resulting domain will contain all child documents of the parents in the current domain
+   *
+   * @param allChildrenQuery a query used to identify all child documents in the collection
+   */
+  public DomainMap setBlockChildQuery(String allChildrenQuery) {
+    if (allChildrenQuery == null) {
+      throw new IllegalArgumentException("Parameter 'allChildrenQuery' must be non-null");
+    }
+
+    put("blockChildren", allChildrenQuery);
+    return this;
+  }
+
+  /**
+   * Transforms the domain by running a join query with the provided {@code from} and {@code to} parameters
+   *
+   * Join modifies the current domain by selecting the documents whose values in field {@code to} match values for the
+   * field {@code from} in the current domain.
+   *
+   * @param from a field-name whose values are matched against {@code to} by the join
+   * @param to a field name whose values should match values specified by the {@code from} field
+   */
+  public DomainMap setJoinTransformation(String from, String to) {
+    if (from == null) {
+      throw new IllegalArgumentException("Parameter 'from' must be non-null");
+    }
+    if (to == null) {
+      throw new IllegalArgumentException("Parameter 'to' must be non-null");
+    }
+
+    final Map<String, Object> joinParameters = new HashMap<>();
+    joinParameters.put("from", from);
+    joinParameters.put("to", to);
+    put("join", joinParameters);
+
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/HeatmapFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/HeatmapFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/HeatmapFacetMap.java
new file mode 100644
index 0000000..ed64e08
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/HeatmapFacetMap.java
@@ -0,0 +1,137 @@
+/*
+ * 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.request.json;
+
+import java.util.Map;
+
+/**
+ * Represents a "heatmap" facet in a JSON request query.
+ *
+ * Ready for use with {@link JsonQueryRequest#withFacet(String, Map)}
+ */
+public class HeatmapFacetMap extends JsonFacetMap<HeatmapFacetMap> {
+  public HeatmapFacetMap(String fieldName) {
+    super("heatmap");
+
+    if (fieldName == null) {
+      throw new IllegalArgumentException("Parameter 'fieldName' must be non-null");
+    }
+
+    put("field", fieldName);
+  }
+
+  @Override
+  public HeatmapFacetMap getThis() { return this; }
+
+  @Override
+  public HeatmapFacetMap withSubFacet(String facetName, JsonFacetMap map) {
+    throw new UnsupportedOperationException(getClass().getName() + " doesn't currently support subfacets");
+  }
+
+  /**
+   * Indicate the region to compute the heatmap facet on.
+   *
+   * Defaults to the "world" ("[-180,-90 TO 180,90]")
+   */
+  public HeatmapFacetMap setRegionQuery(String queryString) {
+    if (queryString == null) {
+      throw new IllegalArgumentException("Parameter 'queryString' must be non-null");
+    }
+
+    put("geom", queryString);
+    return this;
+  }
+
+  /**
+   * Indicates the size of each cell in the computed heatmap grid
+   *
+   * If not set, defaults to being computed by {@code distErrPct} or {@code distErr}
+   *
+   * @param individualCellSize the forced size of each cell in the heatmap grid
+   *
+   * @see #setDistErr(double)
+   * @see #setDistErrPct(double)
+   */
+  public HeatmapFacetMap setGridLevel(int individualCellSize) {
+    if (individualCellSize <= 0) {
+      throw new IllegalArgumentException("Parameter 'individualCellSize' must be a positive integer");
+    }
+    put("gridLevel", individualCellSize);
+    return this;
+  }
+
+  /**
+   * A fraction of the heatmap region that is used to compute the cell size.
+   *
+   * Defaults to 0.15 if not specified.
+   *
+   * @see #setGridLevel(int)
+   * @see #setDistErr(double)
+   */
+  public HeatmapFacetMap setDistErrPct(double distErrPct) {
+    if (distErrPct < 0 || distErrPct > 1) {
+      throw new IllegalArgumentException("Parameter 'distErrPct' must be between 0.0 and 1.0");
+    }
+    put("distErrPct", distErrPct);
+    return this;
+  }
+
+  /**
+   * Indicates the maximum acceptable cell error distance.
+   *
+   * Used to compute the size of each cell in the heatmap grid rather than specifying {@link #setGridLevel(int)}
+   *
+   * @param distErr a positive value representing the maximum acceptable cell error.
+   *
+   * @see #setGridLevel(int)
+   * @see #setDistErrPct(double)
+   */
+  public HeatmapFacetMap setDistErr(double distErr) {
+    if (distErr < 0) {
+      throw new IllegalArgumentException("Parameter 'distErr' must be non-negative");
+    }
+    put("distErr", distErr);
+    return this;
+  }
+
+  public enum HeatmapFormat {
+    INTS2D("ints2D"), PNG("png");
+
+    private final String value;
+
+    HeatmapFormat(String value) {
+      this.value = value;
+    }
+
+    @Override
+    public String toString() { return value; }
+  }
+
+  /**
+   * Sets the format that the computed heatmap should be returned in.
+   *
+   * Defaults to 'ints2D' if not specified.
+   */
+  public HeatmapFacetMap setHeatmapFormat(HeatmapFormat format) {
+    if (format == null) {
+      throw new IllegalArgumentException("Parameter 'format' must be non-null");
+    }
+    put("format", format.toString());
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonFacetMap.java
new file mode 100644
index 0000000..3d3e6de
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonFacetMap.java
@@ -0,0 +1,62 @@
+/*
+ * 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.request.json;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A common parent for a small set of classes that allow easier composition of JSON facet objects.
+ *
+ * Designed for use with {@link JsonQueryRequest#withFacet(String, Map)}
+ */
+public abstract class JsonFacetMap<B extends JsonFacetMap<B>> extends HashMap<String, Object> {
+
+  public abstract B getThis(); // Allows methods shared here to return subclass type
+
+  public JsonFacetMap(String facetType) {
+    super();
+
+    put("type", facetType);
+  }
+
+  public B withDomain(DomainMap domain) {
+    put("domain", domain);
+    return getThis();
+  }
+
+  public B withSubFacet(String facetName, JsonFacetMap map) {
+    if (! containsKey("facet")) {
+      put("facet", new HashMap<String, Object>());
+    }
+
+    final Map<String, Object> subFacetMap = (Map<String, Object>) get("facet");
+    subFacetMap.put(facetName, map);
+    return getThis();
+  }
+
+  public B withStatSubFacet(String facetName, String statFacet) {
+    if (! containsKey("facet")) {
+      put("facet", new HashMap<String, Object>());
+    }
+
+    final Map<String, Object> subFacetMap = (Map<String, Object>) get("facet");
+    subFacetMap.put(facetName, statFacet);
+    return getThis();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
index 781d9c3..1c7b071 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/JsonQueryRequest.java
@@ -133,6 +133,115 @@ public class JsonQueryRequest extends QueryRequest {
   }
 
   /**
+   * Specify a facet sent as a part of this JSON request.
+   *
+   * This method may be called multiple times.  Each call made with a different {@code facetName} value will add a new
+   * top-level facet.  Repeating {@code facetName} values will cause previous facets with that {@code facetName} to be
+   * overwritten.
+   * <p>
+   * <b>Example:</b> You wish to send the JSON request: {"query": "*:*", "facet": { "top_cats":{"type": "terms", "field":"cat"}}}.  You
+   * would represent (and attach) the facet in this request as follows:
+   * <pre>{@code
+   *     final Map<String, Object> catFacetMap = new HashMap<>();
+   *     catFacetMap.put("type", "terms");
+   *     catFacetMap.put("field", "cat");
+   *
+   *     jsonQueryRequest.withStatFacet("top_cats", catFacetMap);
+   * }</pre>
+   *
+   * @param facetName the name of the top-level facet you'd like to add.
+   * @param facetJson a Map of values representing the facet you wish to add to the request
+   */
+  public JsonQueryRequest withFacet(String facetName, Map<String, Object> facetJson) {
+    if (facetName == null) {
+      throw new IllegalArgumentException("'facetName' parameter must be non-null");
+    }
+    if (facetJson == null) {
+      throw new IllegalArgumentException("'facetMap' parameter must be non-null");
+    }
+
+    if (! jsonRequestMap.containsKey("facet")) {
+      jsonRequestMap.put("facet", new HashMap<String, Object>());
+    }
+
+    final Map<String, Object> facetMap = (Map<String, Object>) jsonRequestMap.get("facet");
+    facetMap.put(facetName, facetJson);
+    return this;
+  }
+
+  /**
+   * Specify a facet sent as a part of this JSON request.
+   *
+   * This method may be called multiple times.  Each call made with a different {@code facetName} value will add a new
+   * top-level facet.  Repeating {@code facetName} values will cause previous facets with that {@code facetName} to be
+   * overwritten.
+   * <p>
+   * <b>Example:</b> You wish to send the JSON request: {"query": "*:*", "facet": { "top_cats":{"type": "terms", "field":"cat"}}}.  You
+   * would represent the facet in this request as follows:
+   * <pre>
+   *     final MapWriter facetWriter = new MapWriter() {
+   *         &#64;Override
+   *         public void writeMap(EntryWriter ew) throws IOException {
+   *             ew.put("type", "terms");
+   *             ew.put("field", "cat");
+   *         }
+   *     };
+   * </pre>
+   *
+   * @param facetName the name of the top-level facet you'd like to add.
+   * @param facetWriter a MapWriter representing the facet you wish to add to the request
+   */
+  public JsonQueryRequest withFacet(String facetName, MapWriter facetWriter) {
+    if (facetName == null) {
+      throw new IllegalArgumentException("'facetName' parameter must be non-null");
+    }
+    if (facetWriter == null) {
+      throw new IllegalArgumentException("'facetWriter' parameter must be non-null");
+    }
+
+    if (! jsonRequestMap.containsKey("facet")) {
+      jsonRequestMap.put("facet", new HashMap<String, Object>());
+    }
+
+    final Map<String, Object> facetMap = (Map<String, Object>) jsonRequestMap.get("facet");
+    facetMap.put(facetName, facetWriter);
+    return this;
+  }
+
+  /**
+   * Specify a simple stat or aggregation facet to be sent as a part of this JSON request.
+   *
+   * This method may be called multiple times.  Each call made with a different {@code facetName} value will add a new
+   * top-level facet.  Repeating {@code facetName} values will cause previous facets with that {@code facetName} to be
+   * overwritten.
+   * <p>
+   * <b>Example:</b>  You wish to send the JSON request: {"query": "*:*", "facet": {"avg_price": "avg(price)"}}.  You
+   * would represent the facet in this request as follows:
+   * <pre>{@code
+   *     jsonQueryRequest.withStatFacet("avg_price", "avg(price)");
+   * }</pre>
+   *
+   * @param facetName the name of the top-level stat/agg facet you'd like to add.
+   * @param facetValue a String representing the stat/agg facet computation to perform.
+   */
+  public JsonQueryRequest withStatFacet(String facetName, String facetValue) {
+    if (facetName == null) {
+      throw new IllegalArgumentException("'facetName' parameter must be non-null");
+    }
+    if (facetValue == null) {
+      throw new IllegalArgumentException("'facetValue' parameter must be non-null");
+    }
+
+    if (! jsonRequestMap.containsKey("facet")) {
+      jsonRequestMap.put("facet", new HashMap<String, Object>());
+    }
+
+    final Map<String, Object> facetMap = (Map<String, Object>) jsonRequestMap.get("facet");
+    facetMap.put(facetName, facetValue);
+    return this;
+  }
+
+  /**
    * Specify whether results should be fetched starting from a particular offset (or 'start').
    *
    * Defaults to 0 if not set.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/QueryFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/QueryFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/QueryFacetMap.java
new file mode 100644
index 0000000..7613183
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/QueryFacetMap.java
@@ -0,0 +1,39 @@
+/*
+ * 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.request.json;
+
+import java.util.Map;
+
+/**
+ * Represents a "query" facet in a JSON query request.
+ *
+ * Ready for use in {@link JsonQueryRequest#withFacet(String, Map)}
+ */
+public class QueryFacetMap extends JsonFacetMap<QueryFacetMap> {
+  public QueryFacetMap(String queryString) {
+    super("query");
+
+    if (queryString == null) {
+      throw new IllegalArgumentException("Parameter 'queryString' must be non-null");
+    }
+    put("q", queryString);
+  }
+
+  @Override
+  public QueryFacetMap getThis() { return this; }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
new file mode 100644
index 0000000..24d5123
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/RangeFacetMap.java
@@ -0,0 +1,105 @@
+/*
+ * 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.request.json;
+
+import java.util.Map;
+
+/**
+ * Represents a "range" facet in a JSON request query.
+ *
+ * Ready for use with {@link JsonQueryRequest#withFacet(String, Map)}
+ */
+public class RangeFacetMap extends JsonFacetMap<RangeFacetMap> {
+  public RangeFacetMap(String field, long start, long end, long gap) {
+    super("range");
+
+    if (field == null) {
+      throw new IllegalArgumentException("Parameter 'field' must be non-null");
+    }
+    if (end < start) {
+      throw new IllegalArgumentException("Parameter 'end' must be greater than parameter 'start'");
+    }
+    if (gap <= 0) {
+      throw new IllegalArgumentException("Parameter 'gap' must be a positive integer");
+    }
+
+    put("field", field);
+    put("start", start);
+    put("end", end);
+    put("gap", gap);
+  }
+
+  public RangeFacetMap(String field, double start, double end, double gap) {
+    super("range");
+
+    if (field == null) {
+      throw new IllegalArgumentException("Parameter 'field' must be non-null");
+    }
+    if (end < start) {
+      throw new IllegalArgumentException("Parameter 'end' must be greater than parameter 'start'");
+    }
+    if (gap <= 0) {
+      throw new IllegalArgumentException("Parameter 'gap' must be a positive value");
+    }
+
+    put("field", field);
+    put("start", start);
+    put("end", end);
+    put("gap", gap);
+  }
+
+  @Override
+  public RangeFacetMap getThis() { return this; }
+
+  /**
+   * Indicates whether the facet's last bucket should stop exactly at {@code end}, or be extended to be {@code gap} wide
+   *
+   * Defaults to false if not specified.
+   *
+   * @param hardEnd true if the final bucket should be truncated at {@code end}; false otherwise
+   */
+  public RangeFacetMap setHardEnd(boolean hardEnd) {
+    put("hardend", hardEnd);
+    return this;
+  }
+
+  public enum OtherBuckets {
+    BEFORE("before"), AFTER("after"), BETWEEN("between"), NONE("none"), ALL("all");
+
+    private final String value;
+
+    OtherBuckets(String value) {
+      this.value = value;
+    }
+
+    public String toString() { return value; }
+  }
+
+  /**
+   * Indicates that an additional range bucket(s) should be computed and added to those computed for {@code start} and {@code end}
+   *
+   * See {@link OtherBuckets} for possible options.
+   */
+  public RangeFacetMap setOtherBuckets(OtherBuckets bucketSpecifier) {
+    if (bucketSpecifier == null) {
+      throw new IllegalArgumentException("Parameter 'bucketSpecifier' must be non-null");
+    }
+    put("other", bucketSpecifier.toString());
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/TermsFacetMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/TermsFacetMap.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/TermsFacetMap.java
new file mode 100644
index 0000000..e28f8a8
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/json/TermsFacetMap.java
@@ -0,0 +1,204 @@
+/*
+ * 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.request.json;
+
+import java.util.Map;
+
+/**
+ * Represents a "terms" facet in a JSON query request.
+ *
+ * Ready for use in {@link JsonQueryRequest#withFacet(String, Map)}
+ */
+public class TermsFacetMap extends JsonFacetMap<TermsFacetMap> {
+  public TermsFacetMap(String fieldName) {
+    super("terms");
+
+    put("field", fieldName);
+  }
+
+  @Override
+  public TermsFacetMap getThis() { return this; }
+
+  /**
+   * Indicates that Solr should skip over the N buckets for this facet.
+   *
+   * Used for "paging" in facet results.  Defaults to 0 if not provided.
+   *
+   * @param numToSkip the number of buckets to skip over before selecting the buckets to return
+   */
+  public TermsFacetMap setBucketOffset(int numToSkip) {
+    if (numToSkip < 0) {
+      throw new IllegalArgumentException("Parameter 'numToSkip' must be non-negative");
+    }
+    put("offset", numToSkip);
+    return this;
+  }
+
+  /**
+   * Indicates the maximum number of buckets to be returned by this facet.
+   *
+   * Defaults to 10 if not specified.
+   */
+  public TermsFacetMap setLimit(int maximumBuckets) {
+    if (maximumBuckets < 0) {
+      throw new IllegalArgumentException("Parameter 'maximumBuckets' must be non-negative");
+    }
+    put("limit", maximumBuckets);
+    return this;
+  }
+
+  /**
+   * Indicates the desired ordering for the returned buckets.
+   *
+   * Values can be based on 'count' (the number of results in each bucket), 'index' (the natural order of bucket values),
+   * or on any stat facet that occurs in the bucket.  Defaults to "count desc" if not specified.
+   */
+  public TermsFacetMap setSort(String sortString) {
+    if (sortString == null) {
+      throw new IllegalArgumentException("Parameter 'sortString' must be non-null");
+    }
+    put("sort", sortString);
+    return this;
+  }
+
+  /**
+   * Indicates the number of additional buckets to request internally beyond those required by {@link #setLimit(int)}.
+   *
+   * Defaults to -1 if not specified, which triggers some heuristic guessing based on other settings.
+   */
+  public TermsFacetMap setOverRequest(int numExtraBuckets) {
+    if (numExtraBuckets < -1) {
+      throw new IllegalArgumentException("Parameter 'numExtraBuckets' must be >= -1");
+    }
+    put("overrequest", numExtraBuckets);
+    return this;
+  }
+
+  /**
+   * Indicates whether this facet should use distributed facet refining.
+   *
+   * "Distributed facet refining" is a second, optional stage in the facet process that ensures that counts for the
+   * returned buckets are exact.  Enabling it is a tradeoff between precision and speed/performance.  Defaults to false
+   * if not specified.
+   * @param useRefining true if distributed facet refining should be used; false otherwise
+   */
+  public TermsFacetMap useDistributedFacetRefining(boolean useRefining) {
+    put("refine", useRefining);
+    return this;
+  }
+
+  /**
+   * Indicates how many extra buckets to request during distributed-facet-refining beyond those required by {@link #setLimit(int)}
+   *
+   * Defaults to -1 if not specified, which triggers some heuristic guessing based on other settings.
+   */
+  public TermsFacetMap setOverRefine(int numExtraBuckets) {
+    if (numExtraBuckets < -1) {
+      throw new IllegalArgumentException("Parameter 'numExtraBuckets' must be >= -1");
+    }
+    put("overrefine", numExtraBuckets);
+    return this;
+  }
+
+  /**
+   * Indicates that the facet results should not include any buckets with a count less than {@code minCount}.
+   *
+   * Defaults to 1 if not specified.
+   */
+  public TermsFacetMap setMinCount(int minCount) {
+    if (minCount < 1) {
+      throw new IllegalArgumentException("Parameter 'minCount' must be a positive integer");
+    }
+    put("mincount", minCount);
+    return this;
+  }
+
+  /**
+   * Indicates that Solr should create a bucket corresponding to documents missing the field used by this facet.
+   *
+   * Defaults to false if not specified.
+   *
+   * @param missingBucket true if the special "missing" bucket should be created; false otherwise
+   */
+  public TermsFacetMap includeMissingBucket(boolean missingBucket) {
+    put("missing", missingBucket);
+    return this;
+  }
+
+  /**
+   * Indicates that Solr should include the total number of buckets for this facet.
+   *
+   * Note that this is different than the number of buckets returned.  Defaults to false if not specified
+   *
+   * @param numBuckets true if the "numBuckets" field should be computed; false otherwise
+   */
+  public TermsFacetMap includeTotalNumBuckets(boolean numBuckets) {
+    put("numBuckets", numBuckets);
+    return this;
+  }
+
+  /**
+   * Creates a bucket representing the union of all other buckets.
+   *
+   * For multi-valued fields this is different than a bucket for the entire domain, since documents can belong to
+   * multiple buckets.  Defaults to false if not specified.
+   *
+   * @param shouldInclude true if the union bucket "allBuckets" should be computed; false otherwise
+   */
+  public TermsFacetMap includeAllBucketsUnionBucket(boolean shouldInclude) {
+    put("allBuckets", shouldInclude);
+    return this;
+  }
+
+  /**
+   * Indicates that the facet should only produce buckets for terms that start with the specified prefix.
+   */
+  public TermsFacetMap setTermPrefix(String termPrefix) {
+    if (termPrefix == null) {
+      throw new IllegalArgumentException("Parameter 'termPrefix' must be non-null");
+    }
+    put("prefix", termPrefix);
+    return this;
+  }
+
+  public enum FacetMethod {
+    DV("dv"), UIF("uif"), DVHASH("dvhash"), ENUM("enum"), STREAM("stream"), SMART("smart");
+
+    private final String value;
+    FacetMethod(String value) {
+      this.value = value;
+    }
+
+    public String toString() {
+      return value;
+    }
+  }
+
+  /**
+   * Indicate which method should be used to compute the facet.
+   *
+   * Defaults to "smart" if not specified, which has Solr guess which computation method will be most efficient.
+   */
+  public TermsFacetMap setFacetMethod(FacetMethod method) {
+    if (method == null) {
+      throw new IllegalArgumentException("Parameter 'method' must be non-null");
+    }
+    put("method", method.toString());
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test-files/solrj/techproducts.xml
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test-files/solrj/techproducts.xml b/solr/solrj/src/test-files/solrj/techproducts.xml
new file mode 100644
index 0000000..15650fa
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/techproducts.xml
@@ -0,0 +1,421 @@
+<!--
+ 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.
+-->
+<add>
+  <doc>
+    <field name="id">TWINX2048-3200PRO</field>
+    <field name="name">CORSAIR  XMS 2GB (2 x 1GB) 184-Pin DDR SDRAM Unbuffered DDR 400 (PC 3200) Dual Channel Kit System Memory - Retail</field>
+    <field name="manu">Corsair Microsystems Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">corsair</field>
+    <field name="cat">electronics</field>
+    <field name="cat">memory</field>
+    <field name="features">CAS latency 2,  2-3-3-6 timing, 2.75v, unbuffered, heat-spreader</field>
+    <field name="price">185.00</field>
+    <field name="popularity">5</field>
+    <field name="inStock">true</field>
+    <!-- San Francisco store -->
+    <field name="store">37.7752,-122.4232</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+    <!-- a field for testing payload tagged text via DelimitedPayloadTokenFilter -->
+    <field name="payloads">electronics|6.0 memory|3.0</field>
+  </doc>
+  <doc>
+    <field name="id">VS1GB400C3</field>
+    <field name="name">CORSAIR ValueSelect 1GB 184-Pin DDR SDRAM Unbuffered DDR 400 (PC 3200) System Memory - Retail</field>
+    <field name="manu">Corsair Microsystems Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">corsair</field>
+    <field name="cat">electronics</field>
+    <field name="cat">memory</field>
+    <field name="price">74.99</field>
+    <field name="popularity">7</field>
+    <field name="inStock">true</field>
+    <!-- Dodge City store -->
+    <field name="store">37.7752,-100.0232</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+    <field name="payloads">electronics|4.0 memory|2.0</field>
+  </doc>
+  <doc>
+    <field name="id">VDBDB1A16</field>
+    <field name="name">A-DATA V-Series 1GB 184-Pin DDR SDRAM Unbuffered DDR 400 (PC 3200) System Memory - OEM</field>
+    <field name="manu">A-DATA Technology Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">corsair</field>
+    <field name="cat">electronics</field>
+    <field name="cat">memory</field>
+    <field name="features">CAS latency 3,   2.7v</field>
+    <!-- note: price & popularity is missing on this one -->
+    <field name="popularity">0</field>
+    <field name="inStock">true</field>
+    <!-- Buffalo store -->
+    <field name="store">45.18414,-93.88141</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+    <field name="payloads">electronics|0.9 memory|0.1</field>
+  </doc>
+  <doc>
+    <field name="id">MA147LL/A</field>
+    <field name="name">Apple 60 GB iPod with Video Playback Black</field>
+    <field name="manu">Apple Computer Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">apple</field>
+    <field name="cat">electronics</field>
+    <field name="cat">music</field>
+    <field name="features">iTunes, Podcasts, Audiobooks</field>
+    <field name="features">Stores up to 15,000 songs, 25,000 photos, or 150 hours of video</field>
+    <field name="features">2.5-inch, 320x240 color TFT LCD display with LED backlight</field>
+    <field name="features">Up to 20 hours of battery life</field>
+    <field name="features">Plays AAC, MP3, WAV, AIFF, Audible, Apple Lossless, H.264 video</field>
+    <field name="features">Notes, Calendar, Phone book, Hold button, Date display, Photo wallet, Built-in games, JPEG photo playback, Upgradeable firmware, USB 2.0 compatibility, Playback speed control, Rechargeable capability, Battery level indication</field>
+    <field name="includes">earbud headphones, USB cable</field>
+    <field name="weight">5.5</field>
+    <field name="price">399.00</field>
+    <field name="popularity">10</field>
+    <field name="inStock">true</field>
+    <!-- Dodge City store -->
+    <field name="store">37.7752,-100.0232</field>
+    <field name="manufacturedate_dt">2005-10-12T08:00:00Z</field>
+  </doc>
+  <doc>
+    <field name="id">F8V7067-APL-KIT</field>
+    <field name="name">Belkin Mobile Power Cord for iPod w/ Dock</field>
+    <field name="manu">Belkin</field>
+    <!-- Join -->
+    <field name="manu_id_s">belkin</field>
+    <field name="cat">electronics</field>
+    <field name="cat">connector</field>
+    <field name="features">car power adapter, white</field>
+    <field name="weight">4.0</field>
+    <field name="price">19.95</field>
+    <field name="popularity">1</field>
+    <field name="inStock">false</field>
+    <!-- Buffalo store -->
+    <field name="store">45.18014,-93.87741</field>
+    <field name="manufacturedate_dt">2005-08-01T16:30:25Z</field>
+  </doc>
+  <doc>
+    <field name="id">IW-02</field>
+    <field name="name">iPod &amp; iPod Mini USB 2.0 Cable</field>
+    <field name="manu">Belkin</field>
+    <!-- Join -->
+    <field name="manu_id_s">belkin</field>
+    <field name="cat">electronics</field>
+    <field name="cat">connector</field>
+    <field name="features">car power adapter for iPod, white</field>
+    <field name="weight">2.0</field>
+    <field name="price">11.50</field>
+    <field name="popularity">1</field>
+    <field name="inStock">false</field>
+    <!-- San Francisco store -->
+    <field name="store">37.7752,-122.4232</field>
+    <field name="manufacturedate_dt">2006-02-14T23:55:59Z</field>
+  </doc>
+  <doc>
+    <field name="id">9885A004</field>
+    <field name="name">Canon PowerShot SD500</field>
+    <field name="manu">Canon Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">canon</field>
+    <field name="cat">electronics</field>
+    <field name="cat">camera</field>
+    <field name="features">3x zoop, 7.1 megapixel Digital ELPH</field>
+    <field name="features">movie clips up to 640x480 @30 fps</field>
+    <field name="features">2.0" TFT LCD, 118,000 pixels</field>
+    <field name="features">built in flash, red-eye reduction</field>
+    <field name="includes">32MB SD card, USB cable, AV cable, battery</field>
+    <field name="weight">6.4</field>
+    <field name="price">329.95</field>
+    <field name="popularity">7</field>
+    <field name="inStock">true</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+    <!-- Buffalo store -->
+    <field name="store">45.19614,-93.90341</field>
+  </doc>
+  <doc>
+    <field name="id">VA902B</field>
+    <field name="name">ViewSonic VA902B - flat panel display - TFT - 19"</field>
+    <field name="manu">ViewSonic Corp.</field>
+    <!-- Join -->
+    <field name="manu_id_s">viewsonic</field>
+    <field name="cat">electronics and stuff2</field>
+    <field name="features">19" TFT active matrix LCD, 8ms response time, 1280 x 1024 native resolution</field>
+    <field name="weight">190.4</field>
+    <field name="price">279.95</field>
+    <field name="popularity">6</field>
+    <field name="inStock">true</field>
+    <!-- Buffalo store -->
+    <field name="store">45.18814,-93.88541</field>
+  </doc>
+  <doc>
+    <field name="id">EN7800GTX/2DHTV/256M</field>
+    <field name="name">ASUS Extreme N7800GTX/2DHTV (256 MB)</field>
+    <!-- Denormalized -->
+    <field name="manu">ASUS Computer Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">asus</field>
+    <field name="cat">electronics</field>
+    <field name="cat">graphics card</field>
+    <field name="features">NVIDIA GeForce 7800 GTX GPU/VPU clocked at 486MHz</field>
+    <field name="features">256MB GDDR3 Memory clocked at 1.35GHz</field>
+    <field name="features">PCI Express x16</field>
+    <field name="features">Dual DVI connectors, HDTV out, video input</field>
+    <field name="features">OpenGL 2.0, DirectX 9.0</field>
+    <field name="weight">16.0</field>
+    <field name="price">479.95</field>
+    <field name="popularity">7</field>
+    <field name="store">40.7143,-74.006</field>
+    <field name="inStock">false</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z/DAY</field>
+  </doc>
+  <!-- yes, you can add more than one document at a time -->
+  <doc>
+    <field name="id">100-435805</field>
+    <field name="name">ATI Radeon X1900 XTX 512 MB PCIE Video Card</field>
+    <field name="manu">ATI Technologies</field>
+    <!-- Join -->
+    <field name="manu_id_s">ati</field>
+    <field name="cat">electronics</field>
+    <field name="cat">graphics card</field>
+    <field name="features">ATI RADEON X1900 GPU/VPU clocked at 650MHz</field>
+    <field name="features">512MB GDDR3 SDRAM clocked at 1.55GHz</field>
+    <field name="features">PCI Express x16</field>
+    <field name="features">dual DVI, HDTV, svideo, composite out</field>
+    <field name="features">OpenGL 2.0, DirectX 9.0</field>
+    <field name="weight">48.0</field>
+    <field name="price">649.99</field>
+    <field name="popularity">7</field>
+    <field name="inStock">false</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z/DAY</field>
+    <!-- NYC store -->
+    <field name="store">40.7143,-74.006</field>
+  </doc>
+  <doc>
+    <field name="id">0579B002</field>
+    <field name="name">Canon PIXMA MP500 All-In-One Photo Printer</field>
+    <field name="manu">Canon Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">canon</field>
+    <field name="cat">electronics</field>
+    <field name="cat">multifunction printer</field>
+    <field name="cat">printer</field>
+    <field name="cat">scanner</field>
+    <field name="cat">copier</field>
+    <field name="features">Multifunction ink-jet color photo printer</field>
+    <field name="features">Flatbed scanner, optical scan resolution of 1,200 x 2,400 dpi</field>
+    <field name="features">2.5" color LCD preview screen</field>
+    <field name="features">Duplex Copying</field>
+    <field name="features">Printing speed up to 29ppm black, 19ppm color</field>
+    <field name="features">Hi-Speed USB</field>
+    <field name="features">memory card: CompactFlash, Micro Drive, SmartMedia, Memory Stick, Memory Stick Pro, SD Card, and MultiMediaCard</field>
+    <field name="weight">352.0</field>
+    <field name="price">179.99</field>
+    <field name="popularity">6</field>
+    <field name="inStock">true</field>
+    <!-- Buffalo store -->
+    <field name="store">45.19214,-93.89941</field>
+  </doc>
+  <doc>
+    <field name="id">3007WFP</field>
+    <field name="name">Dell Widescreen UltraSharp 3007WFP</field>
+    <field name="manu">Dell, Inc.</field>
+    <!-- Join -->
+    <field name="manu_id_s">dell</field>
+    <field name="cat">electronics and computer1</field>
+    <field name="features">30" TFT active matrix LCD, 2560 x 1600, .25mm dot pitch, 700:1 contrast</field>
+    <field name="includes">USB cable</field>
+    <field name="weight">401.6</field>
+    <field name="price">2199.0</field>
+    <field name="popularity">6</field>
+    <field name="inStock">true</field>
+    <!-- Buffalo store -->
+    <field name="store">43.17614,-90.57341</field>
+  </doc>
+  <doc>
+    <field name="id">adata</field>
+    <field name="compName_s">A-Data Technology</field>
+    <field name="address_s">46221 Landing Parkway Fremont, CA 94538</field>
+  </doc>
+  <doc>
+    <field name="id">apple</field>
+    <field name="compName_s">Apple</field>
+    <field name="address_s">1 Infinite Way, Cupertino CA</field>
+  </doc>
+  <doc>
+    <field name="id">asus</field>
+    <field name="compName_s">ASUS Computer</field>
+    <field name="address_s">800 Corporate Way Fremont, CA 94539</field>
+  </doc>
+  <doc>
+    <field name="id">ati</field>
+    <field name="compName_s">ATI Technologies</field>
+    <field name="address_s">33 Commerce Valley Drive East Thornhill, ON L3T 7N6 Canada</field>
+  </doc>
+  <doc>
+    <field name="id">belkin</field>
+    <field name="compName_s">Belkin</field>
+    <field name="address_s">12045 E. Waterfront Drive Playa Vista, CA 90094</field>
+  </doc>
+  <doc>
+    <field name="id">canon</field>
+    <field name="compName_s">Canon, Inc.</field>
+    <field name="address_s">One Canon Plaza Lake Success, NY 11042</field>
+  </doc>
+  <doc>
+    <field name="id">corsair</field>
+    <field name="compName_s">Corsair Microsystems</field>
+    <field name="address_s">46221 Landing Parkway Fremont, CA 94538</field>
+  </doc>
+  <doc>
+    <field name="id">dell</field>
+    <field name="compName_s">Dell, Inc.</field>
+    <field name="address_s">One Dell Way Round Rock, Texas 78682</field>
+  </doc>
+  <doc>
+    <field name="id">maxtor</field>
+    <field name="compName_s">Maxtor Corporation</field>
+    <field name="address_s">920 Disc Drive Scotts Valley, CA 95066</field>
+  </doc>
+  <doc>
+    <field name="id">samsung</field>
+    <field name="compName_s">Samsung Electronics Co. Ltd.</field>
+    <field name="address_s">105 Challenger Rd. Ridgefield Park, NJ 07660-0511</field>
+  </doc>
+  <doc>
+    <field name="id">viewsonic</field>
+    <field name="compName_s">ViewSonic Corp</field>
+    <field name="address_s">381 Brea Canyon Road Walnut, CA 91789-0708</field>
+  </doc>
+  <doc>
+    <field name="id">SP2514N</field>
+    <field name="name">Samsung SpinPoint P120 SP2514N - hard drive - 250 GB - ATA-133</field>
+    <field name="manu">Samsung Electronics Co. Ltd.</field>
+    <!-- Join -->
+    <field name="manu_id_s">samsung</field>
+    <field name="cat">electronics</field>
+    <field name="cat">hard drive</field>
+    <field name="features">7200RPM, 8MB cache, IDE Ultra ATA-133</field>
+    <field name="features">NoiseGuard, SilentSeek technology, Fluid Dynamic Bearing (FDB) motor</field>
+    <field name="price">92.0</field>
+    <field name="popularity">6</field>
+    <field name="inStock">true</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+    <!-- Near Oklahoma city -->
+    <field name="store">35.0752,-97.032</field>
+  </doc>
+  <doc>
+    <field name="id">6H500F0</field>
+    <field name="name">Maxtor DiamondMax 11 - hard drive - 500 GB - SATA-300</field>
+    <field name="manu">Maxtor Corp.</field>
+    <!-- Join -->
+    <field name="manu_id_s">maxtor</field>
+    <field name="cat">electronics</field>
+    <field name="cat">hard drive</field>
+    <field name="features">SATA 3.0Gb/s, NCQ</field>
+    <field name="features">8.5ms seek</field>
+    <field name="features">16MB cache</field>
+    <field name="price">350.0</field>
+    <field name="popularity">6</field>
+    <field name="inStock">true</field>
+    <!-- Buffalo store -->
+    <field name="store">45.17614,-93.87341</field>
+    <field name="manufacturedate_dt">2006-02-13T15:26:37Z</field>
+  </doc>
+  <doc>
+    <field name="id">USD</field>
+    <field name="name">One Dollar</field>
+    <field name="manu">Bank of America</field>
+    <field name="manu_id_s">boa</field>
+    <field name="cat">currency</field>
+    <field name="features">Coins and notes</field>
+    <field name="price_c">1,USD</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">EUR</field>
+    <field name="name">One Euro</field>
+    <field name="manu">European Union</field>
+    <field name="manu_id_s">eu</field>
+    <field name="cat">currency</field>
+    <field name="features">Coins and notes</field>
+    <field name="price_c">1,EUR</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">GBP</field>
+    <field name="name">One British Pound</field>
+    <field name="manu">U.K.</field>
+    <field name="manu_id_s">uk</field>
+    <field name="cat">currency</field>
+    <field name="features">Coins and notes</field>
+    <field name="price_c">1,GBP</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">NOK</field>
+    <field name="name">One Krone</field>
+    <field name="manu">Bank of Norway</field>
+    <field name="manu_id_s">nor</field>
+    <field name="cat">currency</field>
+    <field name="features">Coins and notes</field>
+    <field name="price_c">1,NOK</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">UTF8TEST</field>
+    <field name="name">Test with some UTF-8 encoded characters</field>
+    <field name="manu">Apache Software Foundation</field>
+    <field name="cat">software</field>
+    <field name="cat">search</field>
+    <field name="features">No accents here</field>
+    <field name="features">This is an e acute: &#xE9;</field>
+    <field name="features">eaiou with circumflexes: &#xEA;&#xE2;&#xEE;&#xF4;&#xFB;</field>
+    <field name="features">eaiou with umlauts: &#xEB;&#xE4;&#xEF;&#xF6;&#xFC;</field>
+    <field name="features">tag with escaped chars: &lt;nicetag/&gt;</field>
+    <field name="features">escaped ampersand: Bonnie &amp; Clyde</field>
+    <field name="features">Outside the BMP:&#x10308; codepoint=10308, a circle with an x inside. UTF8=f0908c88 UTF16=d800 df08</field>
+    <field name="price">0.0</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">GB18030TEST</field>
+    <field name="name">Test with some GB18030 encoded characters</field>
+    <field name="features">No accents here</field>
+    <field name="features">&#xD5;&#xE2;&#xCA;&#xC7;&#xD2;&#xBB;&#xB8;&#xF6;&#xB9;&#xA6;&#xC4;&#xDC;</field>
+    <field name="features">This is a feature (translated)</field>
+    <field name="features">&#xD5;&#xE2;&#xB7;&#xDD;&#xCE;&#xC4;&#xBC;&#xFE;&#xCA;&#xC7;&#xBA;&#xDC;&#xD3;&#xD0;&#xB9;&#xE2;&#xD4;&#xF3;</field>
+    <field name="features">This document is very shiny (translated)</field>
+    <field name="price">0.0</field>
+    <field name="inStock">true</field>
+  </doc>
+  <doc>
+    <field name="id">SOLR1000</field>
+    <field name="name">Solr, the Enterprise Search Server</field>
+    <field name="manu">Apache Software Foundation</field>
+    <field name="cat">software</field>
+    <field name="cat">search</field>
+    <field name="features">Advanced Full-Text Search Capabilities using Lucene</field>
+    <field name="features">Optimized for High Volume Web Traffic</field>
+    <field name="features">Standards Based Open Interfaces - XML and HTTP</field>
+    <field name="features">Comprehensive HTML Administration Interfaces</field>
+    <field name="features">Scalability - Efficient Replication to other Solr Search Servers</field>
+    <field name="features">Flexible and Adaptable with XML configuration and Schema</field>
+    <field name="features">Good unicode support: h&#xE9;llo (hello with an accent over the e)</field>
+    <field name="price">0.0</field>
+    <field name="popularity">10</field>
+    <field name="inStock">true</field>
+    <field name="incubationdate_dt">2006-01-17T00:00:00.000Z</field>
+  </doc>
+</add>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/52998fa5/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
index b941f2d..5fd3876 100644
--- a/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/JsonRequestApiTest.java
@@ -27,11 +27,13 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
 import org.apache.solr.client.solrj.request.json.JsonQueryRequest;
+import org.apache.solr.client.solrj.request.json.TermsFacetMap;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -58,7 +60,7 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
 
     ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
     up.setParam("collection", COLLECTION_NAME);
-    up.addFile(getFile("solrj/docs2.xml"), "application/xml"); // A subset of the 'techproducts' documents
+    up.addFile(getFile("solrj/techproducts.xml"), "application/xml");
     up.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
     UpdateResponse updateResponse = up.process(cluster.getSolrClient());
     assertEquals(0, updateResponse.getStatus());
@@ -67,7 +69,7 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
   @Test
   public void testSimpleJsonQuery() throws Exception {
     SolrClient solrClient = cluster.getSolrClient();
-    final int expectedResults = 3;
+    final int expectedResults = 4;
 
     // tag::solrj-json-query-simple[]
     final JsonQueryRequest simpleQuery = new JsonQueryRequest()
@@ -116,7 +118,97 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
     // end::solrj-json-query-macro-expansion[]
 
     assertEquals(0, queryResponse.getStatus());
-    assertEquals(3, queryResponse.getResults().size());
+    assertEquals(5, queryResponse.getResults().size());
+  }
+
+  @Test
+  public void testSimpleJsonTermsFacet() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+
+    //tag::solrj-json-simple-terms-facet[]
+    final TermsFacetMap categoryFacet = new TermsFacetMap("cat").setLimit(3);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("categories", categoryFacet);
+    QueryResponse queryResponse = request.process(solrClient, COLLECTION_NAME);
+    //end::solrj-json-simple-terms-facet[]
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(32, queryResponse.getResults().getNumFound());
+    assertEquals(10, queryResponse.getResults().size());
+    assertHasFacetWithBucketValues(queryResponse.getResponse(),"categories",
+        new FacetBucket("electronics",12),
+        new FacetBucket("currency", 4),
+        new FacetBucket("memory", 3));
+  }
+
+  @Test
+  public void testTermsFacet2() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+
+    //tag::solrj-json-terms-facet2[]
+    final TermsFacetMap categoryFacet = new TermsFacetMap("cat").setLimit(5);
+    final JsonQueryRequest request = new JsonQueryRequest()
+        .setQuery("*:*")
+        .withFacet("categories", categoryFacet);
+    QueryResponse queryResponse = request.process(solrClient, COLLECTION_NAME);
+    //end::solrj-json-terms-facet2[]
+
+    assertEquals(0, queryResponse.getStatus());
+    assertEquals(32, queryResponse.getResults().getNumFound());
+    assertEquals(10, queryResponse.getResults().size());
+    assertHasFacetWithBucketValues(queryResponse.getResponse(),"categories",
+        new FacetBucket("electronics",12),
+        new FacetBucket("currency", 4),
+        new FacetBucket("memory", 3),
+        new FacetBucket("connector", 2),
+        new FacetBucket("graphics card", 2));
+  }
+
+  private class FacetBucket {
+    private final Object val;
+    private final int count;
+    FacetBucket(Object val, int count) {
+      this.val = val;
+      this.count = count;
+    }
+
+    public Object getVal() { return val; }
+    public int getCount() { return count; }
+  }
+
+  private void assertHasFacetWithBucketValues(NamedList<Object> rawResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    final NamedList<Object> facetsTopLevel = assertHasFacetResponse(rawResponse);
+    assertFacetResponseHasFacetWithBuckets(facetsTopLevel, expectedFacetName, expectedBuckets);
+  }
+
+  private NamedList<Object> assertHasFacetResponse(NamedList<Object> topLevelResponse) {
+    Object o = topLevelResponse.get("facets");
+    if (o == null) fail("Response has no top-level 'facets' property as expected");
+    if (!(o instanceof NamedList)) fail("Response has a top-level 'facets' property, but it is not a NamedList");
+
+    return (NamedList<Object>) o;
+  }
+
+  private void assertFacetResponseHasFacetWithBuckets(NamedList<Object> facetResponse, String expectedFacetName, FacetBucket... expectedBuckets) {
+    Object o = facetResponse.get(expectedFacetName);
+    if (o == null) fail("Response has no top-level facet named '" + expectedFacetName + "'");
+    if (!(o instanceof NamedList)) fail("Response has a property for the expected facet '" + expectedFacetName + "' property, but it is not a NamedList");
+
+    final NamedList<Object> expectedFacetTopLevel = (NamedList<Object>) o;
+    o = expectedFacetTopLevel.get("buckets");
+    if (o == null) fail("Response has no 'buckets' property under 'facets'");
+    if (!(o instanceof List)) fail("Response has no 'buckets' property containing actual facet information.");
+
+    final List<NamedList> bucketList = (List<NamedList>) o;
+    assertEquals("Expected " + expectedBuckets.length + " buckets, but found " + bucketList.size(),
+        expectedBuckets.length, bucketList.size());
+    for (int i = 0; i < expectedBuckets.length; i++) {
+      final FacetBucket expectedBucket = expectedBuckets[i];
+      final NamedList<Object> actualBucket = bucketList.get(i);
+      assertEquals(expectedBucket.getVal(), actualBucket.get("val"));
+      assertEquals(expectedBucket.getCount(), actualBucket.get("count"));
+    }
   }
 
 }


[28/50] [abbrv] lucene-solr:jira/http2: SOLR-12964: Use DocValuesIterator.advanceExact() instead of the advance()/docID() pattern

Posted by da...@apache.org.
SOLR-12964: Use DocValuesIterator.advanceExact() instead of the advance()/docID() pattern


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

Branch: refs/heads/jira/http2
Commit: 243a8a668a613835e3c114f6e5c31a098699840e
Parents: 1b084db
Author: Tim Underwood <ti...@gmail.com>
Authored: Thu Nov 8 14:34:17 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Nov 8 14:34:17 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                                | 3 +++
 .../apache/solr/search/facet/FacetFieldProcessorByHashDV.java   | 5 +----
 .../org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java | 5 +----
 3 files changed, 5 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/243a8a66/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2f04c03..b8f9f18 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -293,6 +293,9 @@ Improvements
 * LUCENE-8557: Some internal LeafReader.getFieldInfos implementations were being re-computed on-demand instead of
   once up front leading to some slowdowns in places like JSON Facets and field collapsing. (Tim Underwood, David Smiley)
 
+* SOLR-12964: Json Facets: use DocValuesIterator advanceExact() instead of advance() in FacetFieldProcessorByHashDV and
+  UniqueSinglevaluedSlotAcc. (Tim Underwood)
+
 ==================  7.5.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/243a8a66/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
index 5070a97..38feddd 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByHashDV.java
@@ -361,10 +361,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
 
           @Override
           public void collect(int segDoc) throws IOException {
-            if (segDoc > docValues.docID()) {
-              docValues.advance(segDoc);
-            }
-            if (segDoc == docValues.docID()) {
+            if (docValues.advanceExact(segDoc)) {
               long val = toGlobal.get(docValues.ordValue());
               collectValFirstPhase(segDoc, val);
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/243a8a66/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
index ed51c5b..6057dd0 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueSinglevaluedSlotAcc.java
@@ -75,10 +75,7 @@ class UniqueSinglevaluedSlotAcc extends UniqueSlotAcc {
 
   @Override
   public void collect(int doc, int slotNum, IntFunction<SlotContext> slotContext) throws IOException {
-    if (doc > subDv.docID()) {
-      subDv.advance(doc);
-    }
-    if (doc == subDv.docID()) {
+    if (subDv.advanceExact(doc)) {
       int segOrd = subDv.ordValue();
       int ord = toGlobal==null ? segOrd : (int)toGlobal.get(segOrd);
 


[34/50] [abbrv] lucene-solr:jira/http2: LUCENE-8556: Use latitude and longitude instead of encoding values to check if triangle is ear when using morton optimisation

Posted by da...@apache.org.
LUCENE-8556: Use latitude and longitude instead of encoding values to check if triangle is ear when using morton optimisation


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

Branch: refs/heads/jira/http2
Commit: 74e3ff509e85982a5529ca99c8e3e9ec2f96770a
Parents: fbb9872
Author: iverase <iv...@apache.org>
Authored: Fri Nov 9 07:22:10 2018 +0100
Committer: iverase <iv...@apache.org>
Committed: Fri Nov 9 07:22:10 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  5 ++++-
 .../java/org/apache/lucene/geo/Tessellator.java | 23 +++++++-------------
 .../org/apache/lucene/geo/TestTessellator.java  | 14 ++++++++++++
 3 files changed, 26 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e3ff50/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 586ad20..c3ff482 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -227,7 +227,10 @@ Bug fixes:
   shape tessellation. (Ignacio Vera)
 
 * LUCENE-8559: Fix bug where polygon edges were skipped when checking for intersections.
-  (Ignacio Vera)   
+  (Ignacio Vera)
+
+* LUCENE-8556: Use latitude and longitude instead of encoding values to check if triangle is ear
+  when using morton optimisation. (Ignacio Vera)
 
 New Features
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e3ff50/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java b/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
index 8fa685c..1fa0443 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
@@ -353,13 +353,6 @@ final public class Tessellator {
 
   /** Uses morton code for speed to determine whether or a polygon node forms a valid ear w/ adjacent nodes */
   private static final boolean mortonIsEar(final Node ear) {
-    double ax = ear.previous.x;
-    double ay = ear.previous.y;
-    double bx = ear.x;
-    double by = ear.y;
-    double cx = ear.next.x;
-    double cy = ear.next.y;
-
     // triangle bbox (flip the bits so negative encoded values are < positive encoded values)
     int minTX = StrictMath.min(StrictMath.min(ear.previous.x, ear.x), ear.next.x) ^ 0x80000000;
     int minTY = StrictMath.min(StrictMath.min(ear.previous.y, ear.y), ear.next.y) ^ 0x80000000;
@@ -378,21 +371,21 @@ final public class Tessellator {
     while (p != null && Long.compareUnsigned(p.morton, minZ) >= 0
         && n != null && Long.compareUnsigned(n.morton, maxZ) <= 0) {
       if (p.idx != ear.previous.idx && p.idx != ear.next.idx &&
-          pointInEar(p.x, p.y, ax, ay, bx, by, cx, cy) &&
-          area(p.previous.x, p.previous.y, p.x, p.y, p.next.x, p.next.y) >= 0) return false;
+          pointInEar(p.getX(), p.getY(), ear.previous.getX(), ear.previous.getY(), ear.getX(), ear.getY(), ear.next.getX(), ear.next.getY()) &&
+          area(p.previous.getX(), p.previous.getY(), p.getX(), p.getY(), p.next.getX(), p.next.getY()) >= 0) return false;
       p = p.previousZ;
 
       if (n.idx != ear.previous.idx && n.idx != ear.next.idx &&
-          pointInEar(n.x, n.y, ax, ay, bx, by, cx, cy) &&
-          area(n.previous.x, n.previous.y, n.x, n.y, n.next.x, n.next.y) >= 0) return false;
+          pointInEar(n.getX(), n.getY(), ear.previous.getX(), ear.previous.getY(), ear.getX(), ear.getY(), ear.next.getX(), ear.next.getY()) &&
+          area(n.previous.getX(), n.previous.getY(), n.getX(), n.getY(), n.next.getX(), n.next.getY()) >= 0) return false;
       n = n.nextZ;
     }
 
     // first look for points inside the triangle in decreasing z-order
     while (p != null && Long.compareUnsigned(p.morton, minZ) >= 0) {
       if (p.idx != ear.previous.idx && p.idx != ear.next.idx
-            && pointInEar(p.x, p.y, ax, ay, bx, by, cx, cy)
-            && area(p.previous.x, p.previous.y, p.x, p.y, p.next.x, p.next.y) >= 0) {
+            && pointInEar(p.getX(), p.getY(), ear.previous.getX(), ear.previous.getY(), ear.getX(), ear.getY(), ear.next.getX(), ear.next.getY())
+            && area(p.previous.getX(), p.previous.getY(), p.getX(), p.getY(), p.next.getX(), p.next.getY()) >= 0) {
           return false;
         }
       p = p.previousZ;
@@ -401,8 +394,8 @@ final public class Tessellator {
     while (n != null &&
         Long.compareUnsigned(n.morton, maxZ) <= 0) {
         if (n.idx != ear.previous.idx && n.idx != ear.next.idx
-            && pointInEar(n.x, n.y, ax, ay, bx, by, cx, cy)
-            && area(n.previous.x, n.previous.y, n.x, n.y, n.next.x, n.next.y) >= 0) {
+            && pointInEar(n.getX(), n.getY(), ear.previous.getX(), ear.previous.getY(), ear.getX(), ear.getY(), ear.next.getX(), ear.next.getY())
+            && area(n.previous.getX(), n.previous.getY(), n.getX(), n.getY(), n.next.getX(), n.next.getY()) >= 0) {
           return false;
         }
       n = n.nextZ;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e3ff50/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java b/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
index 7904725..d5b7e66 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
@@ -103,4 +103,18 @@ public class TestTessellator extends LuceneTestCase {
     Polygon polygon = (Polygon)SimpleWKTShapeParser.parse(wkt);
     assertTrue(Tessellator.tessellate(polygon).size() > 0);
   }
+
+  public void testLUCENE8556()  throws Exception {
+    String wkt ="POLYGON((-111.4765 68.321,-111.47625 68.32225,-111.4765 68.3225,-111.478 68.3225,-111.47825 68.32275,-111.479 68.32275,-111.47975 68.32325,-111.48125 68.324,-111.4815 68.32375,-111.48175 68.32375," +
+        "-111.48225 68.32425,-111.48275 68.32425,-111.483 68.324,-111.4845 68.324,-111.48475 68.32425,-111.4845 68.32475,-111.48425 68.3245,-111.483 68.325,-111.4835 68.325,-111.48425 68.3255,-111.48525 68.3255,-111.4855 68.32575," +
+        "-111.4855 68.32525,-111.486 68.32475,-111.48725 68.3245,-111.4875 68.32475,-111.48725 68.325,-111.487 68.325,-111.4865 68.32525,-111.487 68.32575,-111.486465 68.326385,-111.486 68.326,-111.48575 68.32625," +
+        "-111.48525 68.32625,-111.485 68.326,-111.48375 68.326,-111.48225 68.3265,-111.483 68.3265,-111.48325 68.32675,-111.4835 68.3265,-111.48675 68.3265,-111.487 68.32675,-111.48675 68.32725,-111.4865 68.327," +
+        "-111.48375 68.32775,-111.485 68.32775,-111.48525 68.3275,-111.4855 68.3275,-111.486 68.32775,-111.48625 68.3275,-111.48675 68.3275,-111.48725 68.327,-111.48775 68.327,-111.4875 68.32625,-111.488 68.32625," +
+        "-111.48825 68.32675,-111.49025 68.327,-111.49025 68.32675,-111.4905 68.3265,-111.49075 68.3265,-111.49125 68.326,-111.492 68.32575,-111.4945 68.32575,-111.49475 68.3255,-111.49525 68.3255,-111.4955 68.32525,-111.49625 68.32525," +
+        "-111.4965 68.325,-111.49775 68.32425,-111.498 68.3245,-111.4985 68.3245,-111.49875 68.32425,-111.49925 68.32425,-111.5005 68.324,-111.50075 68.32375,-111.501 68.32375,-111.501 68.323,-111.5015 68.323,-111.50175 68.32325,-111.5015 68.3235," +
+        "-111.5025 68.32375,-111.50275 68.3235,-111.504 68.32375,-111.50425 68.3235,-111.50525 68.32325,-111.5055 68.3235,-111.506 68.3235,-111.50625 68.32325,-111.5065 68.3225,-111.5075 68.3225,-111.50775 68.32275,-111.50825 68.32275," +
+        "-111.5085 68.3225,-111.50875 68.3225,-111.509 68.32275,-111.5125 68.32275,-111.51325 68.32225,-111.4765 68.321))";
+    Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt);
+    assertTrue(Tessellator.tessellate(polygon).size() > 0);
+  }
 }
\ No newline at end of file


[50/50] [abbrv] lucene-solr:jira/http2: Merge with master

Posted by da...@apache.org.
Merge with master


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

Branch: refs/heads/jira/http2
Commit: 6dd3ef2b85c47ea1c6a9a1a1fd450c3b2a136a9b
Parents: 66da526 f357c06
Author: Cao Manh Dat <da...@apache.org>
Authored: Mon Nov 12 11:54:54 2018 +0000
Committer: Cao Manh Dat <da...@apache.org>
Committed: Mon Nov 12 11:54:54 2018 +0000

----------------------------------------------------------------------
 dev-tools/scripts/jenkins.build.ref.guide.sh    |   5 +-
 lucene/CHANGES.txt                              |  18 +
 .../lucene/analysis/ar/ArabicAnalyzer.java      |   3 +-
 .../ar/ArabicNormalizationFilterFactory.java    |   2 +
 .../analysis/ar/ArabicStemFilterFactory.java    |   2 +
 .../lucene/analysis/bg/BulgarianAnalyzer.java   |   2 +
 .../lucene/analysis/bn/BengaliAnalyzer.java     |   2 +
 .../lucene/analysis/br/BrazilianAnalyzer.java   |   2 +
 .../analysis/br/BrazilianStemFilterFactory.java |   2 +
 .../lucene/analysis/ca/CatalanAnalyzer.java     |   2 +
 .../charfilter/HTMLStripCharFilterFactory.java  |   2 +
 .../apache/lucene/analysis/cjk/CJKAnalyzer.java |   2 +
 .../lucene/analysis/ckb/SoraniAnalyzer.java     |   2 +
 .../commongrams/CommonGramsFilterFactory.java   |   2 +
 .../CommonGramsQueryFilterFactory.java          |   2 +
 ...ictionaryCompoundWordTokenFilterFactory.java |   2 +
 .../lucene/analysis/core/KeywordAnalyzer.java   |   2 +
 .../analysis/core/KeywordTokenizerFactory.java  |   2 +
 .../analysis/core/LetterTokenizerFactory.java   |   2 +
 .../analysis/core/LowerCaseFilterFactory.java   |   2 +
 .../lucene/analysis/core/SimpleAnalyzer.java    |   2 +
 .../lucene/analysis/core/StopAnalyzer.java      |   2 +
 .../lucene/analysis/core/StopFilterFactory.java |   2 +
 .../core/UnicodeWhitespaceAnalyzer.java         |   2 +
 .../analysis/core/WhitespaceAnalyzer.java       |   2 +
 .../core/WhitespaceTokenizerFactory.java        |   2 +
 .../lucene/analysis/custom/CustomAnalyzer.java  |   2 +
 .../lucene/analysis/cz/CzechAnalyzer.java       |   2 +
 .../lucene/analysis/da/DanishAnalyzer.java      |   2 +
 .../lucene/analysis/de/GermanAnalyzer.java      |   2 +
 .../analysis/de/GermanStemFilterFactory.java    |   2 +
 .../lucene/analysis/el/GreekAnalyzer.java       |   2 +
 .../el/GreekLowerCaseFilterFactory.java         |   2 +
 .../lucene/analysis/en/EnglishAnalyzer.java     |   2 +
 .../analysis/en/PorterStemFilterFactory.java    |   2 +
 .../lucene/analysis/es/SpanishAnalyzer.java     |   2 +
 .../lucene/analysis/eu/BasqueAnalyzer.java      |   2 +
 .../lucene/analysis/fa/PersianAnalyzer.java     |   2 +
 .../analysis/fa/PersianCharFilterFactory.java   |   2 +
 .../fa/PersianNormalizationFilterFactory.java   |   2 +
 .../lucene/analysis/fi/FinnishAnalyzer.java     |   2 +
 .../lucene/analysis/fr/FrenchAnalyzer.java      |   2 +
 .../lucene/analysis/ga/IrishAnalyzer.java       |   2 +
 .../lucene/analysis/gl/GalicianAnalyzer.java    |   2 +
 .../lucene/analysis/hi/HindiAnalyzer.java       |   2 +
 .../lucene/analysis/hu/HungarianAnalyzer.java   |   2 +
 .../lucene/analysis/hy/ArmenianAnalyzer.java    |   2 +
 .../lucene/analysis/id/IndonesianAnalyzer.java  |   2 +
 .../lucene/analysis/it/ItalianAnalyzer.java     |   2 +
 .../lucene/analysis/lt/LithuanianAnalyzer.java  |   2 +
 .../lucene/analysis/lv/LatvianAnalyzer.java     |   2 +
 .../ASCIIFoldingFilterFactory.java              |   2 +
 .../ConditionalTokenFilterFactory.java          |   2 +
 .../HyphenatedWordsFilterFactory.java           |   2 +
 .../miscellaneous/KeepWordFilterFactory.java    |   2 +
 .../miscellaneous/LengthFilterFactory.java      |   2 +
 .../miscellaneous/LimitTokenCountAnalyzer.java  |   2 +
 .../miscellaneous/PerFieldAnalyzerWrapper.java  |   2 +
 .../ProtectedTermFilterFactory.java             |   2 +
 .../RemoveDuplicatesTokenFilterFactory.java     |   2 +
 .../miscellaneous/TrimFilterFactory.java        |   2 +
 .../TypeAsSynonymFilterFactory.java             |   2 +
 .../WordDelimiterFilterFactory.java             |   2 +
 .../analysis/ngram/EdgeNGramFilterFactory.java  |   2 +
 .../ngram/EdgeNGramTokenizerFactory.java        |   2 +
 .../analysis/ngram/NGramFilterFactory.java      |   2 +
 .../analysis/ngram/NGramTokenizerFactory.java   |   2 +
 .../lucene/analysis/nl/DutchAnalyzer.java       |   2 +
 .../lucene/analysis/no/NorwegianAnalyzer.java   |   2 +
 .../path/PathHierarchyTokenizerFactory.java     |   2 +
 .../pattern/PatternReplaceFilterFactory.java    |   2 +
 .../SimplePatternSplitTokenizerFactory.java     |   2 +
 .../pattern/SimplePatternTokenizerFactory.java  |   2 +
 .../DelimitedPayloadTokenFilterFactory.java     |   2 +
 .../NumericPayloadTokenFilterFactory.java       |   2 +
 .../TokenOffsetPayloadTokenFilterFactory.java   |   2 +
 .../TypeAsPayloadTokenFilterFactory.java        |   2 +
 .../lucene/analysis/pt/PortugueseAnalyzer.java  |   2 +
 .../query/QueryAutoStopWordAnalyzer.java        |   2 +
 .../lucene/analysis/ro/RomanianAnalyzer.java    |   2 +
 .../lucene/analysis/ru/RussianAnalyzer.java     |   2 +
 .../shingle/FixedShingleFilterFactory.java      |   2 +
 .../shingle/ShingleAnalyzerWrapper.java         |   2 +
 .../analysis/shingle/ShingleFilterFactory.java  |   2 +
 .../snowball/SnowballPorterFilterFactory.java   |   2 +
 .../analysis/standard/ClassicAnalyzer.java      |   2 +
 .../standard/ClassicTokenizerFactory.java       |   2 +
 .../standard/StandardTokenizerFactory.java      |   2 +
 .../standard/UAX29URLEmailAnalyzer.java         |   2 +
 .../standard/UAX29URLEmailTokenizerFactory.java |   2 +
 .../lucene/analysis/sv/SwedishAnalyzer.java     |   2 +
 .../analysis/synonym/SynonymFilterFactory.java  |   4 +-
 .../apache/lucene/analysis/th/ThaiAnalyzer.java |   2 +
 .../analysis/th/ThaiTokenizerFactory.java       |   2 +
 .../lucene/analysis/tr/TurkishAnalyzer.java     |   2 +
 .../lucene/analysis/util/CharFilterFactory.java |   2 +
 .../analysis/util/ElisionFilterFactory.java     |   2 +
 .../analysis/util/TokenFilterFactory.java       |   2 +
 .../lucene/analysis/util/TokenizerFactory.java  |   2 +
 .../wikipedia/WikipediaTokenizerFactory.java    |   2 +
 .../lucene/collation/CollationKeyAnalyzer.java  |   2 +
 .../icu/ICUNormalizer2CharFilterFactory.java    |   2 +
 .../icu/segmentation/ICUTokenizerFactory.java   |   2 +
 .../collation/ICUCollationKeyAnalyzer.java      |   2 +
 .../lucene/analysis/ja/JapaneseAnalyzer.java    |   2 +
 .../JapaneseIterationMarkCharFilterFactory.java |   2 +
 .../analysis/ja/JapaneseTokenizerFactory.java   |   2 +
 .../analysis/morfologik/MorfologikAnalyzer.java |   2 +
 .../uk/UkrainianMorfologikAnalyzer.java         |   2 +
 .../lucene/analysis/ko/KoreanAnalyzer.java      |   2 +
 .../ko/KoreanPartOfSpeechStopFilterFactory.java |   2 +
 .../ko/KoreanReadingFormFilterFactory.java      |   2 +
 .../analysis/ko/KoreanTokenizerFactory.java     |   2 +
 .../phonetic/DoubleMetaphoneFilterFactory.java  |   2 +
 .../phonetic/PhoneticFilterFactory.java         |   2 +
 .../cn/smart/HMMChineseTokenizerFactory.java    |   2 +
 .../analysis/cn/smart/SmartChineseAnalyzer.java |   2 +
 .../lucene/analysis/pl/PolishAnalyzer.java      |   2 +
 .../org/apache/lucene/analysis/Analyzer.java    |   2 +
 .../apache/lucene/analysis/AnalyzerWrapper.java |   2 +
 .../analysis/DelegatingAnalyzerWrapper.java     |   4 +-
 .../lucene/analysis/StopwordAnalyzerBase.java   |   4 +-
 .../analysis/standard/StandardAnalyzer.java     |   2 +
 .../codecs/perfield/PerFieldMergeState.java     |   2 +-
 .../org/apache/lucene/index/FieldInfos.java     |   4 +
 .../org/apache/lucene/index/LeafReader.java     |   4 +
 .../lucene/store/ByteArrayIndexInput.java       | 208 ------
 .../lucene/store/ByteBuffersDirectory.java      |  17 +-
 .../java/org/apache/lucene/util/Version.java    |   7 +
 .../apache/lucene/index/TestPendingDeletes.java |   2 +-
 .../lucene/index/TestTermVectorsReader.java     |   2 +-
 .../apache/lucene/index/memory/MemoryIndex.java |  15 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |  16 +-
 .../document/LatLonShapeBoundingBoxQuery.java   |   9 +-
 .../java/org/apache/lucene/geo/Tessellator.java |  34 +-
 .../document/BaseLatLonShapeTestCase.java       |   8 +-
 .../document/TestLatLonPointShapeQueries.java   |   9 +-
 .../apache/lucene/document/TestLatLonShape.java |  22 +
 .../org/apache/lucene/geo/TestTessellator.java  |  21 +
 .../suggest/document/CompletionAnalyzer.java    |   2 +
 .../org/apache/lucene/search/QueryUtils.java    |   3 +-
 .../lucene/store/BaseDirectoryTestCase.java     |  13 +-
 .../java/org/apache/lucene/util/TestUtil.java   |   5 +
 lucene/tools/build.xml                          |   4 +-
 solr/CHANGES.txt                                | 105 ++-
 solr/NOTICE.txt                                 |   4 +
 .../ExtractingRequestHandlerTest.java           |   2 +-
 ...OpenNLPLangDetectUpdateProcessorFactory.java |   2 +
 .../solr/ltr/model/DefaultWrapperModel.java     |   5 +
 .../apache/solr/ltr/model/LTRScoringModel.java  |  33 +-
 .../org/apache/solr/ltr/model/LinearModel.java  |   5 +
 .../ltr/model/MultipleAdditiveTreesModel.java   |  11 +
 .../solr/ltr/model/NeuralNetworkModel.java      |   5 +
 .../apache/solr/ltr/model/TestAdapterModel.java |  10 +
 .../analysis/ReversedWildcardFilterFactory.java |   1 +
 .../org/apache/solr/analysis/SolrAnalyzer.java  |   1 +
 .../apache/solr/analysis/TokenizerChain.java    |   2 +
 .../apache/solr/cloud/ReplicateFromLeader.java  |   3 +
 .../solr/handler/admin/ClusterStatus.java       |   4 +-
 .../solr/handler/admin/LukeRequestHandler.java  |   2 +
 .../solr/handler/component/ExpandComponent.java |  77 +--
 .../solr/index/SlowCompositeReaderWrapper.java  |   9 +-
 .../org/apache/solr/request/SimpleFacets.java   |   7 +
 .../org/apache/solr/schema/CurrencyField.java   |   8 +
 .../org/apache/solr/schema/FieldProperties.java |   4 +-
 .../java/org/apache/solr/schema/FieldType.java  |  13 +-
 .../org/apache/solr/schema/IndexSchema.java     |  16 +-
 .../org/apache/solr/schema/SchemaField.java     |  22 +-
 .../schema/SpatialPointVectorFieldType.java     |   8 +-
 .../solr/search/CollapsingQParserPlugin.java    |  99 +--
 .../solr/search/ExtendedDismaxQParser.java      |  12 +-
 .../apache/solr/search/SolrIndexSearcher.java   |   7 +-
 .../apache/solr/search/facet/FacetField.java    |   7 +-
 .../facet/FacetFieldProcessorByArrayUIF.java    |   6 +-
 .../facet/FacetFieldProcessorByHashDV.java      |   7 +-
 .../apache/solr/search/facet/FacetHeatmap.java  | 103 +--
 .../apache/solr/search/facet/FacetRequest.java  |   2 +-
 .../search/facet/UniqueSinglevaluedSlotAcc.java |   5 +-
 .../java/org/apache/solr/update/PeerSync.java   | 158 ++---
 .../apache/solr/update/PeerSyncWithLeader.java  |  59 +-
 .../SignatureUpdateProcessorFactory.java        |   3 +
 .../org/apache/solr/util/TestInjection.java     |  12 +-
 .../bad-schema-not-indexed-but-uninvertible.xml |  35 +
 .../collection1/conf/multiword-synonyms.txt     |   4 +-
 .../solr/collection1/conf/schema-behavior.xml   |  13 +
 .../conf/schema-multiword-synonyms.xml          |   2 +
 .../test-files/solr/collection1/conf/schema.xml |  14 +-
 .../solr/collection1/conf/schema11.xml          |   3 +
 .../solr/collection1/conf/schema_latest.xml     |  12 +
 .../org/apache/solr/BasicFunctionalityTest.java |  39 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |   2 +-
 .../solr/cloud/RecoveryAfterSoftCommitTest.java |   3 +
 .../apache/solr/cloud/TestCloudRecovery2.java   | 143 ++++
 .../apache/solr/request/SimpleFacetsTest.java   |  84 +++
 .../apache/solr/request/TestFacetMethods.java   | 312 ++++-----
 .../solr/rest/schema/TestBulkSchemaAPI.java     |  11 +-
 .../solr/rest/schema/TestFieldResource.java     |   4 +-
 .../solr/rest/schema/TestFieldTypeResource.java |   4 +-
 .../apache/solr/schema/BadIndexSchemaTest.java  |   1 +
 .../SchemaVersionSpecificBehaviorTest.java      |   5 +
 .../solr/search/TestCollapseQParserPlugin.java  |  28 +-
 .../test/org/apache/solr/search/TestDocSet.java |   3 +-
 .../solr/search/TestMultiWordSynonyms.java      | 165 ++++-
 .../solr/search/facet/TestJsonFacets.java       |  71 ++
 .../org/apache/solr/update/PeerSyncTest.java    |  50 +-
 .../solr/update/PeerSyncWithLeaderTest.java     |  18 +
 solr/solr-ref-guide/README.adoc                 |   3 +
 solr/solr-ref-guide/src/_config.yml.template    |   3 +-
 .../src/_includes/google_analytics.html         |   6 -
 solr/solr-ref-guide/src/_includes/head.html     |   2 +-
 .../src/_includes/head_print.html               |   6 +-
 solr/solr-ref-guide/src/_includes/toc.html      |   2 +-
 solr/solr-ref-guide/src/_layouts/default.html   |   9 +-
 solr/solr-ref-guide/src/_layouts/page.html      |  45 +-
 .../src/_templates/_attribution.html.slim       |   4 +
 .../src/_templates/_footer.html.slim            |   8 +
 .../src/_templates/_footnotes.html.slim         |  11 +
 .../src/_templates/_hdlist.html.slim            |  20 +
 .../src/_templates/_header.html.slim            |  27 +
 .../src/_templates/_qanda.html.slim             |  12 +
 .../src/_templates/_toc.html.slim               |   4 +
 .../src/_templates/colist.html.slim             |   4 +
 .../src/_templates/dlist.html.slim              |  13 +
 .../src/_templates/document.html.slim           |  30 +
 .../src/_templates/example.html.slim            |   3 +
 solr/solr-ref-guide/src/_templates/helpers.rb   | 670 +++++++++++++++++++
 .../src/_templates/image.html.slim              |   3 +
 .../src/_templates/inline_anchor.html.slim      |  12 +
 .../src/_templates/inline_break.html.slim       |   2 +
 .../src/_templates/inline_button.html.slim      |   1 +
 .../src/_templates/inline_callout.html.slim     |   1 +
 .../src/_templates/inline_footnote.html.slim    |   9 +
 .../src/_templates/inline_image.html.slim       |  10 +
 .../src/_templates/inline_indexterm.html.slim   |   2 +
 .../src/_templates/inline_kbd.html.slim         |   7 +
 .../src/_templates/inline_menu.html.slim        |  15 +
 .../src/_templates/inline_quoted.html.slim      |  29 +
 .../src/_templates/listing.html.slim            |  15 +
 .../src/_templates/literal.html.slim            |   2 +
 .../src/_templates/olist.html.slim              |   4 +
 .../src/_templates/open.html.slim               |   7 +
 .../src/_templates/paragraph.html.slim          |   6 +
 .../src/_templates/pass.html.slim               |   1 +
 .../src/_templates/preamble.html.slim           |   4 +
 .../src/_templates/quote.html.slim              |   6 +
 .../src/_templates/section.html.slim            |  13 +
 .../src/_templates/sidebar.html.slim            |   4 +
 .../src/_templates/table.html.slim              |  39 ++
 .../src/_templates/thematic_break.html.slim     |   1 +
 .../solr-ref-guide/src/_templates/toc.html.slim |  11 +
 .../src/_templates/ulist.html.slim              |  11 +
 .../src/collapse-and-expand-results.adoc        |   2 +-
 solr/solr-ref-guide/src/css/customstyles.css    |  12 +-
 solr/solr-ref-guide/src/css/ref-guide.css       | 146 ++--
 solr/solr-ref-guide/src/defining-fields.adoc    |   1 +
 .../field-type-definitions-and-properties.adoc  |   1 +
 solr/solr-ref-guide/src/json-facet-api.adoc     |  52 ++
 solr/solr-ref-guide/src/json-request-api.adoc   |  69 +-
 solr/solr-ref-guide/src/solr-tutorial.adoc      |  16 +
 .../src/stream-source-reference.adoc            |  14 +-
 .../src/the-extended-dismax-query-parser.adoc   |  15 +-
 solr/solr-ref-guide/src/the-tagger-handler.adoc |  16 +
 .../solr-ref-guide/src/time-routed-aliases.adoc |  16 +
 solr/solr-ref-guide/src/vector-math.adoc        |  35 +
 solr/solr-ref-guide/src/vectorization.adoc      |  80 +++
 .../tools/CheckLinksAndAnchors.java             |  28 +-
 .../cloud/autoscaling/AddReplicaSuggester.java  |   2 +-
 .../client/solrj/cloud/autoscaling/Clause.java  |   3 +
 .../cloud/autoscaling/MoveReplicaSuggester.java |   2 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java   |  16 +-
 .../solrj/cloud/autoscaling/RangeVal.java       |   1 +
 .../solrj/cloud/autoscaling/Suggester.java      |  24 +-
 .../solrj/cloud/autoscaling/Suggestion.java     |   2 +-
 .../solrj/impl/HttpClusterStateProvider.java    |  18 +-
 .../solrj/impl/SolrClientNodeStateProvider.java |   8 +-
 .../org/apache/solr/client/solrj/io/Lang.java   |   4 +
 .../solrj/io/eval/LeftShiftEvaluator.java       |  53 ++
 .../client/solrj/io/eval/PivotEvaluator.java    | 111 +++
 .../solrj/io/eval/RightShiftEvaluator.java      |  54 ++
 .../client/solrj/io/stream/FacetStream.java     | 351 ++++++++--
 .../request/json/DirectJsonQueryRequest.java    |  50 ++
 .../client/solrj/request/json/DomainMap.java    | 139 ++++
 .../solrj/request/json/HeatmapFacetMap.java     | 137 ++++
 .../client/solrj/request/json/JsonFacetMap.java |  62 ++
 .../solrj/request/json/JsonQueryRequest.java    | 420 ++++++++++++
 .../solrj/request/json/QueryFacetMap.java       |  39 ++
 .../solrj/request/json/RangeFacetMap.java       | 105 +++
 .../solrj/request/json/TermsFacetMap.java       | 204 ++++++
 .../client/solrj/request/json/package-info.java |  21 +
 .../solr/client/solrj/util/ClientUtils.java     |   3 +-
 .../org/apache/solr/common/luke/FieldFlag.java  |   1 +
 .../autoscaling/testUnresolvedSuggestion.json   | 212 ++++++
 .../solrj/src/test-files/solrj/techproducts.xml | 421 ++++++++++++
 .../ref_guide_examples/JsonRequestApiTest.java  | 214 ++++++
 .../solrj/cloud/autoscaling/TestPolicy2.java    |  47 +-
 .../apache/solr/client/solrj/io/TestLang.java   |   2 +-
 .../solrj/io/stream/MathExpressionTest.java     |  85 +++
 .../solrj/io/stream/StreamExpressionTest.java   | 141 ++++
 .../stream/StreamExpressionToExpessionTest.java | 173 ++++-
 ...JsonQueryRequestFacetingIntegrationTest.java | 615 +++++++++++++++++
 .../solrj/request/json/DomainMapTest.java       | 177 +++++
 .../solrj/request/json/HeatmapFacetMapTest.java | 130 ++++
 ...JsonQueryRequestFacetingIntegrationTest.java | 530 +++++++++++++++
 .../json/JsonQueryRequestIntegrationTest.java   | 306 +++++++++
 .../request/json/JsonQueryRequestUnitTest.java  | 329 +++++++++
 .../solrj/request/json/QueryFacetMapTest.java   |  45 ++
 .../solrj/request/json/RangeFacetMapTest.java   |  84 +++
 .../solrj/request/json/TermsFacetMapTest.java   | 189 ++++++
 .../angular/controllers/cluster-suggestions.js  |   4 +-
 .../webapp/web/js/angular/controllers/schema.js |   3 +-
 .../web/partials/cluster_suggestions.html       |  10 +-
 solr/webapp/web/partials/schema.html            |   7 +
 312 files changed, 8631 insertions(+), 1067 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6dd3ef2b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --cc solr/CHANGES.txt
index e8e4213,8a0e36b..9a9251b
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@@ -63,27 -56,7 +63,26 @@@ Upgrade Note
    "date.formats" configuration.  To ensure date strings are properly parsed, use ParseDateFieldUpdateProcessorFactory
    (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein)
  
 -* SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true.  If there are unforseen highlight problems,
 +* SOLR-12639: Solr is now using HTTP/2 (Http2SolrClient) for most of internal requests
 +  (UpdateShardHandler, HttpShardHandler).
 +
 +* SOLR-12643: Since Http2SolrClient does not support exposing connections related metrics. These metrics are no longer
 +  available 'QUERY.httpShardHandler.{availableConnections, leasedConnections, maxConnections, pendingConnections}',
 +  'UPDATE.updateShardHandler.{availableConnections, leasedConnections, maxConnections, pendingConnections}'
-   (Cao Manh Dat)
 +
 +* SOLR-12605: UpdateShardHandler's updateOnlyClient is now a Http2SolrClient (previous HttpSolrClient). This new
-   client does not support 'maxConnections' parameter (Cao Manh Dat). StreamingsSolrClient which is removed and
++  client does not support 'maxConnections','maxConnectionsPerHost' parameters (Cao Manh Dat). StreamingsSolrClient which is removed and
 +  ConcurrentUpdateSolrClient are no longer used to send updates, updates will be sent asynchronously (without-batching)
 +  by Http2SolrClient.
 +
 +* SOLR-12640: HttpShardHandlerFactory's defaultClient is now a Http2SolrClient (previous HttpSolrClient). This new
-   client does not support 'maxConnections' parameter (Cao Manh Dat). LBHttpSolrClient.Req and LBHttpSolrClient.Rsp
++  client does not support 'maxConnections','maxConnectionsPerHost' parameters. LBHttpSolrClient.Req and LBHttpSolrClient.Rsp
 +  are marked as deprecated, uses LBSolrClient.Req and LBSolrClient.Rsp instead.
 +
 +* SOLR-12644: Custom AuthenticationPlugin must provide its own setup for Http2SolrClient through
-   implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated. (Cao Manh Dat)
++  implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated.
 +
- * SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true.  If there are unforseen highlight problems,
++* SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true. If there are unforseen highlight problems,
    this may be the culprit.
  
  New Features
@@@ -125,6 -98,22 +124,21 @@@ Other Change
  
  * SOLR-12620: Remove the Admin UI Cloud -> Graph (Radial) view (janhoy)
  
+ ==================  7.7.0 ==================
+ 
+ Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+ 
+ Versions of Major Components
+ ---------------------
+ Apache Tika 1.19.1
+ Carrot2 3.16.0
+ Velocity 1.7 and Velocity Tools 2.0
+ Apache ZooKeeper 3.4.11
+ Jetty 9.4.11.v20180605
+ 
+ 
+ (No Changes)
+ 
 -
  ==================  7.6.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/6dd3ef2b/solr/core/src/java/org/apache/solr/update/PeerSync.java
----------------------------------------------------------------------


[05/50] [abbrv] lucene-solr:jira/http2: SOLR-12795: Introduce 'rows' and 'offset' parameter in FacetStream

Posted by da...@apache.org.
SOLR-12795: Introduce 'rows' and 'offset' parameter in FacetStream


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

Branch: refs/heads/jira/http2
Commit: 3d942131104a38a470b21020bfeb4a12c2dcd99b
Parents: 0ddbc4b
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Nov 6 11:22:51 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Nov 6 11:23:40 2018 -0500

----------------------------------------------------------------------
 .../client/solrj/io/stream/FacetStream.java     | 351 ++++++++++++++++---
 .../solrj/io/stream/StreamExpressionTest.java   | 141 ++++++++
 .../stream/StreamExpressionToExpessionTest.java | 203 ++++++++++-
 3 files changed, 631 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3d942131/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
index 4564ba0..6923a3b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
@@ -64,13 +64,21 @@ public class FacetStream extends TupleStream implements Expressible  {
 
   private Bucket[] buckets;
   private Metric[] metrics;
+  private int rows;
+  private int offset;
+  private int overfetch;
   private int bucketSizeLimit;
+  private boolean refine;
+  private String method;
   private FieldComparator[] bucketSorts;
   private List<Tuple> tuples = new ArrayList<Tuple>();
   private int index;
   private String zkHost;
   private ModifiableSolrParams params;
   private String collection;
+  private boolean resortNeeded;
+  private boolean serializeBucketSizeLimit;
+
   protected transient SolrClientCache cache;
   protected transient CloudSolrClient cloudSolrClient;
 
@@ -81,7 +89,11 @@ public class FacetStream extends TupleStream implements Expressible  {
                      Metric[] metrics,
                      FieldComparator[] bucketSorts,
                      int bucketSizeLimit) throws IOException {
-    init(collection, params, buckets, bucketSorts, metrics, bucketSizeLimit, zkHost);
+
+    if(bucketSizeLimit == -1) {
+      bucketSizeLimit = Integer.MAX_VALUE;
+    }
+    init(collection, params, buckets, bucketSorts, metrics, bucketSizeLimit,0, bucketSizeLimit, false, null, true, 0, zkHost);
   }
   
   public FacetStream(StreamExpression expression, StreamFactory factory) throws IOException{   
@@ -91,9 +103,14 @@ public class FacetStream extends TupleStream implements Expressible  {
     StreamExpressionNamedParameter bucketExpression = factory.getNamedOperand(expression, "buckets");
     StreamExpressionNamedParameter bucketSortExpression = factory.getNamedOperand(expression, "bucketSorts");
     List<StreamExpression> metricExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, Metric.class);
-    StreamExpressionNamedParameter limitExpression = factory.getNamedOperand(expression, "bucketSizeLimit");
+    StreamExpressionNamedParameter bucketLimitExpression = factory.getNamedOperand(expression, "bucketSizeLimit");
     StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
-    
+    StreamExpressionNamedParameter rowsExpression = factory.getNamedOperand(expression, "rows");
+    StreamExpressionNamedParameter offsetExpression = factory.getNamedOperand(expression, "offset");
+    StreamExpressionNamedParameter overfetchExpression = factory.getNamedOperand(expression, "overfetch");
+    StreamExpressionNamedParameter refineExpression = factory.getNamedOperand(expression, "refine");
+    StreamExpressionNamedParameter methodExpression = factory.getNamedOperand(expression, "method");
+
     // Validate there are no unknown parameters
     if(expression.getParameters().size() != 1 + namedParams.size() + metricExpressions.size()){
       throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found",expression));
@@ -112,7 +129,15 @@ public class FacetStream extends TupleStream implements Expressible  {
     // pull out known named params
     ModifiableSolrParams params = new ModifiableSolrParams();
     for(StreamExpressionNamedParameter namedParam : namedParams){
-      if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("buckets") && !namedParam.getName().equals("bucketSorts") && !namedParam.getName().equals("limit")){
+      if(!namedParam.getName().equals("zkHost") &&
+          !namedParam.getName().equals("buckets") &&
+          !namedParam.getName().equals("bucketSorts") &&
+          !namedParam.getName().equals("bucketSizeLimit") &&
+          !namedParam.getName().equals("method") &&
+          !namedParam.getName().equals("offset") &&
+          !namedParam.getName().equals("rows") &&
+          !namedParam.getName().equals("refine") &&
+          !namedParam.getName().equals("overfetch")){
         params.add(namedParam.getName(), namedParam.getParameter().toString().trim());
       }
     }
@@ -130,45 +155,121 @@ public class FacetStream extends TupleStream implements Expressible  {
         }
       }
     }
+
     if(null == buckets){      
       throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one bucket expected. eg. 'buckets=\"name\"'",expression,collectionName));
     }
-    
-    // bucketSorts, required
-    FieldComparator[] bucketSorts = null;
-    if(null != bucketSortExpression){
-      if(bucketSortExpression.getParameter() instanceof StreamExpressionValue){
-        bucketSorts = parseBucketSorts(((StreamExpressionValue)bucketSortExpression.getParameter()).getValue());
-      }
+
+    String bucketSortString = null;
+
+    if(bucketSortExpression == null) {
+      bucketSortString = "count(*) desc";
+    } else {
+      bucketSortString = ((StreamExpressionValue)bucketSortExpression.getParameter()).getValue();
     }
-    if(null == bucketSorts || 0 == bucketSorts.length){      
+
+    FieldComparator[] bucketSorts = parseBucketSorts(bucketSortString, buckets);
+
+    if(null == bucketSorts || 0 == bucketSorts.length) {
       throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one bucket sort expected. eg. 'bucketSorts=\"name asc\"'",expression,collectionName));
     }
-    
+
     // Construct the metrics
     Metric[] metrics = new Metric[metricExpressions.size()];
-    for(int idx = 0; idx < metricExpressions.size(); ++idx){
+    for(int idx = 0; idx < metricExpressions.size(); ++idx) {
       metrics[idx] = factory.constructMetric(metricExpressions.get(idx));
     }
-    if(0 == metrics.length){
+
+    if(0 == metrics.length) {
       throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one metric expected.",expression,collectionName));
     }
-    
-    if(null == limitExpression || null == limitExpression.getParameter() || !(limitExpression.getParameter() instanceof StreamExpressionValue)){
-      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single 'limit' parameter of type positive integer but didn't find one",expression));
-    }
-    String limitStr = ((StreamExpressionValue)limitExpression.getParameter()).getValue();
-    int limitInt = 0;
-    try{
-      limitInt = Integer.parseInt(limitStr);
-      if(limitInt <= 0 && limitInt != -1){
-        throw new IOException(String.format(Locale.ROOT,"invalid expression %s - limit '%s' must be greater than 0 or -1.",expression, limitStr));
+
+    boolean refine = false;
+
+    if(refineExpression != null) {
+      String refineStr = ((StreamExpressionValue) refineExpression.getParameter()).getValue();
+      if (refineStr != null) {
+        refine = Boolean.parseBoolean(refineStr);
       }
     }
-    catch(NumberFormatException e){
-      throw new IOException(String.format(Locale.ROOT,"invalid expression %s - limit '%s' is not a valid integer.",expression, limitStr));
+
+    if(bucketLimitExpression != null && (rowsExpression != null ||
+                                         offsetExpression != null ||
+                                         overfetchExpression != null)) {
+      throw new IOException("bucketSizeLimit is incompatible with rows, offset and overfetch.");
     }
-    
+
+    String methodStr = null;
+    if(methodExpression != null) {
+      methodStr = ((StreamExpressionValue) methodExpression.getParameter()).getValue();
+    }
+
+    int overfetchInt = 150;
+    if(overfetchExpression != null) {
+      String overfetchStr = ((StreamExpressionValue) overfetchExpression.getParameter()).getValue();
+      overfetchInt = Integer.parseInt(overfetchStr);
+    }
+
+    int offsetInt = 0;
+    if(offsetExpression != null) {
+      String offsetStr = ((StreamExpressionValue) offsetExpression.getParameter()).getValue();
+      offsetInt = Integer.parseInt(offsetStr);
+    }
+
+    int rowsInt = Integer.MIN_VALUE;
+    int bucketLimit = Integer.MIN_VALUE;
+    boolean bucketLimitSet = false;
+
+    if(null != rowsExpression) {
+      String rowsStr = ((StreamExpressionValue)rowsExpression.getParameter()).getValue();
+      try {
+        rowsInt = Integer.parseInt(rowsStr);
+        if (rowsInt <= 0 && rowsInt != -1) {
+          throw new IOException(String.format(Locale.ROOT, "invalid expression %s - limit '%s' must be greater than 0 or -1.", expression, rowsStr));
+        }
+        //Rows is set so configure the bucketLimitSize
+        if(rowsInt == -1) {
+          bucketLimit = rowsInt = Integer.MAX_VALUE;
+        } else if(overfetchInt == -1) {
+          bucketLimit = Integer.MAX_VALUE;
+        }else{
+          bucketLimit = offsetInt+overfetchInt+rowsInt;
+        }
+      } catch (NumberFormatException e) {
+        throw new IOException(String.format(Locale.ROOT, "invalid expression %s - limit '%s' is not a valid integer.", expression, rowsStr));
+      }
+    }
+
+    if(bucketLimitExpression != null) {
+      String bucketLimitStr = ((StreamExpressionValue) bucketLimitExpression.getParameter()).getValue();
+      try {
+        bucketLimit = Integer.parseInt(bucketLimitStr);
+        bucketLimitSet = true;
+
+        if (bucketLimit <= 0 && bucketLimit != -1) {
+          throw new IOException(String.format(Locale.ROOT, "invalid expression %s - bucketSizeLimit '%s' must be greater than 0 or -1.", expression, bucketLimitStr));
+        }
+
+        // Bucket limit is set. So set rows.
+        if(bucketLimit == -1) {
+         rowsInt = bucketLimit = Integer.MAX_VALUE;
+        } else {
+          rowsInt = bucketLimit;
+        }
+      }  catch (NumberFormatException e) {
+        throw new IOException(String.format(Locale.ROOT, "invalid expression %s - bucketSizeLimit '%s' is not a valid integer.", expression, bucketLimitStr));
+      }
+    }
+
+    if(rowsExpression == null && bucketLimitExpression == null) {
+      rowsInt = 10;
+      if(overfetchInt == -1) {
+        bucketLimit = Integer.MAX_VALUE;
+      }else{
+        bucketLimit = offsetInt+overfetchInt+rowsInt;
+      }
+    }
+
     // zkHost, optional - if not provided then will look into factory list to get
     String zkHost = null;
     if(null == zkHostExpression){
@@ -176,16 +277,44 @@ public class FacetStream extends TupleStream implements Expressible  {
       if(zkHost == null) {
         zkHost = factory.getDefaultZkHost();
       }
-    }
-    else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
+    } else if(zkHostExpression.getParameter() instanceof StreamExpressionValue) {
       zkHost = ((StreamExpressionValue)zkHostExpression.getParameter()).getValue();
     }
+
     if(null == zkHost){
       throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
     }
     
     // We've got all the required items
-    init(collectionName, params, buckets, bucketSorts, metrics, limitInt, zkHost);
+    init(collectionName,
+         params,
+         buckets,
+         bucketSorts,
+         metrics,
+         rowsInt,
+         offsetInt,
+         bucketLimit,
+         refine,
+         methodStr,
+         bucketLimitSet,
+         overfetchInt,
+         zkHost);
+  }
+
+  public int getBucketSizeLimit() {
+    return this.bucketSizeLimit;
+  }
+
+  public int getRows() {
+    return this.rows;
+  }
+
+  public int getOffset() {
+    return this.offset;
+  }
+
+  public int getOverfetch() {
+    return this.overfetch;
   }
 
   public Bucket[] getBuckets() {
@@ -196,7 +325,7 @@ public class FacetStream extends TupleStream implements Expressible  {
     return this.collection;
   }
 
-  private FieldComparator[] parseBucketSorts(String bucketSortString) throws IOException {
+  private FieldComparator[] parseBucketSorts(String bucketSortString, Bucket[] buckets) throws IOException {
 
     String[] sorts = bucketSortString.split(",");
     FieldComparator[] comps = new FieldComparator[sorts.length];
@@ -217,17 +346,20 @@ public class FacetStream extends TupleStream implements Expressible  {
     return comps;
   }
 
-  private void init(String collection, SolrParams params, Bucket[] buckets, FieldComparator[] bucketSorts, Metric[] metrics, int bucketSizeLimit, String zkHost) throws IOException {
+  private void init(String collection, SolrParams params, Bucket[] buckets, FieldComparator[] bucketSorts, Metric[] metrics, int rows, int offset, int bucketSizeLimit, boolean refine, String method, boolean serializeBucketSizeLimit, int overfetch, String zkHost) throws IOException {
     this.zkHost  = zkHost;
     this.params = new ModifiableSolrParams(params);
     this.buckets = buckets;
     this.metrics = metrics;
+    this.rows = rows;
+    this.offset = offset;
+    this.refine = refine;
     this.bucketSizeLimit   = bucketSizeLimit;
-    if (this.bucketSizeLimit == -1) {
-      this.bucketSizeLimit = Integer.MAX_VALUE;
-    }
     this.collection = collection;
     this.bucketSorts = bucketSorts;
+    this.method = method;
+    this.serializeBucketSizeLimit = serializeBucketSizeLimit;
+    this.overfetch = overfetch;
     
     // In a facet world it only makes sense to have the same field name in all of the sorters
     // Because FieldComparator allows for left and right field names we will need to validate
@@ -280,8 +412,31 @@ public class FacetStream extends TupleStream implements Expressible  {
       expression.addParameter(metric.toExpression(factory));
     }
     
-    // limit
-    expression.addParameter(new StreamExpressionNamedParameter("bucketSizeLimit", Integer.toString(bucketSizeLimit)));
+    if(serializeBucketSizeLimit) {
+      if(bucketSizeLimit == Integer.MAX_VALUE) {
+        expression.addParameter(new StreamExpressionNamedParameter("bucketSizeLimit", Integer.toString(-1)));
+      } else {
+        expression.addParameter(new StreamExpressionNamedParameter("bucketSizeLimit", Integer.toString(bucketSizeLimit)));
+      }
+    } else {
+      if (rows == Integer.MAX_VALUE) {
+        expression.addParameter(new StreamExpressionNamedParameter("rows", Integer.toString(-1)));
+      } else{
+        expression.addParameter(new StreamExpressionNamedParameter("rows", Integer.toString(rows)));
+      }
+
+      expression.addParameter(new StreamExpressionNamedParameter("offset", Integer.toString(offset)));
+
+      if(overfetch == Integer.MAX_VALUE) {
+        expression.addParameter(new StreamExpressionNamedParameter("overfetch", Integer.toString(-1)));
+      } else {
+        expression.addParameter(new StreamExpressionNamedParameter("overfetch", Integer.toString(overfetch)));
+      }
+    }
+
+    if(method != null) {
+      expression.addParameter(new StreamExpressionNamedParameter("method", this.method));
+    }
         
     // zkHost
     expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
@@ -333,8 +488,10 @@ public class FacetStream extends TupleStream implements Expressible  {
     }
 
     FieldComparator[] adjustedSorts = adjustSorts(buckets, bucketSorts);
-    String json = getJsonFacetString(buckets, metrics, adjustedSorts, bucketSizeLimit);
+    this.resortNeeded = resortNeeded(adjustedSorts);
 
+    String json = getJsonFacetString(buckets, metrics, adjustedSorts, method, refine, bucketSizeLimit);
+    assert expectedJson(json);
     ModifiableSolrParams paramsLoc = new ModifiableSolrParams(params);
     paramsLoc.set("json.facet", json);
     paramsLoc.set("rows", "0");
@@ -343,13 +500,58 @@ public class FacetStream extends TupleStream implements Expressible  {
     try {
       NamedList response = cloudSolrClient.request(request, collection);
       getTuples(response, buckets, metrics);
-      Collections.sort(tuples, getStreamSort());
 
+      if(resortNeeded) {
+        Collections.sort(tuples, getStreamSort());
+      }
+
+      index=this.offset;
     } catch (Exception e) {
       throw new IOException(e);
     }
   }
 
+  private boolean expectedJson(String json) {
+    if(this.method != null) {
+      if(!json.contains("\"method\":\""+this.method+"\"")) {
+        return false;
+      }
+    }
+
+    if(this.refine) {
+      if(!json.contains("\"refine\":true")) {
+        return false;
+      }
+    }
+
+    if(serializeBucketSizeLimit) {
+      if(!json.contains("\"limit\":"+bucketSizeLimit)) {
+        return false;
+      }
+    } else {
+      if(!json.contains("\"limit\":"+(this.rows+this.offset+this.overfetch))) {
+        return false;
+      }
+    }
+
+    for(Bucket bucket : buckets) {
+      if(!json.contains("\""+bucket.toString()+"\":")) {
+        return false;
+      }
+    }
+
+    for(Metric metric: metrics) {
+      String func = metric.getFunctionName();
+      if(!func.equals("count")) {
+        if (!json.contains(metric.getIdentifier())) {
+          return false;
+        }
+      }
+    }
+
+    return true;
+  }
+
   public void close() throws IOException {
     if(cache == null) {
       if (cloudSolrClient != null) {
@@ -359,21 +561,32 @@ public class FacetStream extends TupleStream implements Expressible  {
   }
 
   public Tuple read() throws IOException {
-    if(index < tuples.size() && index < bucketSizeLimit) {
+    if(index < tuples.size() && index < (offset+rows)) {
       Tuple tuple = tuples.get(index);
       ++index;
       return tuple;
     } else {
       Map fields = new HashMap();
+
+      if(bucketSizeLimit == Integer.MAX_VALUE) {
+        fields.put("totalRows", tuples.size());
+      }
+
       fields.put("EOF", true);
+
       Tuple tuple = new Tuple(fields);
       return tuple;
     }
   }
 
-  private String getJsonFacetString(Bucket[] _buckets, Metric[] _metrics, FieldComparator[] _sorts, int _limit) {
+  private String getJsonFacetString(Bucket[] _buckets,
+                                    Metric[] _metrics,
+                                    FieldComparator[] _sorts,
+                                    String _method,
+                                    boolean _refine,
+                                    int _limit) {
     StringBuilder buf = new StringBuilder();
-    appendJson(buf, _buckets, _metrics, _sorts, _limit, 0);
+    appendJson(buf, _buckets, _metrics, _sorts, _limit, _method, _refine,0);
     return "{"+buf.toString()+"}";
   }
 
@@ -399,11 +612,22 @@ public class FacetStream extends TupleStream implements Expressible  {
     }
   }
 
+  private boolean resortNeeded(FieldComparator[] fieldComparators) {
+    for(FieldComparator fieldComparator : fieldComparators) {
+      if(fieldComparator.getLeftFieldName().contains("(")) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   private void appendJson(StringBuilder buf,
                           Bucket[] _buckets,
                           Metric[] _metrics,
                           FieldComparator[] _sorts,
                           int _limit,
+                          String method,
+                          boolean refine,
                           int level) {
     buf.append('"');
     buf.append(_buckets[level].toString());
@@ -412,42 +636,63 @@ public class FacetStream extends TupleStream implements Expressible  {
     buf.append("\"type\":\"terms\"");
     buf.append(",\"field\":\""+_buckets[level].toString()+"\"");
     buf.append(",\"limit\":"+_limit);
-    buf.append(",\"sort\":{\""+getFacetSort(_sorts[level].getLeftFieldName(), _metrics)+"\":\""+_sorts[level].getOrder()+"\"}");
+
+    if(refine) {
+      buf.append(",\"refine\":true");
+    }
+
+    if(method != null) {
+      buf.append(",\"method\":\""+method+"\"");
+    }
+
+    String fsort = getFacetSort(_sorts[level].getLeftFieldName(), _metrics);
+
+    buf.append(",\"sort\":{\""+fsort+"\":\""+_sorts[level].getOrder()+"\"}");
 
     buf.append(",\"facet\":{");
     int metricCount = 0;
+
+
+    ++level;
     for(Metric metric : _metrics) {
-      String identifier = metric.getIdentifier();
-      if(!identifier.startsWith("count(")) {
-        if(metricCount>0) {
-          buf.append(",");
+      //Only compute the metric if it's a leaf node or if the branch level sort equals is the metric
+      String facetKey = "facet_"+metricCount;
+      if(level == _buckets.length || fsort.equals(facetKey) ) {
+        String identifier = metric.getIdentifier();
+        if (!identifier.startsWith("count(")) {
+          if (metricCount > 0) {
+            buf.append(",");
+          }
+          buf.append("\""+ facetKey + "\":\"" + identifier + "\"");
+          ++metricCount;
         }
-        buf.append("\"facet_" + metricCount + "\":\"" +identifier+"\"");
-        ++metricCount;
       }
     }
-    ++level;
+
     if(level < _buckets.length) {
       if(metricCount>0) {
         buf.append(",");
       }
-      appendJson(buf, _buckets, _metrics, _sorts, _limit, level);
+      appendJson(buf, _buckets, _metrics, _sorts, _limit, method, refine, level);
     }
     buf.append("}}");
   }
 
   private String getFacetSort(String id, Metric[] _metrics) {
     int index = 0;
+    int metricCount=0;
     for(Metric metric : _metrics) {
       if(metric.getIdentifier().startsWith("count(")) {
         if(id.startsWith("count(")) {
           return "count";
         }
+        ++index;
       } else {
         if (id.equals(_metrics[index].getIdentifier())) {
-          return "facet_" + index;
+          return "facet_" + metricCount;
         }
         ++index;
+        ++metricCount;
       }
     }
     return "index";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3d942131/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 24264b6..7e2451e 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
@@ -986,6 +986,147 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertTrue(count.doubleValue() == 2);
 
 
+    clause = "facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "fl=\"a_s,a_i,a_f\", "
+        +   "sort=\"a_s asc\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) desc\", "
+        +   "rows=2, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")";
+
+    stream = factory.constructStream(clause);
+    tuples = getTuples(stream);
+
+
+    //Test rows
+
+    tuple = tuples.get(0);
+    assertEquals(tuples.size(), 2);
+
+    bucket = tuple.getString("a_s");
+    sumi = tuple.getDouble("sum(a_i)");
+    sumf = tuple.getDouble("sum(a_f)");
+    mini = tuple.getDouble("min(a_i)");
+    minf = tuple.getDouble("min(a_f)");
+    maxi = tuple.getDouble("max(a_i)");
+    maxf = tuple.getDouble("max(a_f)");
+    avgi = tuple.getDouble("avg(a_i)");
+    avgf = tuple.getDouble("avg(a_f)");
+    count = tuple.getDouble("count(*)");
+
+    assertTrue(bucket.equals("hello3"));
+    assertTrue(sumi.doubleValue() == 38.0D);
+    assertTrue(sumf.doubleValue() == 26.0D);
+    assertTrue(mini.doubleValue() == 3.0D);
+    assertTrue(minf.doubleValue() == 3.0D);
+    assertTrue(maxi.doubleValue() == 13.0D);
+    assertTrue(maxf.doubleValue() == 9.0D);
+    assertTrue(avgi.doubleValue() == 9.5D);
+    assertTrue(avgf.doubleValue() == 6.5D);
+    assertTrue(count.doubleValue() == 4);
+
+    tuple = tuples.get(1);
+    bucket = tuple.getString("a_s");
+    sumi = tuple.getDouble("sum(a_i)");
+    sumf = tuple.getDouble("sum(a_f)");
+    mini = tuple.getDouble("min(a_i)");
+    minf = tuple.getDouble("min(a_f)");
+    maxi = tuple.getDouble("max(a_i)");
+    maxf = tuple.getDouble("max(a_f)");
+    avgi = tuple.getDouble("avg(a_i)");
+    avgf = tuple.getDouble("avg(a_f)");
+    count = tuple.getDouble("count(*)");
+
+    assertTrue(bucket.equals("hello0"));
+    assertTrue(sumi.doubleValue() == 17.0D);
+    assertTrue(sumf.doubleValue() == 18.0D);
+    assertTrue(mini.doubleValue() == 0.0D);
+    assertTrue(minf.doubleValue() == 1.0D);
+    assertTrue(maxi.doubleValue() == 14.0D);
+    assertTrue(maxf.doubleValue() == 10.0D);
+    assertTrue(avgi.doubleValue() == 4.25D);
+    assertTrue(avgf.doubleValue() == 4.5D);
+    assertTrue(count.doubleValue() == 4);
+
+
+    clause = "facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "fl=\"a_s,a_i,a_f\", "
+        +   "sort=\"a_s asc\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) desc\", "
+        +   "rows=2, offset=1, method=dvhash, refine=true,"
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")";
+
+    stream = factory.constructStream(clause);
+    tuples = getTuples(stream);
+
+
+    //Test offset
+
+    tuple = tuples.get(0);
+    assertEquals(tuples.size(), 2);
+
+    tuple = tuples.get(0);
+    bucket = tuple.getString("a_s");
+    sumi = tuple.getDouble("sum(a_i)");
+    sumf = tuple.getDouble("sum(a_f)");
+    mini = tuple.getDouble("min(a_i)");
+    minf = tuple.getDouble("min(a_f)");
+    maxi = tuple.getDouble("max(a_i)");
+    maxf = tuple.getDouble("max(a_f)");
+    avgi = tuple.getDouble("avg(a_i)");
+    avgf = tuple.getDouble("avg(a_f)");
+    count = tuple.getDouble("count(*)");
+
+    assertTrue(bucket.equals("hello0"));
+    assertTrue(sumi.doubleValue() == 17.0D);
+    assertTrue(sumf.doubleValue() == 18.0D);
+    assertTrue(mini.doubleValue() == 0.0D);
+    assertTrue(minf.doubleValue() == 1.0D);
+    assertTrue(maxi.doubleValue() == 14.0D);
+    assertTrue(maxf.doubleValue() == 10.0D);
+    assertTrue(avgi.doubleValue() == 4.25D);
+    assertTrue(avgf.doubleValue() == 4.5D);
+    assertTrue(count.doubleValue() == 4);
+
+    tuple = tuples.get(1);
+    bucket = tuple.getString("a_s");
+    sumi = tuple.getDouble("sum(a_i)");
+    sumf = tuple.getDouble("sum(a_f)");
+    mini = tuple.getDouble("min(a_i)");
+    minf = tuple.getDouble("min(a_f)");
+    maxi = tuple.getDouble("max(a_i)");
+    maxf = tuple.getDouble("max(a_f)");
+    avgi = tuple.getDouble("avg(a_i)");
+    avgf = tuple.getDouble("avg(a_f)");
+    count = tuple.getDouble("count(*)");
+
+    assertTrue(bucket.equals("hello4"));
+    assertTrue(sumi.longValue() == 15);
+    assertTrue(sumf.doubleValue() == 11.0D);
+    assertTrue(mini.doubleValue() == 4.0D);
+    assertTrue(minf.doubleValue() == 4.0D);
+    assertTrue(maxi.doubleValue() == 11.0D);
+    assertTrue(maxf.doubleValue() == 7.0D);
+    assertTrue(avgi.doubleValue() == 7.5D);
+    assertTrue(avgf.doubleValue() == 5.5D);
+    assertTrue(count.doubleValue() == 2);
+
+
     //Test index sort
     clause = "facet("
         +   "collection1, "

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3d942131/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
index e43176a..289b925 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.client.solrj.io.stream;
 
+import java.io.IOException;
+
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
@@ -247,17 +249,17 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
     
     // Basic test
     try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
-                                                        +   "collection1, "
-                                                        +   "q=\"*:*\", "
-                                                        +   "buckets=\"a_s\", "
-                                                        +   "bucketSorts=\"sum(a_i) asc\", "
-                                                        +   "bucketSizeLimit=100, "
-                                                        +   "sum(a_i), sum(a_f), "
-                                                        +   "min(a_i), min(a_f), "
-                                                        +   "max(a_i), max(a_f), "
-                                                        +   "avg(a_i), avg(a_f), "
-                                                        +   "count(*)"
-                                                        + ")"), factory)){
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) asc\", "
+        +   "bucketSizeLimit=100, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
       expressionString = stream.toExpression(factory).toString();
       assertTrue(expressionString.contains("facet(collection1"));
       assertTrue(expressionString.contains("q=\"*:*\""));
@@ -273,7 +275,186 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
       assertTrue(expressionString.contains("avg(a_i,false)"));
       assertTrue(expressionString.contains("avg(a_f,false)"));
       assertTrue(expressionString.contains("count(*)"));
+      assertEquals(stream.getBucketSizeLimit(), 100);
+      assertEquals(stream.getRows(), 100);
+      assertEquals(stream.getOffset(), 0);
     }
+
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) asc\", "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
+      assertTrue(expressionString.contains("rows=10"));
+      assertTrue(expressionString.contains("offset=0"));
+      assertTrue(expressionString.contains("overfetch=150"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+      assertEquals(stream.getOverfetch(), 150);
+      assertEquals(stream.getBucketSizeLimit(), 160);
+      assertEquals(stream.getRows(), 10);
+      assertEquals(stream.getOffset(), 0);
+    }
+
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSizeLimit=100, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"count(*) desc\""));
+      assertTrue(expressionString.contains("bucketSizeLimit=100"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+    }
+
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) asc\", "
+        +   "rows=10, method=dvhash, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
+      assertTrue(!expressionString.contains("bucketSizeLimit"));
+      assertTrue(expressionString.contains("rows=10"));
+      assertTrue(expressionString.contains("offset=0"));
+      assertTrue(expressionString.contains("overfetch=150"));
+      assertTrue(expressionString.contains("method=dvhash"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+      assertEquals(stream.getBucketSizeLimit(), 160);
+      assertEquals(stream.getRows(), 10);
+      assertEquals(stream.getOffset(), 0);
+      assertEquals(stream.getOverfetch(), 150);
+
+    }
+
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) asc\", "
+        +   "rows=10, offset=100, overfetch=30, method=dvhash, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
+      assertTrue(!expressionString.contains("bucketSizeLimit"));
+      assertTrue(expressionString.contains("rows=10"));
+      assertTrue(expressionString.contains("offset=100"));
+      assertTrue(expressionString.contains("overfetch=30"));
+      assertTrue(expressionString.contains("method=dvhash"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+      assertEquals(stream.getBucketSizeLimit(), 140);
+      assertEquals(stream.getRows(), 10);
+      assertEquals(stream.getOffset(), 100);
+      assertEquals(stream.getOverfetch(), 30);
+
+    }
+
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
+        +   "collection1, "
+        +   "q=\"*:*\", "
+        +   "buckets=\"a_s\", "
+        +   "bucketSorts=\"sum(a_i) asc\", "
+        +   "rows=-1, offset=100, overfetch=-1, method=dvhash, "
+        +   "sum(a_i), sum(a_f), "
+        +   "min(a_i), min(a_f), "
+        +   "max(a_i), max(a_f), "
+        +   "avg(a_i), avg(a_f), "
+        +   "count(*)"
+        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
+      assertTrue(!expressionString.contains("bucketSizeLimit"));
+      assertTrue(expressionString.contains("rows=-1"));
+      assertTrue(expressionString.contains("offset=100"));
+      assertTrue(expressionString.contains("overfetch=-1"));
+      assertTrue(expressionString.contains("method=dvhash"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+      assertEquals(stream.getBucketSizeLimit(), Integer.MAX_VALUE);
+      assertEquals(stream.getRows(), Integer.MAX_VALUE);
+      assertEquals(stream.getOffset(), 100);
+      assertEquals(stream.getOverfetch(), -1);
+    }
+
   }
   
   @Test


[09/50] [abbrv] lucene-solr:jira/http2: LUCENE-8559: Fix bug where polygon edges were skipped when checking for intersections

Posted by da...@apache.org.
LUCENE-8559: Fix bug where polygon edges were skipped when checking for intersections


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

Branch: refs/heads/jira/http2
Commit: 9952af099ae65f051056fc8ff55c8e8f4cfb3b93
Parents: 6f6a880
Author: iverase <iv...@apache.org>
Authored: Wed Nov 7 07:44:32 2018 +0100
Committer: iverase <iv...@apache.org>
Committed: Wed Nov 7 07:44:32 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                                       |  3 +++
 .../src/java/org/apache/lucene/geo/Tessellator.java      | 11 +++++++----
 .../src/test/org/apache/lucene/geo/TestTessellator.java  |  7 +++++++
 3 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9952af09/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 313d7bf..dadc53d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -226,6 +226,9 @@ Bug fixes:
 * LUCENE-8534: Fix incorrect computation for triangles intersecting polygon edges in
   shape tessellation. (Ignacio Vera)
 
+* LUCENE-8559: Fix bug where polygon edges were skipped when checking for intersections.
+  (Ignacio Vera)   
+
 New Features
 
 * LUCENE-8496: Selective indexing - modify BKDReader/BKDWriter to allow users

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9952af09/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java b/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
index 345a73b..8fa685c 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/geo/Tessellator.java
@@ -540,9 +540,7 @@ final public class Tessellator {
     Node nextNode;
     do {
       nextNode = node.next;
-      if(node.getX() != x0 && node.getY() != y0 && nextNode.getX() != x0
-          && nextNode.getY() != y0 && node.getX() != x1 && node.getY() != y1
-          && nextNode.getX() != x1 && nextNode.getY() != y1) {
+      if(isVertexEquals(node, x0, y0) == false && isVertexEquals(node, x1, y1) == false) {
         if (linesIntersect(node.getX(), node.getY(), nextNode.getX(), nextNode.getY(), x0, y0, x1, y1)) {
           return true;
         }
@@ -709,7 +707,12 @@ final public class Tessellator {
 
   /** Determines if two point vertices are equal. **/
   private static final boolean isVertexEquals(final Node a, final Node b) {
-    return a.getX() == b.getX() && a.getY() == b.getY();
+    return isVertexEquals(a, b.getX(), b.getY());
+  }
+
+  /** Determines if two point vertices are equal. **/
+  private static final boolean isVertexEquals(final Node a, final double x, final  double y) {
+    return a.getX() == x && a.getY() == y;
   }
 
   /** Compute signed area of triangle */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9952af09/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java b/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
index 82ba5b4..7904725 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/geo/TestTessellator.java
@@ -96,4 +96,11 @@ public class TestTessellator extends LuceneTestCase {
     Polygon polygon = (Polygon)SimpleWKTShapeParser.parse(wkt);
     assertTrue(Tessellator.tessellate(polygon).size() == 8);
   }
+
+  public void testLUCENE8559()  throws Exception {
+    String wkt = "POLYGON((-0.1348674 51.7458255,-0.1345884 51.7455067,-0.1329898 51.745314,-0.1326358 51.745314,-0.1324105 51.744404,-0.131981 51.7444423,-0.1312196 51.7445102,-0.1310908 51.7456794,-0.1319706 51.7460713,-0.1343095 51.7465828,-0.1348674 51.7458255)," +
+        "(-0.1322388 51.7447959,-0.1322388 51.7454336,-0.1318633 51.7457126,-0.1313912 51.7456262,-0.1318985 51.7448032,-0.1322388 51.7447959))";
+    Polygon polygon = (Polygon)SimpleWKTShapeParser.parse(wkt);
+    assertTrue(Tessellator.tessellate(polygon).size() > 0);
+  }
 }
\ No newline at end of file


[12/50] [abbrv] lucene-solr:jira/http2: Remove hyperlink to - now spammy - heliosearch domain

Posted by da...@apache.org.
Remove hyperlink to - now spammy - heliosearch domain


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

Branch: refs/heads/jira/http2
Commit: 936ecf12fbb0e9543cdf3e4df91897c219e541ff
Parents: 50cf1cc
Author: Alexandre Rafalovitch <ar...@apache.org>
Authored: Wed Nov 7 07:26:12 2018 -0500
Committer: Alexandre Rafalovitch <ar...@apache.org>
Committed: Wed Nov 7 07:26:12 2018 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/collapse-and-expand-results.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/936ecf12/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
index d967b93..6eb4e1b 100644
--- a/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
+++ b/solr/solr-ref-guide/src/collapse-and-expand-results.adoc
@@ -120,7 +120,7 @@ The CollapsingQParserPlugin fully supports the QueryElevationComponent.
 
 == Expand Component
 
-The ExpandComponent can be used to expand the groups that were collapsed by the http://heliosearch.org/the-collapsingqparserplugin-solrs-new-high-performance-field-collapsing-postfilter/[CollapsingQParserPlugin].
+The ExpandComponent can be used to expand the groups that were collapsed by the CollapsingQParserPlugin.
 
 Example usage with the CollapsingQParserPlugin: