You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ct...@apache.org on 2021/01/15 21:45:31 UTC

[lucene-solr] 20/38: SOLR-12559: fix error when multi-val fields are derefernced in JSON aggs

This is an automated email from the ASF dual-hosted git repository.

ctargett pushed a commit to branch jira/solr-13105-toMerge
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit e054ad740d698097a78a8092790af83cf3012961
Author: Munendra S N <mu...@apache.org>
AuthorDate: Sun Nov 29 05:16:35 2020 +0530

    SOLR-12559: fix error when multi-val fields are derefernced in JSON aggs
    
    This ensures all derefernced fields are not parsed into actual valuesource
    but parsed into a placeholder value. This works for 1-level of dereferencing
---
 solr/CHANGES.txt                                   |  3 ++
 .../org/apache/solr/search/FunctionQParser.java    | 26 ++++++----
 .../apache/solr/search/facet/TestJsonFacets.java   | 14 +++---
 .../search/facet/TestJsonFacetsStatsParsing.java   | 56 ++++++++++++----------
 4 files changed, 55 insertions(+), 44 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 79610ad..68886d1 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -322,6 +322,9 @@ Bug Fixes
 * SOLR-14514: Avoid picking 'stream' method in JSON facet when any of 'allBuckets', 'numBuckets', and 'missing' parameters are enabled
   (hossman, Munendra S N)
 
+* SOLR-12559: Fix placeholder valuesource 'FIELDNAME' not working with de-referenced values in JSON aggregator parsing.
+  (hossman, Munendra S N)
+
 Other Changes
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/search/FunctionQParser.java b/solr/core/src/java/org/apache/solr/search/FunctionQParser.java
index 60cbecf..73f6d7a 100644
--- a/solr/core/src/java/org/apache/solr/search/FunctionQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/FunctionQParser.java
@@ -356,17 +356,23 @@ public class FunctionQParser extends QParser {
         throw new SyntaxError("Missing param " + param + " while parsing function '" + sp.val + "'");
       }
 
-      QParser subParser = subQuery(val, "func");
-      if (subParser instanceof FunctionQParser) {
-        ((FunctionQParser)subParser).setParseMultipleSources(true);
-      }
-      Query subQuery = subParser.getQuery();
-      if (subQuery == null) {
-        valueSource = new ConstValueSource(0.0f);
-      } else if (subQuery instanceof FunctionQuery) {
-        valueSource = ((FunctionQuery) subQuery).getValueSource();
+      if ((flags & FLAG_USE_FIELDNAME_SOURCE) != 0 && req.getSchema().getFieldOrNull(val) != null) {
+        // Don't try to create a ValueSource for the field, just use a placeholder.
+        // this handles the case like x=myfunc($qq)&qq=something
+        valueSource = new FieldNameValueSource(val);
       } else {
-        valueSource = new QueryValueSource(subQuery, 0.0f);
+        QParser subParser = subQuery(val, "func");
+        if (subParser instanceof FunctionQParser) {
+          ((FunctionQParser) subParser).setParseMultipleSources(true);
+        }
+        Query subQuery = subParser.getQuery();
+        if (subQuery == null) {
+          valueSource = new ConstValueSource(0.0f);
+        } else if (subQuery instanceof FunctionQuery) {
+          valueSource = ((FunctionQuery) subQuery).getValueSource();
+        } else {
+          valueSource = new QueryValueSource(subQuery, 0.0f);
+        }
       }
 
       /***
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 166968b..179661b 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
@@ -1123,8 +1123,6 @@ public class TestJsonFacets extends SolrTestCaseHS {
     fieldLists.put("sparse_s", getAlternatives("sparse_s"));
     fieldLists.put("multi_ss", getAlternatives("multi_ss"));
 
-    // TODO: if a field will be used as a function source, we can't use multi-valued types for it (currently)
-
     int maxAlt = 0;
     for (List<String> fieldList : fieldLists.values()) {
       maxAlt = Math.max(fieldList.size(), maxAlt);
@@ -1923,24 +1921,24 @@ public class TestJsonFacets extends SolrTestCaseHS {
     );
 
     // stats at top level on multi-valued fields
-    client.testJQ(params(p, "q", "*:*"
+    client.testJQ(params(p, "q", "*:*", "myfield", "${multi_ss}"
         , "json.facet", "{ sum1:'sum(${num_fs})', sumsq1:'sumsq(${num_fs})', avg1:'avg(${num_fs})', mind:'min(${num_fs})', maxd:'max(${num_fs})'" +
             ", mini:'min(${num_is})', maxi:'max(${num_is})', mins:'min(${multi_ss})', maxs:'max(${multi_ss})'" +
             ", stddev:'stddev(${num_fs})', variance:'variance(${num_fs})', median:'percentile(${num_fs}, 50)'" +
-            ", perc:'percentile(${num_fs}, 0,75,100)'" +
+            ", perc:'percentile(${num_fs}, 0,75,100)', maxss:'max($multi_ss)'" +
             " }"
         )
         , "facets=={ 'count':6, " +
             "sum1:0.0, sumsq1:51.5, avg1:0.0, mind:-5.0, maxd:3.0" +
             ", mini:-5, maxi:3, mins:'a', maxs:'b'" +
-            ", stddev:2.712405363721075, variance:7.3571428571, median:0.0, perc:[-5.0,2.25,3.0]" +
+            ", stddev:2.712405363721075, variance:7.3571428571, median:0.0, perc:[-5.0,2.25,3.0], maxss:'b'" +
             "}"
     );
 
     // test sorting by multi-valued
-    client.testJQ(params(p, "q", "*:*"
-        , "json.facet", "{f1:{terms:{${terms} field:'${cat_s}', sort:'n1 desc', facet:{n1:'avg(${num_is})'}  }}" +
-            " , f2:{terms:{${terms} field:'${cat_s}', sort:'n1 asc', facet:{n1:'avg(${num_is})'}  }} }"
+    client.testJQ(params(p, "q", "*:*", "my_field", "${num_is}"
+        , "json.facet", "{f1:{terms:{${terms} field:'${cat_s}', sort:'n1 desc', facet:{n1:'avg($my_field)'}  }}" +
+            " , f2:{terms:{${terms} field:'${cat_s}', sort:'n1 asc', facet:{n1:'avg($my_field)'}  }} }"
         )
         , "facets=={ 'count':6, " +
             "  f1:{  'buckets':[{ val:'B', count:3, n1: 0.25}, { val:'A', count:2, n1:0.0}]}" +
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java
index 24ca10f..ed75cf5 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetsStatsParsing.java
@@ -20,16 +20,17 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.lucene.queries.function.valuesource.IntFieldSource;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.search.function.FieldNameValueSource;
+import org.hamcrest.MatcherAssert;
 import org.junit.BeforeClass;
+import org.noggit.ObjectBuilder;
+
 import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
 
-import org.noggit.ObjectBuilder;
-
 /** Whitebox test of the various syntaxes for specifying stats in JSON Facets */
 public class TestJsonFacetsStatsParsing extends SolrTestCaseJ4 {
 
@@ -56,44 +57,47 @@ public class TestJsonFacetsStatsParsing extends SolrTestCaseJ4 {
     try (SolrQueryRequest req = req("custom_req_param","foo_i",
                                     "overridden_param","xxxxx_i")) {
       
-      // NOTE: we don't bother trying to test 'min(foo_i)' because of SOLR-12559
-      // ...once that bug is fixed, several assertions below will need to change
       @SuppressWarnings({"unchecked"})
       final FacetRequest fr = FacetRequest.parse
         (req, (Map<String,Object>) Utils.fromJSONString
          ("{ " +
-          "  s1:'min(field(\"foo_i\"))', " +
-          "  s2:'min($custom_req_param)', " +
-          "  s3:'min(field($custom_req_param))', " +
-          "  s4:{ func:'min($custom_req_param)' }, " +
-          "  s5:{ type:func, func:'min($custom_req_param)' }, " +
-          "  s6:{ type:func, func:'min($custom_local_param)', custom_local_param:foo_i }, " +
-          "  s7:{ type:func, func:'min($overridden_param)', overridden_param:foo_i }, " +
-          // test the test...
-          "  diff:'min(field(\"bar_i\"))'," +
-          "}"));
+             // with valuesource
+             "  f1:'min(field(\"foo_i\"))', " +
+             "  f2:'min(field($custom_req_param))', " +
+             // with fieldName and query de-reference
+             "  s1:'min(foo_i)', " +
+             "  s2:'min($custom_req_param)', " +
+             "  s3:{ func:'min($custom_req_param)' }, " +
+             "  s4:{ type:func, func:'min($custom_req_param)' }, " +
+             "  s5:{ type:func, func:'min($custom_local_param)', custom_local_param:foo_i }, " +
+             "  s6:{ type:func, func:'min($overridden_param)', overridden_param:foo_i }, " +
+             // test the test...
+             "  diff:'min(field(\"bar_i\"))'," +
+             "}"));
          
       final Map<String, AggValueSource> stats = fr.getFacetStats();
-      assertEquals(8, stats.size());
+      assertEquals(9, stats.size());
       
       for (Map.Entry<String,AggValueSource> entry : stats.entrySet()) {
         final String key = entry.getKey();
         final AggValueSource agg = entry.getValue();
         
         assertEquals("name of " + key, "min", agg.name());
-        assertThat("type of " + key, agg, instanceOf(SimpleAggValueSource.class));
+        MatcherAssert.assertThat("type of " + key, agg, instanceOf(SimpleAggValueSource.class));
         SimpleAggValueSource sagg = (SimpleAggValueSource) agg;
-        assertThat("vs of " + key, sagg.getArg(), instanceOf(IntFieldSource.class));
-        
-        if ("diff".equals(key)) {
-          assertEquals("field of " + key, "bar_i", ((IntFieldSource)sagg.getArg()).getField());
-          assertFalse("diff.equals(s1) ?!?!", agg.equals(stats.get("s1")));
-          
-        } else {
+
+        if (key.startsWith("f")) { // value source as arg to min
+          MatcherAssert.assertThat("vs of " + key, sagg.getArg(), instanceOf(IntFieldSource.class));
           assertEquals("field of " + key, "foo_i", ((IntFieldSource)sagg.getArg()).getField());
-          
+          assertEquals(key + ".equals(f1)", agg, stats.get("f1"));
+        } else if (key.startsWith("s")) { // field as arg to min
+          MatcherAssert.assertThat("vs of " + key, sagg.getArg(), instanceOf(FieldNameValueSource.class));
+          assertEquals("field of " + key, "foo_i", ((FieldNameValueSource)sagg.getArg()).getFieldName());
           assertEquals(key + ".equals(s1)", agg, stats.get("s1"));
           assertEquals("s1.equals("+key+")", stats.get("s1"), agg);
+        } else if ("diff".equals(key)) {
+          assertEquals("field of " + key, "bar_i", ((IntFieldSource)sagg.getArg()).getField());
+          assertNotEquals("diff.equals(s1) ?!?!", agg, stats.get("f1"));
         }
       }
     }
@@ -113,7 +117,7 @@ public class TestJsonFacetsStatsParsing extends SolrTestCaseJ4 {
       assertEquals(1, stats.size());
       AggValueSource agg = stats.get("x");
       assertNotNull(agg);
-      assertThat(agg, instanceOf(DebugAgg.class));
+      MatcherAssert.assertThat(agg, instanceOf(DebugAgg.class));
       
       DebugAgg x = (DebugAgg)agg;
       assertEquals(new String[] {"abc", "xyz"}, x.localParams.getParams("foo"));