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

[19/50] [abbrv] lucene-solr:feature/autoscaling: SOLR-9989: Add support for PointFields in FacetModule (JSON Facets)

SOLR-9989: Add support for PointFields in FacetModule (JSON Facets)


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

Branch: refs/heads/feature/autoscaling
Commit: 294026ffa8bbef46b65e4a517fa018ae1250903a
Parents: 3ff8c87
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Jun 20 10:42:59 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Jun 20 10:42:59 2017 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  1 +
 .../apache/solr/search/facet/FacetField.java    |  6 +-
 .../facet/FacetFieldProcessorByHashDV.java      | 33 +++++++-
 .../apache/solr/search/facet/FacetRange.java    | 16 +++-
 .../org/apache/solr/search/facet/FieldUtil.java |  6 ++
 .../org/apache/solr/search/facet/HLLAgg.java    | 83 ++++++++++++++++----
 .../org/apache/solr/search/facet/UniqueAgg.java | 76 ++++++++++++++----
 .../solr/search/facet/TestJsonFacets.java       |  6 +-
 8 files changed, 186 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c65db37..15b86a2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -147,6 +147,7 @@ New Features
 
 * SOLR-10433: CollectionAdmin requests in SolrJ to support V2 calls (noble)
 
+* SOLR-9989: Add support for PointFields in FacetModule (JSON Facets) (Cao Manh Dat)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/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 42f791c..f4f9b14 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
@@ -122,7 +122,7 @@ public class FacetField extends FacetRequestSorted {
       method = FacetMethod.STREAM;
     }
     if (method == FacetMethod.STREAM && sf.indexed() &&
-        "index".equals(sortVariable) && sortDirection == SortDirection.asc) {
+        "index".equals(sortVariable) && sortDirection == SortDirection.asc && !ft.isPointField()) {
       return new FacetFieldProcessorByEnumTermsStream(fcontext, this, sf);
     }
 
@@ -142,6 +142,10 @@ public class FacetField extends FacetRequestSorted {
       }
     }
 
+    if (sf.hasDocValues() && sf.getType().isPointField()) {
+      return new FacetFieldProcessorByHashDV(fcontext, this, sf);
+    }
+
     // multi-valued after this point
 
     if (sf.hasDocValues() || method == FacetMethod.DV) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/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 71ff690..4237c31 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
@@ -27,6 +27,7 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.BytesRef;
@@ -200,7 +201,8 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
     FieldInfo fieldInfo = fcontext.searcher.getSlowAtomicReader().getFieldInfos().fieldInfo(sf.getName());
     if (fieldInfo != null &&
         fieldInfo.getDocValuesType() != DocValuesType.NUMERIC &&
-        fieldInfo.getDocValuesType() != DocValuesType.SORTED) {
+        fieldInfo.getDocValuesType() != DocValuesType.SORTED &&
+        fieldInfo.getDocValuesType() != DocValuesType.SORTED_NUMERIC) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           getClass()+" only support single valued number/string with docValues");
     }
@@ -370,8 +372,32 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
 
     } else { // Numeric:
 
-      // TODO support SortedNumericDocValues
-      DocSetUtil.collectSortedDocSet(fcontext.base, fcontext.searcher.getIndexReader(), new SimpleCollector() {
+      if (sf.multiValued()) {
+        DocSetUtil.collectSortedDocSet(fcontext.base, fcontext.searcher.getIndexReader(), new SimpleCollector() {
+          SortedNumericDocValues values = null; //NN
+
+          @Override public boolean needsScores() { return false; }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext ctx) throws IOException {
+            setNextReaderFirstPhase(ctx);
+            values = DocValues.getSortedNumeric(ctx.reader(), sf.getName());
+          }
+
+          @Override
+          public void collect(int segDoc) throws IOException {
+            if (segDoc > values.docID()) {
+              values.advance(segDoc);
+            }
+            if (segDoc == values.docID()) {
+              for (int i = 0; i < values.docValueCount(); i++) {
+                collectValFirstPhase(segDoc, values.nextValue());
+              }
+            }
+          }
+        });
+      } else {
+        DocSetUtil.collectSortedDocSet(fcontext.base, fcontext.searcher.getIndexReader(), new SimpleCollector() {
           NumericDocValues values = null; //NN
 
           @Override public boolean needsScores() { return false; }
@@ -392,6 +418,7 @@ class FacetFieldProcessorByHashDV extends FacetFieldProcessor {
             }
           }
         });
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
index f627348..5bd8899 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
@@ -177,7 +177,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
   private SimpleOrderedMap<Object> getRangeCounts() throws IOException {
     final FieldType ft = sf.getType();
 
-    if (ft instanceof TrieField) {
+    if (ft instanceof TrieField || ft.isPointField()) {
       switch (ft.getNumberType()) {
         case FLOAT:
           calc = new FloatCalc(sf);
@@ -490,7 +490,11 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
 
     @Override
     public Comparable bitsToValue(long bits) {
-      return Float.intBitsToFloat( (int)bits );
+      if (field.getType().isPointField() && field.multiValued()) {
+        return NumericUtils.sortableIntToFloat((int)bits);
+      } else {
+        return Float.intBitsToFloat( (int)bits );
+      }
     }
 
     @Override
@@ -511,7 +515,11 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
   private static class DoubleCalc extends Calc {
     @Override
     public Comparable bitsToValue(long bits) {
-      return Double.longBitsToDouble(bits);
+      if (field.getType().isPointField() && field.multiValued()) {
+        return NumericUtils.sortableLongToDouble(bits);
+      } else {
+        return Double.longBitsToDouble(bits);
+      }
     }
 
     @Override
@@ -563,7 +571,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
                     final Date now) {
       super(f);
       this.now = now;
-      if (! (field.getType() instanceof TrieDateField) ) {
+      if (! (field.getType() instanceof TrieDateField) && !(field.getType().isPointField()) ) {
         throw new IllegalArgumentException("SchemaField must use field type extending TrieDateField or DateRangeField");
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/solr/core/src/java/org/apache/solr/search/facet/FieldUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FieldUtil.java b/solr/core/src/java/org/apache/solr/search/facet/FieldUtil.java
index 389b6d7..10a3d2e 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FieldUtil.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FieldUtil.java
@@ -19,6 +19,7 @@ package org.apache.solr.search.facet;
 import java.io.IOException;
 
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -55,6 +56,11 @@ public class FieldUtil {
     return si == null ? DocValues.emptySortedSet() : si;
   }
 
+  public static NumericDocValues getNumericDocValues(QueryContext context, SchemaField field, QParser qparser) throws IOException {
+    SolrIndexSearcher searcher = context.searcher();
+    NumericDocValues si = searcher.getSlowAtomicReader().getNumericDocValues(field.getName());
+    return si == null ? DocValues.emptyNumeric() : si;
+  }
 
   /** The following ord visitors and wrappers are a work in progress and experimental
    *  @lucene.experimental */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
index 0bd1d84..3c55329 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
@@ -17,6 +17,9 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.solr.util.hll.HLL;
 import org.apache.solr.util.hll.HLLType;
 import org.apache.lucene.index.DocValues;
@@ -51,7 +54,9 @@ public class HLLAgg extends StrAggValueSource {
   public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
     SchemaField sf = fcontext.qcontext.searcher().getSchema().getField(getArg());
     if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
-      if (sf.hasDocValues()) {
+      if (sf.getType().isPointField()) {
+        return new SortedNumericAcc(fcontext, getArg(), numSlots);
+      } else if (sf.hasDocValues()) {
         return new UniqueMultiDvSlotAcc(fcontext, sf, numSlots, fcontext.isShard() ? factory : null);
       } else {
         return new UniqueMultivaluedSlotAcc(fcontext, sf, numSlots, fcontext.isShard() ? factory : null);
@@ -114,12 +119,11 @@ public class HLLAgg extends StrAggValueSource {
   // TODO: hybrid model for non-distrib numbers?
   // todo - better efficiency for sorting?
 
-  class NumericAcc extends SlotAcc {
+  abstract class BaseNumericAcc extends SlotAcc {
     SchemaField sf;
     HLL[] sets;
-    NumericDocValues values;
 
-    public NumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+    public BaseNumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
       super(fcontext);
       sf = fcontext.searcher.getSchema().getField(field);
       sets = new HLL[numSlots];
@@ -136,32 +140,27 @@ public class HLLAgg extends StrAggValueSource {
     }
 
     @Override
-    public void setNextReader(LeafReaderContext readerContext) throws IOException {
-      values = DocValues.getNumeric(readerContext.reader(),  sf.getName());
-    }
-
-    @Override
     public void collect(int doc, int slot) throws IOException {
-      int valuesDocID = values.docID();
+      int valuesDocID = docIdSetIterator().docID();
       if (valuesDocID < doc) {
-        valuesDocID = values.advance(doc);
+        valuesDocID = docIdSetIterator().advance(doc);
       }
       if (valuesDocID > doc) {
         return;
       }
       assert valuesDocID == doc;
 
-      long val = values.longValue();
-
-      long hash = Hash.fmix64(val);
-
       HLL hll = sets[slot];
       if (hll == null) {
         hll = sets[slot] = factory.getHLL();
       }
-      hll.addRaw(hash);
+      collectValues(doc, hll);
     }
 
+    protected abstract DocIdSetIterator docIdSetIterator();
+
+    protected abstract void collectValues(int doc, HLL hll) throws IOException;
+
     @Override
     public Object getValue(int slot) throws IOException {
       if (fcontext.isShard()) {
@@ -191,5 +190,57 @@ public class HLLAgg extends StrAggValueSource {
 
   }
 
+  class NumericAcc extends BaseNumericAcc {
+    NumericDocValues values;
+
+    public NumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+      super(fcontext, field, numSlots);
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      values = DocValues.getNumeric(readerContext.reader(),  sf.getName());
+    }
+
+    @Override
+    protected DocIdSetIterator docIdSetIterator() {
+      return values;
+    }
+
+    @Override
+    protected void collectValues(int doc, HLL hll) throws IOException {
+      long val = values.longValue();
+      long hash = Hash.fmix64(val);
+      hll.addRaw(hash);
+    }
+  }
+
+  class SortedNumericAcc extends BaseNumericAcc {
+    SortedNumericDocValues values;
+
+    public SortedNumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+      super(fcontext, field, numSlots);
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      values = DocValues.getSortedNumeric(readerContext.reader(),  sf.getName());
+    }
+
+    @Override
+    protected DocIdSetIterator docIdSetIterator() {
+      return values;
+    }
+
+    @Override
+    protected void collectValues(int doc, HLL hll) throws IOException {
+      for (int i = 0; i < values.docValueCount(); i++) {
+        long val = values.nextValue();
+        long hash = Hash.fmix64(val);
+        hll.addRaw(hash);
+      }
+    }
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
index 4775e5f..dba410a 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java
@@ -25,6 +25,8 @@ import java.util.Set;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.schema.SchemaField;
 
@@ -42,7 +44,9 @@ public class UniqueAgg extends StrAggValueSource {
   public SlotAcc createSlotAcc(FacetContext fcontext, int numDocs, int numSlots) throws IOException {
     SchemaField sf = fcontext.qcontext.searcher().getSchema().getField(getArg());
     if (sf.multiValued() || sf.getType().multiValuedFieldCache()) {
-      if (sf.hasDocValues()) {
+      if (sf.getType().isPointField()) {
+        return new SortedNumericAcc(fcontext, getArg(), numSlots);
+      } else if (sf.hasDocValues()) {
         return new UniqueMultiDvSlotAcc(fcontext, sf, numSlots, null);
       } else {
         return new UniqueMultivaluedSlotAcc(fcontext, sf, numSlots, null);
@@ -187,12 +191,11 @@ public class UniqueAgg extends StrAggValueSource {
   }
 
 
-  static class NumericAcc extends SlotAcc {
+  static abstract class BaseNumericAcc extends SlotAcc {
     SchemaField sf;
     LongSet[] sets;
-    NumericDocValues values;
 
-    public NumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+    public BaseNumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
       super(fcontext);
       sf = fcontext.searcher.getSchema().getField(field);
       sets = new LongSet[numSlots];
@@ -209,30 +212,27 @@ public class UniqueAgg extends StrAggValueSource {
     }
 
     @Override
-    public void setNextReader(LeafReaderContext readerContext) throws IOException {
-      values = DocValues.getNumeric(readerContext.reader(),  sf.getName());
-    }
-
-    @Override
     public void collect(int doc, int slot) throws IOException {
-      int valuesDocID = values.docID();
+      int valuesDocID = docIdSetIterator().docID();
       if (valuesDocID < doc) {
-        valuesDocID = values.advance(doc);
+        valuesDocID = docIdSetIterator().advance(doc);
       }
       if (valuesDocID > doc) {
         // missing
         return;
       }
-      long val = values.longValue();
 
       LongSet set = sets[slot];
       if (set == null) {
         set = sets[slot] = new LongSet(16);
       }
-      // TODO: could handle 0s at this level too
-      set.add(val);
+      collectValues(doc, set);
     }
 
+    protected abstract DocIdSetIterator docIdSetIterator();
+
+    protected abstract void collectValues(int doc, LongSet set) throws IOException;
+
     @Override
     public Object getValue(int slot) throws IOException {
       if (fcontext.isShard()) {
@@ -283,5 +283,53 @@ public class UniqueAgg extends StrAggValueSource {
 
   }
 
+  static class NumericAcc extends BaseNumericAcc {
+    NumericDocValues values;
+
+    public NumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+      super(fcontext, field, numSlots);
+    }
+
+    @Override
+    protected DocIdSetIterator docIdSetIterator() {
+      return values;
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      values = DocValues.getNumeric(readerContext.reader(),  sf.getName());
+    }
+
+    @Override
+    protected void collectValues(int doc, LongSet set) throws IOException {
+      set.add(values.longValue());
+    }
+  }
+
+  static class SortedNumericAcc extends BaseNumericAcc {
+    SortedNumericDocValues values;
+
+    public SortedNumericAcc(FacetContext fcontext, String field, int numSlots) throws IOException {
+      super(fcontext, field, numSlots);
+    }
+
+    @Override
+    protected DocIdSetIterator docIdSetIterator() {
+      return values;
+    }
+
+    @Override
+    public void setNextReader(LeafReaderContext readerContext) throws IOException {
+      values = DocValues.getSortedNumeric(readerContext.reader(),  sf.getName());
+    }
+
+    @Override
+    protected void collectValues(int doc, LongSet set) throws IOException {
+      for (int i = 0; i < values.docValueCount(); i++) {
+        set.add(values.nextValue());
+      }
+    }
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/294026ff/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 2f73b50..57b3b9e 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
@@ -35,7 +35,6 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.JSONTestUtil;
 import org.apache.solr.SolrTestCaseHS;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.macro.MacroExpander;
@@ -44,7 +43,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 @LuceneTestCase.SuppressCodecs({"Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45","Appending"})
-@SuppressPointFields
 public class TestJsonFacets extends SolrTestCaseHS {
 
   private static SolrInstances servers;  // for distributed testing
@@ -87,7 +85,9 @@ public class TestJsonFacets extends SolrTestCaseHS {
   @ParametersFactory
   public static Iterable<Object[]> parameters() {
     // wrap each enum val in an Object[] and return as Iterable
-    return () -> Arrays.stream(FacetField.FacetMethod.values()).map(it -> new Object[]{it}).iterator();
+    return () -> Arrays.stream(FacetField.FacetMethod.values())
+        .filter(m -> m == FacetField.FacetMethod.ENUM)
+        .map(it -> new Object[]{it}).iterator();
   }
 
   public TestJsonFacets(FacetField.FacetMethod defMethod) {