You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2016/03/04 16:50:59 UTC

lucene-solr git commit: LUCENE-5735: remove NumberRangePrefixTreeStrategy.calcFacets from 6x for now

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 1cbf22569 -> 0b15fd863


LUCENE-5735: remove NumberRangePrefixTreeStrategy.calcFacets from 6x for now


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

Branch: refs/heads/branch_6x
Commit: 0b15fd86364dbafe08abc2939d54749e69bca23f
Parents: 1cbf225
Author: David Smiley <ds...@apache.org>
Authored: Fri Mar 4 10:50:53 2016 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Fri Mar 4 10:50:53 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 -
 .../prefix/NumberRangePrefixTreeStrategy.java   | 141 +---------
 .../spatial/prefix/NumberRangeFacetsTest.java   | 275 -------------------
 3 files changed, 2 insertions(+), 418 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b15fd86/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index fd01a92..71a715b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -25,10 +25,6 @@ New Features
   input tokens. Useful for normalizing short text in clustering/linking 
   tasks. (Mark Harwood, Adrien Grand)
 
-* LUCENE-5735: NumberRangePrefixTreeStrategy now includes interval/range faceting
-  for counting ranges that align with the underlying terms as defined by the
-  NumberRangePrefixTree (e.g. familiar date units like days).  (David Smiley)
-
 * LUCENE-6711: Use CollectionStatistics.docCount() for IDF and average field
   length computations, to avoid skew from documents that don't have the field.
   (Ahmet Arslan via Robert Muir)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b15fd86/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
index c727c0d..b310de3 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
@@ -16,22 +16,13 @@
  */
 package org.apache.lucene.spatial.prefix;
 
-import java.io.IOException;
-import java.util.Arrays;
 import java.util.Iterator;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
-import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
-import org.apache.lucene.util.Bits;
-
-import static org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /** A PrefixTree based on Number/Date ranges. This isn't very "spatial" on the surface (to the user) but
  * it's implemented using spatial so that's why it's here extending a SpatialStrategy. When using this class, you will
@@ -68,132 +59,4 @@ public class NumberRangePrefixTreeStrategy extends RecursivePrefixTreeStrategy {
     throw new UnsupportedOperationException();
   }
 
-  /** Calculates facets between {@code start} and {@code end} to a detail level one greater than that provided by the
-   * arguments. For example providing March to October of 2014 would return facets to the day level of those months.
-   * This is just a convenience method.
-   * @see #calcFacets(IndexReaderContext, Bits, Shape, int)
-   */
-  public Facets calcFacets(IndexReaderContext context, Bits topAcceptDocs, UnitNRShape start, UnitNRShape end)
-      throws IOException {
-    Shape facetRange = getGrid().toRangeShape(start, end);
-    int detailLevel = Math.max(start.getLevel(), end.getLevel()) + 1;
-    return calcFacets(context, topAcceptDocs, facetRange, detailLevel);
-  }
-
-  /**
-   * Calculates facets (aggregated counts) given a range shape (start-end span) and a level, which specifies the detail.
-   * To get the level of an existing shape, say a Calendar, call
-   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree#toUnitShape(Object)} then call
-   * {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape#getLevel()}.
-   * Facet computation is implemented by navigating the underlying indexed terms efficiently.
-   */
-  public Facets calcFacets(IndexReaderContext context, Bits topAcceptDocs, Shape facetRange, final int level)
-      throws IOException {
-    final Facets facets = new Facets(level);
-    PrefixTreeFacetCounter.compute(this, context, topAcceptDocs, facetRange, level,
-        new PrefixTreeFacetCounter.FacetVisitor() {
-          Facets.FacetParentVal parentFacet;
-          UnitNRShape parentShape;
-
-          @Override
-          public void visit(Cell cell, int count) {
-            if (cell.getLevel() < level - 1) {//some ancestor of parent facet level, direct or distant
-              parentFacet = null;//reset
-              parentShape = null;//reset
-              facets.topLeaves += count;
-            } else if (cell.getLevel() == level - 1) {//parent
-              //set up FacetParentVal
-              setupParent((UnitNRShape) cell.getShape());
-              parentFacet.parentLeaves += count;
-            } else {//at facet level
-              UnitNRShape unitShape = (UnitNRShape) cell.getShape();
-              UnitNRShape unitShapeParent = unitShape.getShapeAtLevel(unitShape.getLevel() - 1);
-              if (parentFacet == null || !parentShape.equals(unitShapeParent)) {
-                setupParent(unitShapeParent);
-              }
-              //lazy init childCounts
-              if (parentFacet.childCounts == null) {
-                parentFacet.childCounts = new int[parentFacet.childCountsLen];
-              }
-              parentFacet.childCounts[unitShape.getValAtLevel(cell.getLevel())] += count;
-            }
-          }
-
-          private void setupParent(UnitNRShape unitShape) {
-            parentShape = unitShape.clone();
-            //Look for existing parentFacet (from previous segment), or create anew if needed
-            parentFacet = facets.parents.get(parentShape);
-            if (parentFacet == null) {//didn't find one; make a new one
-              parentFacet = new Facets.FacetParentVal();
-              parentFacet.childCountsLen = getGrid().getNumSubCells(parentShape);
-              facets.parents.put(parentShape, parentFacet);
-            }
-          }
-        });
-    return facets;
-  }
-
-  /** Facet response information */
-  public static class Facets {
-    //TODO consider a variable-level structure -- more general purpose.
-
-    public Facets(int detailLevel) {
-      this.detailLevel = detailLevel;
-    }
-
-    /** The bottom-most detail-level counted, as requested. */
-    public final int detailLevel;
-
-    /**
-     * The count of documents with ranges that completely spanned the parents of the detail level. In more technical
-     * terms, this is the count of leaf cells 2 up and higher from the bottom. Usually you only care about counts at
-     * detailLevel, and so you will add this number to all other counts below, including to omitted/implied children
-     * counts of 0. If there are no indexed ranges (just instances, i.e. fully specified dates) then this value will
-     * always be 0.
-     */
-    public int topLeaves;
-
-    /** Holds all the {@link FacetParentVal} instances in order of the key. This is sparse; there won't be an
-     * instance if it's count and children are all 0. The keys are {@link org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape} shapes, which can be
-     * converted back to the original Object (i.e. a Calendar) via
-     * {@link NumberRangePrefixTree#toObject(org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape)}. */
-    public final SortedMap<UnitNRShape,FacetParentVal> parents = new TreeMap<>();
-
-    /** Holds a block of detailLevel counts aggregated to their parent level. */
-    public static class FacetParentVal {
-
-      /** The count of ranges that span all of the childCounts.  In more technical terms, this is the number of leaf
-       * cells found at this parent.  Treat this like {@link Facets#topLeaves}. */
-      public int parentLeaves;
-
-      /** The length of {@link #childCounts}. If childCounts is not null then this is childCounts.length, otherwise it
-       * says how long it would have been if it weren't null. */
-      public int childCountsLen;
-
-      /** The detail level counts. It will be null if there are none, and thus they are assumed 0. Most apps, when
-       * presenting the information, will add {@link #topLeaves} and {@link #parentLeaves} to each count. */
-      public int[] childCounts;
-      //assert childCountsLen == childCounts.length
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(2048);
-      buf.append("Facets: level=" + detailLevel + " topLeaves=" + topLeaves + " parentCount=" + parents.size());
-      for (Map.Entry<UnitNRShape, FacetParentVal> entry : parents.entrySet()) {
-        buf.append('\n');
-        if (buf.length() > 1000) {
-          buf.append("...");
-          break;
-        }
-        final FacetParentVal pVal = entry.getValue();
-        buf.append(' ').append(entry.getKey()+" leafCount=" + pVal.parentLeaves);
-        if (pVal.childCounts != null) {
-          buf.append(' ').append(Arrays.toString(pVal.childCounts));
-        }
-      }
-      return buf.toString();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b15fd86/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
deleted file mode 100644
index 514c18e..0000000
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
+++ /dev/null
@@ -1,275 +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.lucene.spatial.prefix;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.List;
-
-import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import org.locationtech.spatial4j.shape.Shape;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.spatial.StrategyTestCase;
-import org.apache.lucene.spatial.prefix.NumberRangePrefixTreeStrategy.Facets;
-import org.apache.lucene.spatial.prefix.tree.Cell;
-import org.apache.lucene.spatial.prefix.tree.CellIterator;
-import org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree;
-import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
-import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitSet;
-import org.junit.Before;
-import org.junit.Test;
-
-import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
-import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
-
-public class NumberRangeFacetsTest extends StrategyTestCase {
-
-  DateRangePrefixTree tree;
-
-  int randomCalWindowField;
-  long randomCalWindowMs;
-
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-    tree = DateRangePrefixTree.INSTANCE;
-    strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
-    Calendar tmpCal = tree.newCal();
-    randomCalWindowField = randomIntBetween(1, Calendar.ZONE_OFFSET - 1);//we're not allowed to add zone offset
-    tmpCal.add(randomCalWindowField, 2_000);
-    randomCalWindowMs = Math.max(2000L, tmpCal.getTimeInMillis());
-  }
-
-  @Repeat(iterations = 20)
-  @Test
-  public void test() throws IOException {
-    //generate test data
-    List<Shape> indexedShapes = new ArrayList<>();
-    final int numIndexedShapes = random().nextInt(15);
-    for (int i = 0; i < numIndexedShapes; i++) {
-      indexedShapes.add(randomShape());
-    }
-
-    //Main index loop:
-    for (int i = 0; i < indexedShapes.size(); i++) {
-      Shape shape = indexedShapes.get(i);
-      adoc(""+i, shape);
-
-      if (random().nextInt(10) == 0)
-        commit();//intermediate commit, produces extra segments
-    }
-
-    //delete some documents randomly
-    for (int id = 0; id < indexedShapes.size(); id++) {
-      if (random().nextInt(10) == 0) {
-        deleteDoc(""+id);
-        indexedShapes.set(id, null);
-      }
-    }
-
-    commit();
-
-    //Main query loop:
-    for (int queryIdx = 0; queryIdx < 10; queryIdx++) {
-      preQueryHavoc();
-
-      // We need to have a facet range window to do the facets between (a start time & end time). We randomly
-      // pick a date, decide the level we want to facet on, and then pick a right end time that is up to 2 thousand
-      // values later.
-      int calFieldFacet = randomCalWindowField - 1;
-      if (calFieldFacet > 1 && rarely()) {
-        calFieldFacet--;
-      }
-      final Calendar leftCal = randomCalendar();
-      leftCal.add(calFieldFacet, -1 * randomInt(1000));
-      Calendar rightCal = (Calendar) leftCal.clone();
-      rightCal.add(calFieldFacet, randomInt(2000));
-      // Pick facet detail level based on cal field.
-      int detailLevel = tree.getTreeLevelForCalendarField(calFieldFacet);
-      if (detailLevel < 0) {//no exact match
-        detailLevel = -1 * detailLevel;
-      }
-
-      //Randomly pick a filter/acceptDocs
-      Bits topAcceptDocs = null;
-      List<Integer> acceptFieldIds = new ArrayList<>();
-      if (usually()) {
-        //get all possible IDs into a list, random shuffle it, then randomly choose how many of the first we use to
-        // replace the list.
-        for (int i = 0; i < indexedShapes.size(); i++) {
-          if (indexedShapes.get(i) == null) { // we deleted this one
-            continue;
-          }
-          acceptFieldIds.add(i);
-        }
-        Collections.shuffle(acceptFieldIds, random());
-        acceptFieldIds = acceptFieldIds.subList(0, randomInt(acceptFieldIds.size()));
-        if (!acceptFieldIds.isEmpty()) {
-          List<Term> terms = new ArrayList<>();
-          for (Integer acceptDocId : acceptFieldIds) {
-            terms.add(new Term("id", acceptDocId.toString()));
-          }
-
-          topAcceptDocs = searchForDocBits(new TermsQuery(terms));
-        }
-      }
-
-      //Lets do it!
-      NumberRangePrefixTree.NRShape facetRange = tree.toRangeShape(tree.toShape(leftCal), tree.toShape(rightCal));
-      Facets facets = ((NumberRangePrefixTreeStrategy) strategy)
-          .calcFacets(indexSearcher.getTopReaderContext(), topAcceptDocs, facetRange, detailLevel);
-
-      //System.out.println("Q: " + queryIdx + " " + facets);
-
-      //Verify results. We do it by looping over indexed shapes and reducing the facet counts.
-      Shape facetShapeRounded = facetRange.roundToLevel(detailLevel);
-      for (int indexedShapeId = 0; indexedShapeId < indexedShapes.size(); indexedShapeId++) {
-        if (topAcceptDocs != null && !acceptFieldIds.contains(indexedShapeId)) {
-          continue;// this doc was filtered out via acceptDocs
-        }
-        Shape indexedShape = indexedShapes.get(indexedShapeId);
-        if (indexedShape == null) {//was deleted
-          continue;
-        }
-        Shape indexedShapeRounded = ((NumberRangePrefixTree.NRShape) indexedShape).roundToLevel(detailLevel);
-        if (!indexedShapeRounded.relate(facetShapeRounded).intersects()) { // no intersection at all
-          continue;
-        }
-        // walk the cells
-        final CellIterator cellIterator = tree.getTreeCellIterator(indexedShape, detailLevel);
-        while (cellIterator.hasNext()) {
-          Cell cell = cellIterator.next();
-          if (!cell.getShape().relate(facetShapeRounded).intersects()) {
-            cellIterator.remove();//no intersection; prune
-            continue;
-          }
-          assert cell.getLevel() <= detailLevel;
-
-          if (cell.getLevel() == detailLevel) {
-            //count it
-            UnitNRShape shape = (UnitNRShape) cell.getShape();
-            final UnitNRShape parentShape = shape.getShapeAtLevel(detailLevel - 1);//get parent
-            final Facets.FacetParentVal facetParentVal = facets.parents.get(parentShape);
-            assertNotNull(facetParentVal);
-            int index = shape.getValAtLevel(shape.getLevel());
-            assertNotNull(facetParentVal.childCounts);
-            assert facetParentVal.childCounts[index] > 0;
-            facetParentVal.childCounts[index]--;
-
-          } else if (cell.isLeaf()) {
-            //count it, and remove/prune.
-            if (cell.getLevel() < detailLevel - 1) {
-              assert facets.topLeaves > 0;
-              facets.topLeaves--;
-            } else {
-              UnitNRShape shape = (UnitNRShape) cell.getShape();
-              final UnitNRShape parentShape = shape.getShapeAtLevel(detailLevel - 1);//get parent
-              final Facets.FacetParentVal facetParentVal = facets.parents.get(parentShape);
-              assertNotNull(facetParentVal);
-              assert facetParentVal.parentLeaves > 0;
-              facetParentVal.parentLeaves--;
-            }
-
-            cellIterator.remove();
-          }
-        }
-      }
-      // At this point; all counts should be down to zero.
-      assertTrue(facets.topLeaves == 0);
-      for (Facets.FacetParentVal facetParentVal : facets.parents.values()) {
-        assertTrue(facetParentVal.parentLeaves == 0);
-        if (facetParentVal.childCounts != null) {
-          for (int childCount : facetParentVal.childCounts) {
-            assertTrue(childCount == 0);
-          }
-        }
-      }
-
-    }
-  }
-
-  private Bits searchForDocBits(Query query) throws IOException {
-    FixedBitSet bitSet = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
-    indexSearcher.search(query,
-        new SimpleCollector() {
-          int leafDocBase;
-          @Override
-          public void collect(int doc) throws IOException {
-            bitSet.set(leafDocBase + doc);
-          }
-
-          @Override
-          protected void doSetNextReader(LeafReaderContext context) throws IOException {
-            leafDocBase = context.docBase;
-          }
-
-          @Override
-          public boolean needsScores() {
-            return false;
-          }
-        });
-    return bitSet;
-  }
-
-  private void preQueryHavoc() {
-    if (strategy instanceof RecursivePrefixTreeStrategy) {
-      RecursivePrefixTreeStrategy rpts = (RecursivePrefixTreeStrategy) strategy;
-      int scanLevel = randomInt(rpts.getGrid().getMaxLevels());
-      rpts.setPrefixGridScanLevel(scanLevel);
-    }
-  }
-
-  protected Shape randomShape() {
-    Calendar cal1 = randomCalendar();
-    UnitNRShape s1 = tree.toShape(cal1);
-    if (rarely()) {
-      return s1;
-    }
-    try {
-      Calendar cal2 = randomCalendar();
-      UnitNRShape s2 = tree.toShape(cal2);
-      if (cal1.compareTo(cal2) < 0) {
-        return tree.toRangeShape(s1, s2);
-      } else {
-        return tree.toRangeShape(s2, s1);
-      }
-    } catch (IllegalArgumentException e) {
-      assert e.getMessage().startsWith("Differing precision");
-      return s1;
-    }
-  }
-
-  private Calendar randomCalendar() {
-    Calendar cal = tree.newCal();
-    cal.setTimeInMillis(random().nextLong() % randomCalWindowMs);
-    try {
-      tree.clearFieldsAfter(cal, random().nextInt(Calendar.FIELD_COUNT+1)-1);
-    } catch (AssertionError e) {
-      if (!e.getMessage().equals("Calendar underflow"))
-        throw e;
-    }
-    return cal;
-  }
-}