You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mk...@apache.org on 2020/12/07 10:51:28 UTC

[lucene-solr] branch branch_8x updated: SOLR-8673: Enable custom aggregate functions by opening up FacetContext (#2125)

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

mkhl pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 8d63769  SOLR-8673: Enable custom aggregate functions by opening up FacetContext (#2125)
8d63769 is described below

commit 8d63769795bf25fde29f1762047339d8107b550f
Author: Mikhail Khludnev <mk...@users.noreply.github.com>
AuthorDate: Mon Dec 7 02:51:07 2020 -0800

    SOLR-8673: Enable custom aggregate functions by opening up FacetContext (#2125)
    
    Prior to this change, it was not feasible to write a custom aggregate function in plugin code
    because the FacetContext class did not expose its fields for access outside its package. Also
    some of the useful abstract classes for SlotAcc were package-private too
    
    Co-authored-by: Tim Owen <ti...@brandwatch.com>
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/search/facet/FacetContext.java |  36 +++++++
 .../java/org/apache/solr/search/facet/SlotAcc.java |  20 ++--
 .../solr/search/function/AggValueSourceTest.java   | 106 +++++++++++++++++++++
 4 files changed, 154 insertions(+), 10 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9a4de19..5fedd29 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -33,6 +33,8 @@ Improvements
 
 * SOLR-15015 : Add interleaving algorithm parameter support in Learning To Rank (Alessandro Benedetti)
 
+* SOLR-8673: Make JSON Facets extendable (Tim Owen via Mikhail Khludnev)
+
 * SOLR-14965: metrics: Adds two metrics to the SolrCloud Overseer: solr_metrics_overseer_stateUpdateQueueSize
   and solr_metrics_overseer_collectionWorkQueueSize with corresponding entries in the the Prometheus exporter's
   default/stock configuration.  (Saatchi Bhalla, Megan Carey, Andrzej BiaƂecki, David Smiley)
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetContext.java b/solr/core/src/java/org/apache/solr/search/facet/FacetContext.java
index 86aa3ad..cef9d01 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetContext.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetContext.java
@@ -53,6 +53,42 @@ public class FacetContext {
     return (flags & IS_SHARD) != 0;
   }
 
+  public FacetProcessor<?> getFacetProcessor() {
+    return processor;
+  }
+
+  public Map<String, Object> getFacetInfo() {
+    return facetInfo;
+  }
+
+  public QueryContext getQueryContext() {
+    return qcontext;
+  }
+
+  public SolrQueryRequest getRequest() {
+    return req;
+  }
+
+  public SolrIndexSearcher getSearcher() {
+    return searcher;
+  }
+
+  public Query getFilter() {
+    return filter;
+  }
+
+  public DocSet getBase() {
+    return base;
+  }
+
+  public FacetContext getParent() {
+    return parent;
+  }
+
+  public int getFlags() {
+    return flags;
+  }
+
   /**
    * @param filter The filter for the bucket that resulted in this context/domain.  Can be null if this is the root context.
    * @param domain The resulting set of documents for this facet.
diff --git a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
index 0d49475..e89cfd8 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
@@ -278,7 +278,7 @@ public abstract class SlotAcc implements Closeable {
 
   // TODO: we should really have a decoupled value provider...
 // This would enhance reuse and also prevent multiple lookups of same value across diff stats
-  abstract static class FuncSlotAcc extends SlotAcc {
+  public abstract static class FuncSlotAcc extends SlotAcc {
     protected final ValueSource valueSource;
     protected FunctionValues values;
 
@@ -301,9 +301,9 @@ public abstract class SlotAcc implements Closeable {
 // double-slot-func -> func-slot -> slot -> acc
 // double-slot-func -> double-slot -> slot -> acc
 
-  abstract static class DoubleFuncSlotAcc extends FuncSlotAcc {
-    double[] result; // TODO: use DoubleArray
-    double initialValue;
+  public abstract static class DoubleFuncSlotAcc extends FuncSlotAcc {
+    protected double[] result; // TODO: use DoubleArray
+    protected double initialValue;
 
   public DoubleFuncSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
       this(values, fcontext, numSlots, 0);
@@ -339,9 +339,9 @@ public abstract class SlotAcc implements Closeable {
     }
   }
 
-  abstract static class LongFuncSlotAcc extends FuncSlotAcc {
-    long[] result;
-    long initialValue;
+  public abstract static class LongFuncSlotAcc extends FuncSlotAcc {
+    protected long[] result;
+    protected long initialValue;
 
   public LongFuncSlotAcc(ValueSource values, FacetContext fcontext, int numSlots, long initialValue) {
       super(values, fcontext, numSlots);
@@ -373,9 +373,9 @@ public abstract class SlotAcc implements Closeable {
     }
   }
 
-  abstract class IntSlotAcc extends SlotAcc {
-    int[] result; // use LongArray32
-    int initialValue;
+  public abstract static class IntSlotAcc extends SlotAcc {
+    protected int[] result; // use LongArray32
+    protected int initialValue;
 
   public IntSlotAcc(FacetContext fcontext, int numSlots, int initialValue) {
       super(fcontext);
diff --git a/solr/core/src/test/org/apache/solr/search/function/AggValueSourceTest.java b/solr/core/src/test/org/apache/solr/search/function/AggValueSourceTest.java
new file mode 100644
index 0000000..eca48e1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/function/AggValueSourceTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search.function;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.ConstValueSource;
+import org.apache.solr.SolrTestCase;
+import org.apache.solr.search.facet.FacetContext;
+import org.apache.solr.search.facet.FacetMerger;
+import org.apache.solr.search.facet.SimpleAggValueSource;
+import org.apache.solr.search.facet.SlotAcc;
+import org.junit.Test;
+
+/** Tests that AggValueSource can be extended outside its package */
+public class AggValueSourceTest extends SolrTestCase {
+
+  @Test
+  public void testCustomAgg() {
+    // All we're really interested in testing here is that the custom agg compiles and can be created
+    final CustomAggregate customAggregate = new CustomAggregate(new ConstValueSource(123.0f));
+    final FacetMerger facetMerger = customAggregate.createFacetMerger(0.0D);
+  }
+
+  static class CustomAggregate extends SimpleAggValueSource {
+
+    CustomAggregate(ValueSource vs) {
+      super("customagg", vs);
+    }
+
+    @Override
+    public SlotAcc createSlotAcc(FacetContext fcontext, long numDocs, int numSlots) {
+      // check we can get access to the request and searcher, via the context
+      if (fcontext.getRequest().getCore() != fcontext.getSearcher().getCore()) {
+        throw new IllegalStateException("Searcher and request out of sync");
+      }
+      return new CustomSlotAcc(getArg(), fcontext, numSlots);
+    }
+
+    static class CustomSlotAcc extends SlotAcc.DoubleFuncSlotAcc {
+
+      CustomSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
+        super(values, fcontext, numSlots);
+      }
+
+      @Override
+      public void collect(int doc, int slot, IntFunction<SlotContext> slotContext) throws IOException {
+        result[slot] += values.doubleVal(doc);
+      }
+
+      @Override
+      public Object getValue(int slot) {
+        if (fcontext.isShard()) {
+          // shard-specific logic here
+        }
+        return super.getValue(slot);
+      }
+    }
+
+    @Override
+    public FacetMerger createFacetMerger(Object prototype) {
+      return new FacetMerger() {
+        double total = 0.0D;
+
+        @Override
+        public void merge(Object facetResult, Context mcontext) {
+          total += (Double)facetResult;
+        }
+
+        @Override
+        public void finish(Context mcontext) { }
+
+        @Override
+        public Object getMergedResult() {
+          return total;
+        }
+      };
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    @Override
+    public String description() {
+      return "customagg()";
+    }
+  }
+}