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 2016/12/18 20:06:31 UTC

lucene-solr:branch_6x: LUCENE-7590: add Sorted(Set)DocValuesStats

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 63a5cd001 -> 47bb32c3b


LUCENE-7590: add Sorted(Set)DocValuesStats


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

Branch: refs/heads/branch_6x
Commit: 47bb32c3bb77a2dfaaf9d1db50e244599cf053a6
Parents: 63a5cd0
Author: Shai Erera <sh...@apache.org>
Authored: Sun Dec 18 12:20:42 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Sun Dec 18 22:05:54 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    |  88 +++++++++++++
 .../search/TestDocValuesStatsCollector.java     | 127 +++++++++++++++++--
 2 files changed, 201 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/47bb32c3/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index d7871ec..99b81fa 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -21,7 +21,11 @@ import java.io.IOException;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
 /** Holds statistics for a DocValues field. */
 public abstract class DocValuesStats<T> {
@@ -332,4 +336,88 @@ public abstract class DocValuesStats<T> {
     }
   }
 
+  private static BytesRef copyFrom(BytesRef src, BytesRef dest) {
+    if (dest == null) {
+      return BytesRef.deepCopyOf(src);
+    }
+
+    dest.bytes = ArrayUtil.grow(dest.bytes, src.length);
+    System.arraycopy(src.bytes, src.offset, dest.bytes, 0, src.length);
+    dest.offset = 0;
+    dest.length = src.length;
+    return dest;
+  }
+
+  /** Holds statistics for a sorted DocValues field. */
+  public static class SortedDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedDocValues sdv;
+    protected Bits docsWithField;
+
+    protected SortedDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      sdv = context.reader().getSortedDocValues(field);
+      docsWithField = context.reader().getDocsWithField(field);
+      return sdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return docsWithField.get(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int doc, int count) throws IOException {
+      BytesRef val = sdv.get(doc);
+      if (max == null || val.compareTo(max) > 0) {
+        max = copyFrom(val, max);
+      }
+      if (min == null || val.compareTo(min) < 0) {
+        min = copyFrom(val, min);
+      }
+    }
+  }
+
+  /** Holds statistics for a sorted-set DocValues field. */
+  public static class SortedSetDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedSetDocValues ssdv;
+    protected Bits docsWithField;
+
+    protected SortedSetDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      ssdv = context.reader().getSortedSetDocValues(field);
+      docsWithField = context.reader().getDocsWithField(field);
+      return ssdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return docsWithField.get(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int doc, int count) throws IOException {
+      ssdv.setDocument(doc);
+      long ord;
+      while ((ord = ssdv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        BytesRef val = ssdv.lookupOrd(ord);
+        if (max == null || val.compareTo(max) > 0) {
+          max = copyFrom(val, max);
+        }
+        if (min == null || val.compareTo(min) < 0) {
+          min = copyFrom(val, min);
+        }
+      }
+    }
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/47bb32c3/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 5fa4b04..3471c03 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.DoubleSummaryStatistics;
 import java.util.LongSummaryStatistics;
+import java.util.Objects;
 import java.util.function.Predicate;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
@@ -29,16 +30,21 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedDoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedLongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedSetDocValuesStats;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -227,8 +233,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedLongDocValuesStats stats = new SortedLongDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue());
@@ -281,8 +287,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedDoubleDocValuesStats stats = new SortedDoubleDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
@@ -298,6 +304,96 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
+  public void testDocsWithSortedValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[] docValues = new BytesRef[numDocs];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          BytesRef val = TestUtil.randomBinaryTerm(random());
+          doc.add(new SortedDocValuesField(field, val));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = val;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedDocValuesStats stats = new SortedDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
+  public void testDocsWithSortedSetValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[][] docValues = new BytesRef[numDocs][];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new BytesRef[numValues];
+          for (int j = 0; j < numValues; j++) {
+            BytesRef val = TestUtil.randomBinaryTerm(random());
+            doc.add(new SortedSetDocValuesField(field, val));
+            docValues[i][j] = val;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedSetDocValuesStats stats = new SortedSetDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
   private static LongStream getPositiveValues(long[] values) {
     return Arrays.stream(values).filter(v -> v > 0);
   }
@@ -314,14 +410,6 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static Stream<long[]> filterValues(long[][] values, Predicate<? super long[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
-  private static Stream<double[]> filterValues(double[][] values, Predicate<? super double[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
   private static double computeVariance(long[] values, double mean, int count) {
     return getPositiveValues(values).mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
   }
@@ -331,11 +419,11 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
   }
 
   private static LongStream filterAndFlatValues(long[][] values, Predicate<? super long[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToLong(Arrays::stream);
+    return nonNull(values).flatMapToLong(Arrays::stream);
   }
 
   private static DoubleStream filterAndFlatValues(double[][] values, Predicate<? super double[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToDouble(Arrays::stream);
+    return nonNull(values).flatMapToDouble(Arrays::stream);
   }
 
   private static double computeVariance(LongStream values, double mean, int count) {
@@ -346,4 +434,15 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return values.map(v -> (v - mean) * (v-mean)).sum() / count;
   }
 
+  private static <T> Stream<T> nonNull(T[] values) {
+    return filterValues(values, Objects::nonNull);
+  }
+
+  private static <T> Stream<T> isNull(T[] values) {
+    return filterValues(values, Objects::isNull);
+  }
+
+  private static <T> Stream<T> filterValues(T[] values, Predicate<? super T> p) {
+    return Arrays.stream(values).filter(p);
+  }
 }