You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2014/09/12 00:05:28 UTC

svn commit: r1624406 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/component/ solr/core/src/java/org/apache/solr/request/ solr/core/src/test/org/apache/solr/handler/component/

Author: tflobbe
Date: Thu Sep 11 22:05:27 2014
New Revision: 1624406

URL: http://svn.apache.org/r1624406
Log:
SOLR-6452: StatsComponent's stat 'missing' will work on fields with docValues=true and indexed=false

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesStats.java   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1624406&r1=1624405&r2=1624406&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Thu Sep 11 22:05:27 2014
@@ -106,6 +106,9 @@ Bug Fixes
 
 * SOLR-6493: Fix fq exclusion via "ex" local param in multivalued stats.field (hossman)
 
+* SOLR-6452: StatsComponent's stat 'missing' will work on fields with docValues=true and 
+  indexed=false (Xu Zhang via Tomás Fernández Löbbe)
+
 * SOLR-6499: Log warning about multiple update request handlers
   (Noble Paul, Andreas Hubold, hossman)
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java?rev=1624406&r1=1624405&r2=1624406&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/FieldFacetStats.java Thu Sep 11 22:05:27 2014
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -51,7 +52,7 @@ public class FieldFacetStats {
   final boolean calcDistinct;
 
   public final Map<String, StatsValues> facetStatsValues;
-
+  private final Map<Integer, Integer> missingStats;
   List<HashMap<String, Integer>> facetStatsTerms;
 
   final AtomicReader topLevelReader;
@@ -73,6 +74,7 @@ public class FieldFacetStats {
 
     facetStatsValues = new HashMap<>();
     facetStatsTerms = new ArrayList<>();
+    missingStats = new HashMap<>();
   }
 
   private StatsValues getStatsValues(String key) throws IOException {
@@ -100,8 +102,10 @@ public class FieldFacetStats {
     if (topLevelSortedValues == null) {
       topLevelSortedValues = FieldCache.DEFAULT.getTermsIndex(topLevelReader, name);
     }
+ 
     
     int term = topLevelSortedValues.getOrd(docID);
+    
     int arrIdx = term;
     if (arrIdx >= 0 && arrIdx < topLevelSortedValues.getValueCount()) {
       final String key;
@@ -113,6 +117,8 @@ public class FieldFacetStats {
       while (facetStatsTerms.size() <= statsTermNum) {
         facetStatsTerms.add(new HashMap<String, Integer>());
       }
+      
+      
       final Map<String, Integer> statsTermCounts = facetStatsTerms.get(statsTermNum);
       Integer statsTermCount = statsTermCounts.get(key);
       if (statsTermCount == null) {
@@ -122,6 +128,7 @@ public class FieldFacetStats {
       }
       return true;
     }
+    
     return false;
   }
 
@@ -132,8 +139,7 @@ public class FieldFacetStats {
     while (facetStatsTerms.size() <= statsTermNum) {
       facetStatsTerms.add(new HashMap<String, Integer>());
     }
-    for (Map.Entry<String, Integer> stringIntegerEntry : facetStatsTerms.get(statsTermNum).entrySet()) {
-      Map.Entry pairs = (Map.Entry) stringIntegerEntry;
+    for (Map.Entry<String, Integer> pairs : facetStatsTerms.get(statsTermNum).entrySet()) {
       String key = (String) pairs.getKey();
       StatsValues facetStats = facetStatsValues.get(key);
       if (facetStats == null) {
@@ -156,6 +162,35 @@ public class FieldFacetStats {
     }
   }
 
+  public void facetMissingNum(int docID) throws IOException {
+    if (topLevelSortedValues == null) {
+      topLevelSortedValues = DocValues.getSorted(topLevelReader, name);
+    }
+    
+    int ord = topLevelSortedValues.getOrd(docID);
+    if (ord != -1) {
+      Integer missingCount = missingStats.get(ord);
+      if (missingCount == null) {
+        missingStats.put(ord, 1);
+      } else {
+        missingStats.put(ord, missingCount + 1);
+      }
+    }
+  }
+  
+  public void accumulateMissing() throws IOException {
+    StatsValues statsValue;
+    
+    for (Map.Entry<Integer, Integer> entry : missingStats.entrySet()) {
+      if (entry.getKey() >= 0) {
+        String key = topLevelSortedValues.lookupOrd(entry.getKey()).utf8ToString();
+        if ((statsValue = facetStatsValues.get(key)) != null) {
+          statsValue.addMissing(entry.getValue());
+        }
+      }
+    }
+    return;
+  }
 }
 
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesStats.java?rev=1624406&r1=1624405&r2=1624406&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesStats.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesStats.java Thu Sep 11 22:05:27 2014
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
 import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
@@ -32,10 +31,9 @@ import org.apache.lucene.index.SortedSet
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongValues;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.component.FieldFacetStats;
 import org.apache.solr.handler.component.StatsValues;
 import org.apache.solr.handler.component.StatsValuesFactory;
@@ -62,11 +60,17 @@ public class DocValuesStats {
     //Initialize facetstats, if facets have been passed in
     final FieldFacetStats[] facetStats = new FieldFacetStats[facet.length];
     int upto = 0;
+       
     for (String facetField : facet) {
+      SchemaField fsf = searcher.getSchema().getField(facetField);
+      if ( fsf.multiValued()) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Stats can only facet on single-valued fields, not: " + facetField );
+      }
+      
       SchemaField facetSchemaField = searcher.getSchema().getField(facetField);
       facetStats[upto++] = new FieldFacetStats(searcher, facetField, schemaField, facetSchemaField, calcDistinct);
     }
-    
     // TODO: remove multiValuedFieldCache(), check dv type / uninversion type?
     final boolean multiValued = schemaField.multiValued() || ft.multiValuedFieldCache();
 
@@ -74,6 +78,7 @@ public class DocValuesStats {
     OrdinalMap ordinalMap = null; // for mapping per-segment ords to global ones
     if (multiValued) {
       si = searcher.getAtomicReader().getSortedSetDocValues(fieldName);
+      
       if (si instanceof MultiSortedSetDocValues) {
         ordinalMap = ((MultiSortedSetDocValues)si).mapping;
       }
@@ -90,26 +95,27 @@ public class DocValuesStats {
     if (si.getValueCount() >= Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("Currently this stats method is limited to " + Integer.MAX_VALUE + " unique terms");
     }
-
-    DocSet missing = docs.andNot( searcher.getDocSet(new TermRangeQuery(fieldName, null, null, false, false)));
-
-    final int nTerms = (int) si.getValueCount();   
     
+    int missingDocCountTotal = 0;
+    final int nTerms = (int) si.getValueCount();    
     // count collection array only needs to be as big as the number of terms we are
     // going to collect counts for.
     final int[] counts = new int[nTerms];
     
     Filter filter = docs.getTopFilter();
     List<AtomicReaderContext> leaves = searcher.getTopReaderContext().leaves();
+    
     for (int subIndex = 0; subIndex < leaves.size(); subIndex++) {
       AtomicReaderContext leaf = leaves.get(subIndex);
       DocIdSet dis = filter.getDocIdSet(leaf, null); // solr docsets already exclude any deleted docs
       DocIdSetIterator disi = null;
+      
       if (dis != null) {
         disi = dis.iterator();
       }
       if (disi != null) {
         int docBase = leaf.docBase;
+        
         if (multiValued) {
           SortedSetDocValues sub = leaf.reader().getSortedSetDocValues(fieldName);
           if (sub == null) {
@@ -118,23 +124,23 @@ public class DocValuesStats {
           final SortedDocValues singleton = DocValues.unwrapSingleton(sub);
           if (singleton != null) {
             // some codecs may optimize SORTED_SET storage for single-valued fields
-            accumSingle(counts, docBase, facetStats, singleton, disi, subIndex, ordinalMap);
+            missingDocCountTotal += accumSingle(counts, docBase, facetStats, singleton, disi, subIndex, ordinalMap);
           } else {
-            accumMulti(counts, docBase, facetStats, sub, disi, subIndex, ordinalMap);
+            missingDocCountTotal += accumMulti(counts, docBase, facetStats, sub, disi, subIndex, ordinalMap);
           }
         } else {
           SortedDocValues sub = leaf.reader().getSortedDocValues(fieldName);
           if (sub == null) {
             sub = DocValues.emptySorted();
           }
-          accumSingle(counts, docBase, facetStats, sub, disi, subIndex, ordinalMap);
+          missingDocCountTotal += accumSingle(counts, docBase, facetStats, sub, disi, subIndex, ordinalMap);
         }
       }
     }
-    
     // add results in index order
     for (int ord = 0; ord < counts.length; ord++) {
       int count = counts[ord];
+
       if (count > 0) {
         final BytesRef value = si.lookupOrd(ord);
         res.accumulate(value, count);
@@ -143,26 +149,23 @@ public class DocValuesStats {
         }
       }
     }
-
-    res.addMissing(missing.size());
+    res.addMissing(missingDocCountTotal);
+    
     if (facetStats.length > 0) {
       for (FieldFacetStats f : facetStats) {
-        Map<String, StatsValues> facetStatsValues = f.facetStatsValues;
-        FieldType facetType = searcher.getSchema().getFieldType(f.name);
-        for (Map.Entry<String,StatsValues> entry : facetStatsValues.entrySet()) {
-          String termLabel = entry.getKey();
-          int missingCount = searcher.numDocs(new TermQuery(new Term(f.name, facetType.toInternal(termLabel))), missing);
-          entry.getValue().addMissing(missingCount);
-        }
+        Map<String,StatsValues> facetStatsValues = f.facetStatsValues;
+        f.accumulateMissing();
         res.addFacet(f.name, facetStatsValues);
       }
     }
+    
     return res;
   }
 
   /** accumulates per-segment single-valued stats */
-  static void accumSingle(int counts[], int docBase, FieldFacetStats[] facetStats, SortedDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
+  static int accumSingle(int counts[], int docBase, FieldFacetStats[] facetStats, SortedDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
     final LongValues ordMap = map == null ? null : map.getGlobalOrds(subIndex);
+    int missingDocCount = 0;
     int doc;
     while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
       int term = si.getOrd(doc);
@@ -174,18 +177,29 @@ public class DocValuesStats {
         for (FieldFacetStats f : facetStats) {
           f.facetTermNum(docBase + doc, term);
         }
+      }else{
+        for (FieldFacetStats f : facetStats) {
+          f.facetMissingNum(docBase + doc);
+        }
+        
+        missingDocCount++;
       }
     }
+    return missingDocCount;
   }
   
   /** accumulates per-segment multi-valued stats */
-  static void accumMulti(int counts[], int docBase, FieldFacetStats[] facetStats, SortedSetDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
+  
+  static int accumMulti(int counts[], int docBase, FieldFacetStats[] facetStats, SortedSetDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
     final LongValues ordMap = map == null ? null : map.getGlobalOrds(subIndex);
+    int missingDocCount = 0;
     int doc;
     while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
       si.setDocument(doc);
       long ord;
+      boolean emptyTerm = true;
       while ((ord = si.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        emptyTerm = false;
         int term = (int) ord;
         if (map != null) {
           term = (int) ordMap.get(term);
@@ -195,6 +209,15 @@ public class DocValuesStats {
           f.facetTermNum(docBase + doc, term);
         }
       }
+      if (emptyTerm){
+        for (FieldFacetStats f : facetStats) {
+          f.facetMissingNum(docBase + doc);
+        }
+        
+        missingDocCount++;
+      }
     }
+    
+    return missingDocCount;
   }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java?rev=1624406&r1=1624405&r2=1624406&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java Thu Sep 11 22:05:27 2014
@@ -63,9 +63,8 @@ public class StatsComponentTest extends 
     for (String f : new String[] {
             "stats_i","stats_l","stats_f","stats_d",
             "stats_ti","stats_tl","stats_tf","stats_td",
-            "stats_ti_dv","stats_tl_dv","stats_tf_dv","stats_td_dv"
-//            , TODO: enable this test after SOLR-6452 is fixed
-//            "stats_ti_ni_dv","stats_tl_ni_dv","stats_tf_ni_dv","stats_td_ni_dv"
+            "stats_ti_dv","stats_tl_dv","stats_tf_dv","stats_td_dv", 
+            "stats_ti_ni_dv","stats_tl_ni_dv","stats_tf_ni_dv","stats_td_ni_dv"
     }) {
 
       // all of our checks should work with all of these params
@@ -91,9 +90,8 @@ public class StatsComponentTest extends 
     for (String f : new String[] {"stats_ii", // plain int
             "stats_is",    // sortable int
             "stats_tis","stats_tfs","stats_tls","stats_tds",  // trie fields
-            "stats_tis_dv","stats_tfs_dv","stats_tls_dv","stats_tds_dv"  // Doc Values
-//          , TODO: enable this test after SOLR-6452 is fixed
-            //"stats_tis_ni_dv","stats_tfs_ni_dv","stats_tls_ni_dv","stats_tds_ni_dv"  // Doc Values Not indexed
+            "stats_tis_dv","stats_tfs_dv","stats_tls_dv","stats_tds_dv",  // Doc Values
+            "stats_tis_ni_dv","stats_tfs_ni_dv","stats_tls_ni_dv","stats_tds_ni_dv"  // Doc Values Not indexed
                                   }) {
 
       doTestMVFieldStatisticsResult(f);
@@ -153,6 +151,9 @@ public class StatsComponentTest extends 
     assertU(adoc("id", "3", f, "-30", f, "-1", "active_s", "false"));
     assertU(adoc("id", "4", f, "-40", f, "10", "active_s", "false"));
     assertU(adoc("id", "5", "active_s", "false"));
+    assertU(adoc("id", "6", "active_s", "false"));
+    assertU(adoc("id", "7", "active_s", "true"));
+    
     assertU(commit());
 
     // with or w/o these excluded filters, results should be the same
@@ -171,7 +172,7 @@ public class StatsComponentTest extends 
               , "//double[@name='max'][.='200.0']"
               , "//double[@name='sum'][.='9.0']"
               , "//long[@name='count'][.='8']"
-              , "//long[@name='missing'][.='1']"
+              , "//long[@name='missing'][.='3']"
               , "//long[@name='countDistinct'][.='8']"
               , "count(//arr[@name='distinctValues']/*)=8"
               , "//double[@name='sumOfSquares'][.='53101.0']"
@@ -186,7 +187,7 @@ public class StatsComponentTest extends 
               , "//double[@name='max'][.='200.0']"
               , "//double[@name='sum'][.='119.0']"
               , "//long[@name='count'][.='6']"
-              , "//long[@name='missing'][.='1']"
+              , "//long[@name='missing'][.='3']"
               , "//long[@name='countDistinct'][.='6']"
               , "count(//arr[@name='distinctValues']/*)=6"
               , "//double[@name='sumOfSquares'][.='43001.0']"
@@ -202,7 +203,7 @@ public class StatsComponentTest extends 
               , "//double[@name='max'][.='200.0']"
               , "//double[@name='sum'][.='9.0']"
               , "//long[@name='count'][.='8']"
-              , "//long[@name='missing'][.='1']"
+              , "//long[@name='missing'][.='3']"
               , "//long[@name='countDistinct'][.='8']"
               , "count(//lst[@name='" + f + "']/arr[@name='distinctValues']/*)=8"
               , "//double[@name='sumOfSquares'][.='53101.0']"
@@ -216,7 +217,7 @@ public class StatsComponentTest extends 
               , "//lst[@name='true']/double[@name='max'][.='200.0']"
               , "//lst[@name='true']/double[@name='sum'][.='70.0']"
               , "//lst[@name='true']/long[@name='count'][.='4']"
-              , "//lst[@name='true']/long[@name='missing'][.='0']"
+              , "//lst[@name='true']/long[@name='missing'][.='1']"
               , "//lst[@name='true']//long[@name='countDistinct'][.='4']"
               , "count(//lst[@name='true']/arr[@name='distinctValues']/*)=4"
               , "//lst[@name='true']/double[@name='sumOfSquares'][.='50500.0']"
@@ -230,7 +231,7 @@ public class StatsComponentTest extends 
               , "//lst[@name='false']/double[@name='max'][.='10.0']"
               , "//lst[@name='false']/double[@name='sum'][.='-61.0']"
               , "//lst[@name='false']/long[@name='count'][.='4']"
-              , "//lst[@name='false']/long[@name='missing'][.='1']"
+              , "//lst[@name='false']/long[@name='missing'][.='2']"
               , "//lst[@name='true']//long[@name='countDistinct'][.='4']"
               , "count(//lst[@name='true']/arr[@name='distinctValues']/*)=4"
               , "//lst[@name='false']/double[@name='sumOfSquares'][.='2601.0']"
@@ -711,4 +712,31 @@ public class StatsComponentTest extends 
     Collections.addAll(cat_docValues, comparables);
     return cat_docValues;
   }
+  
+  
+//  public void testOtherFacetStatsResult() throws Exception {
+//    
+//    assertU(adoc("id", "1", "stats_tls_dv", "10", "active_i", "1"));
+//    assertU(adoc("id", "2", "stats_tls_dv", "20", "active_i", "1"));
+//    assertU(commit());
+//    assertU(adoc("id", "3", "stats_tls_dv", "30", "active_i", "2"));
+//    assertU(adoc("id", "4", "stats_tls_dv", "40", "active_i", "2"));
+//    assertU(commit());
+//    
+//    final String pre = "//lst[@name='stats_fields']/lst[@name='stats_tls_dv']/lst[@name='facets']/lst[@name='active_i']";
+//
+//    assertQ("test value for active_s=true", req("q", "*:*", "stats", "true", "stats.field", "stats_tls_dv", "stats.facet", "active_i","indent", "true")
+//            , "*[count("+pre+")=1]"
+//            , pre+"/lst[@name='1']/double[@name='min'][.='10.0']"
+//            , pre+"/lst[@name='1']/double[@name='max'][.='20.0']"
+//            , pre+"/lst[@name='1']/double[@name='sum'][.='30.0']"
+//            , pre+"/lst[@name='1']/long[@name='count'][.='2']"
+//            , pre+"/lst[@name='1']/long[@name='missing'][.='0']"
+//            , pre + "/lst[@name='true']/long[@name='countDistinct'][.='2']"
+//            , "count(" + pre + "/lst[@name='true']/arr[@name='distinctValues']/*)=2"
+//            , pre+"/lst[@name='1']/double[@name='sumOfSquares'][.='500.0']"
+//            , pre+"/lst[@name='1']/double[@name='mean'][.='15.0']"
+//            , pre+"/lst[@name='1']/double[@name='stddev'][.='7.0710678118654755']"
+//    );
+//  }
 }