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 2017/05/10 12:34:49 UTC

[02/50] [abbrv] lucene-solr:jira/solr-10290: SOLR-10547: consolidate MinAgg+MaxAgg, add min/max support for single valued string fields

SOLR-10547: consolidate MinAgg+MaxAgg, add min/max support for single valued string 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/1f6e30fe
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1f6e30fe
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1f6e30fe

Branch: refs/heads/jira/solr-10290
Commit: 1f6e30fef088c1629ad35ad149c0a655f105a414
Parents: 9396227
Author: yonik <yo...@apache.org>
Authored: Sat May 6 05:49:19 2017 -0400
Committer: Cassandra Targett <ca...@lucidworks.com>
Committed: Wed May 10 07:30:32 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/search/ValueSourceParser.java   |   7 +-
 .../org/apache/solr/search/facet/MaxAgg.java    |  57 -----
 .../org/apache/solr/search/facet/MinAgg.java    |  54 -----
 .../org/apache/solr/search/facet/MinMaxAgg.java | 236 +++++++++++++++++++
 .../org/apache/solr/search/facet/SlotAcc.java   |  34 ---
 .../solr/search/facet/TestJsonFacets.java       |  24 ++
 7 files changed, 266 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 38aac79..ff7bdcf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -85,6 +85,9 @@ New Features
 
 * SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke)
 
+* SOLR-10547: JSON Facet API: Implement support for single-valued string fields for min/max aggregations.
+  (yonik)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index b511656..517b86f 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -56,8 +56,7 @@ import org.apache.solr.search.facet.AggValueSource;
 import org.apache.solr.search.facet.AvgAgg;
 import org.apache.solr.search.facet.CountAgg;
 import org.apache.solr.search.facet.HLLAgg;
-import org.apache.solr.search.facet.MaxAgg;
-import org.apache.solr.search.facet.MinAgg;
+import org.apache.solr.search.facet.MinMaxAgg;
 import org.apache.solr.search.facet.PercentileAgg;
 import org.apache.solr.search.facet.StddevAgg;
 import org.apache.solr.search.facet.SumAgg;
@@ -1009,14 +1008,14 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
     addParser("agg_min", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) throws SyntaxError {
-        return new MinAgg(fp.parseValueSource());
+        return new MinMaxAgg("min", fp.parseValueSource());
       }
     });
 
     addParser("agg_max", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) throws SyntaxError {
-        return new MaxAgg(fp.parseValueSource());
+        return new MinMaxAgg("max", fp.parseValueSource());
       }
     });
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/core/src/java/org/apache/solr/search/facet/MaxAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/MaxAgg.java b/solr/core/src/java/org/apache/solr/search/facet/MaxAgg.java
deleted file mode 100644
index a66e968..0000000
--- a/solr/core/src/java/org/apache/solr/search/facet/MaxAgg.java
+++ /dev/null
@@ -1,57 +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.solr.search.facet;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.ValueSource;
-
-
-public class MaxAgg extends SimpleAggValueSource {
-  public MaxAgg(ValueSource vs) {
-    super("max", vs);
-  }
-
-  @Override
-  public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
-    return new MaxSlotAcc(getArg(), fcontext, numSlots);
-  }
-
-  @Override
-  public FacetMerger createFacetMerger(Object prototype) {
-    return new Merger();
-  }
-
-  private static class Merger extends FacetDoubleMerger {
-    double val = Double.NaN;
-
-    @Override
-    public void merge(Object facetResult, Context mcontext) {
-      double result = ((Number)facetResult).doubleValue();
-      if (result > val || Double.isNaN(val)) {
-        val = result;
-      }
-    }
-
-    @Override
-    protected double getDouble() {
-      return val;
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/core/src/java/org/apache/solr/search/facet/MinAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/MinAgg.java b/solr/core/src/java/org/apache/solr/search/facet/MinAgg.java
deleted file mode 100644
index 72c7581..0000000
--- a/solr/core/src/java/org/apache/solr/search/facet/MinAgg.java
+++ /dev/null
@@ -1,54 +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.solr.search.facet;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.ValueSource;
-
-public class MinAgg extends SimpleAggValueSource {
-  public MinAgg(ValueSource vs) {
-    super("min", vs);
-  }
-
-  @Override
-  public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
-    return new MinSlotAcc(getArg(), fcontext, numSlots);
-  }
-
-  @Override
-  public FacetMerger createFacetMerger(Object prototype) {
-    return new Merger();
-  }
-
-  private static class Merger extends FacetDoubleMerger {
-    double val = Double.NaN;
-
-    @Override
-    public void merge(Object facetResult, Context mcontext) {
-      double result = ((Number)facetResult).doubleValue();
-      if (result < val || Double.isNaN(val)) {
-        val = result;
-      }
-    }
-
-    @Override
-    protected double getDouble() {
-      return val;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java b/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
new file mode 100644
index 0000000..2c37863
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/facet/MinMaxAgg.java
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search.facet;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LongValues;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrFieldSource;
+
+public class MinMaxAgg extends SimpleAggValueSource {
+  final int minmax; // a multiplier to reverse the normal order of compare if this is max instead of min (i.e. max will be -1)
+
+  public MinMaxAgg(String minOrMax, ValueSource vs) {
+    super(minOrMax, vs);
+    minmax = "min".equals(name) ? 1 : -1;
+  }
+
+  @Override
+  public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
+    ValueSource vs = getArg();
+
+    if (vs instanceof StrFieldSource) {
+      String field = ((StrFieldSource) vs).getField();
+      SchemaField sf = fcontext.qcontext.searcher().getSchema().getField(field);
+      if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
+        if (sf.hasDocValues()) {
+          // dv
+        } else {
+          // uif
+        }
+      } else {
+        return new SingleValuedOrdAcc(fcontext, sf, numSlots);
+      }
+    }
+
+    // numeric functions
+    return new ValSlotAcc(vs, fcontext, numSlots);
+  }
+
+  @Override
+  public FacetMerger createFacetMerger(Object prototype) {
+    if (prototype instanceof Number)
+      return new NumericMerger();
+    else if (prototype instanceof Comparable) {
+      return new ComparableMerger();
+    } else {
+      throw new UnsupportedOperationException("min/max merge of " + prototype);
+    }
+  }
+
+  // TODO: can this be replaced by ComparableMerger?
+  private class NumericMerger extends FacetDoubleMerger {
+    double val = Double.NaN;
+
+    @Override
+    public void merge(Object facetResult, Context mcontext) {
+      double result = ((Number)facetResult).doubleValue();
+      if (Double.compare(result, val)*minmax < 0 || Double.isNaN(val)) {
+        val = result;
+      }
+    }
+
+    @Override
+    protected double getDouble() {
+      return val;
+    }
+  }
+
+  private class ComparableMerger extends FacetSortableMerger {
+    Comparable val;
+    @Override
+    public void merge(Object facetResult, Context mcontext) {
+      Comparable other = (Comparable)facetResult;
+      if (val == null) {
+        val = other;
+      } else {
+        if ( other.compareTo(val) * minmax < 0 ) {
+          val = other;
+        }
+      }
+    }
+
+    @Override
+    public Object getMergedResult() {
+      return val;
+    }
+
+    @Override
+    public int compareTo(FacetSortableMerger other, FacetRequest.SortDirection direction) {
+      // NOTE: we don't use the minmax multiplier here because we still want natural ordering between slots (i.e. min(field) asc and max(field) asc) both sort "A" before "Z")
+      return this.val.compareTo(((ComparableMerger)other).val);
+    }
+  }
+
+  class ValSlotAcc extends DoubleFuncSlotAcc {
+    public ValSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
+      super(values, fcontext, numSlots, Double.NaN);
+    }
+
+    @Override
+    public void collect(int doc, int slotNum) throws IOException {
+      double val = values.doubleVal(doc);
+      if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
+
+      double currVal = result[slotNum];
+      if (Double.compare(val, currVal) * minmax < 0 || Double.isNaN(currVal)) {
+        result[slotNum] = val;
+      }
+    }
+  }
+
+
+  abstract class OrdAcc extends SlotAcc {
+    final static int MISSING = -1;
+    SchemaField field;
+    int[] slotOrd;
+
+    public OrdAcc(FacetContext fcontext, SchemaField field, int numSlots) throws IOException {
+      super(fcontext);
+      this.field = field;
+      slotOrd = new int[numSlots];
+      if (MISSING != 0) Arrays.fill(slotOrd, MISSING);
+    }
+
+    abstract BytesRef lookupOrd(int ord) throws IOException;
+
+    @Override
+    public int compare(int slotA, int slotB) {
+      int a = slotOrd[slotA];
+      int b = slotOrd[slotB];
+      // NOTE: we don't use the minmax multiplier here because we still want natural ordering between slots (i.e. min(field) asc and max(field) asc) both sort "A" before "Z")
+      return a - b;  // TODO: we probably want sort-missing-last functionality
+    }
+
+    @Override
+    public Object getValue(int slotNum) throws IOException {
+      int globOrd = slotOrd[slotNum];
+      if (globOrd == MISSING) return null;
+      BytesRef term = lookupOrd(globOrd);
+      return field.getType().toObject(field, term);
+    }
+
+    @Override
+    public void reset() throws IOException {
+      Arrays.fill(slotOrd, MISSING);
+    }
+
+    @Override
+    public void resize(Resizer resizer) {
+      slotOrd = resizer.resize(slotOrd, MISSING);
+    }
+  }
+
+
+  class SingleValuedOrdAcc extends OrdAcc {
+    SortedDocValues topLevel;
+    SortedDocValues[] subDvs;
+    MultiDocValues.OrdinalMap ordMap;
+    LongValues toGlobal;
+    SortedDocValues subDv;
+
+    public SingleValuedOrdAcc(FacetContext fcontext, SchemaField field, int numSlots) throws IOException {
+      super(fcontext, field, numSlots);
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      topLevel = FieldUtil.getSortedDocValues(fcontext.qcontext, field, null);
+      if (topLevel instanceof MultiDocValues.MultiSortedDocValues) {
+        ordMap = ((MultiDocValues.MultiSortedDocValues)topLevel).mapping;
+        subDvs = ((MultiDocValues.MultiSortedDocValues)topLevel).values;
+      } else {
+        ordMap = null;
+        subDvs = null;
+      }
+    }
+
+    @Override
+    protected BytesRef lookupOrd(int ord) throws IOException {
+      return topLevel.lookupOrd(ord);
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      if (topLevel == null) {
+        reset();
+      }
+      super.setNextReader(readerContext);
+      if (subDvs != null) {
+        subDv = subDvs[readerContext.ord];
+        toGlobal = ordMap.getGlobalOrds(readerContext.ord);
+      } else {
+        assert readerContext.ord==0 || topLevel.getValueCount() == 0;
+        subDv = topLevel;
+      }
+    }
+
+    @Override
+    public void collect(int doc, int slotNum) throws IOException {
+      if (doc > subDv.docID()) {
+        subDv.advance(doc);
+      }
+      if (doc == subDv.docID()) {
+        int segOrd = subDv.ordValue();
+        int ord = toGlobal==null ? segOrd : (int)toGlobal.get(segOrd);
+        if ((ord - slotOrd[slotNum]) * minmax < 0 || slotOrd[slotNum]==MISSING) {
+          slotOrd[slotNum] = ord;
+        }
+      }
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java b/solr/core/src/java/org/apache/solr/search/facet/SlotAcc.java
index f5cdf82..4f8552b 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
@@ -280,40 +280,6 @@ class SumsqSlotAcc extends DoubleFuncSlotAcc {
   }
 }
 
-class MinSlotAcc extends DoubleFuncSlotAcc {
-  public MinSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
-    super(values, fcontext, numSlots, Double.NaN);
-  }
-
-  @Override
-  public void collect(int doc, int slotNum) throws IOException {
-    double val = values.doubleVal(doc);
-    if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
-
-    double currMin = result[slotNum];
-    if (!(val >= currMin)) { // val>=currMin will be false for staring value: val>=NaN
-      result[slotNum] = val;
-    }
-  }
-}
-
-class MaxSlotAcc extends DoubleFuncSlotAcc {
-  public MaxSlotAcc(ValueSource values, FacetContext fcontext, int numSlots) {
-    super(values, fcontext, numSlots, Double.NaN);
-  }
-
-  @Override
-  public void collect(int doc, int slotNum) throws IOException {
-    double val = values.doubleVal(doc);
-    if (val == 0 && !values.exists(doc)) return; // depend on fact that non existing values return 0 for func query
-
-    double currMax = result[slotNum];
-    if (!(val <= currMax)) { // reversed order to handle NaN
-      result[slotNum] = val;
-    }
-  }
-
-}
 
 class AvgSlotAcc extends DoubleFuncSlotAcc {
   int[] counts;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1f6e30fe/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 4ca435c..87504f6 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
@@ -1372,6 +1372,30 @@ public class TestJsonFacets extends SolrTestCaseHS {
 
     );
 
+
+    // test min/max of string field
+    if (where_s.equals("where_s") || where_s.equals("where_sd")) {  // supports only single valued currently...
+      client.testJQ(params(p, "q", "*:* -(+${cat_s}:A +${where_s}:NJ)"  // make NY the only value in bucket A
+          , "json.facet", "{" +
+              "  f1:{type:terms, field:'${cat_s}', facet:{min:'min(${where_s})', max:'max(${where_s})'}   }" +
+              ", f2:{type:terms, field:'${cat_s}', facet:{min:'min(${where_s})', max:'max(${where_s})'} , sort:'min desc'}" +
+              ", f3:{type:terms, field:'${cat_s}', facet:{min:'min(${where_s})', max:'max(${where_s})'} , sort:'min asc'}" +
+              ", f4:{type:terms, field:'${cat_s}', facet:{min:'min(${super_s})', max:'max(${super_s})'} , sort:'max asc'}" +
+              ", f5:{type:terms, field:'${cat_s}', facet:{min:'min(${super_s})', max:'max(${super_s})'} , sort:'max desc'}" +
+              "}"
+          )
+          , "facets=={ count:5, " +
+              " f1:{ buckets:[{val:B, count:3, min:NJ, max:NY}, {val:A, count:1, min:NY, max:NY}]}" +
+              ",f2:{ buckets:[{val:A, count:1, min:NY, max:NY}, {val:B, count:3, min:NJ, max:NY}]}" +
+              ",f3:{ buckets:[{val:B, count:3, min:NJ, max:NY}, {val:A, count:1, min:NY, max:NY}]}" +
+              ",f4:{ buckets:[{val:B, count:3, min:batman, max:superman}, {val:A, count:1, min:zodiac, max:zodiac}]}" +
+              ",f5:{ buckets:[{val:A, count:1, min:zodiac, max:zodiac}, {val:B, count:3, min:batman, max:superman}]}" +
+              " } "
+      );
+
+
+    }
+
   }
 
   @Test