You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/06/16 08:48:22 UTC

[GitHub] [lucene-solr] jpountz commented on a change in pull request #1351: LUCENE-9280: Collectors to skip noncompetitive documents

jpountz commented on a change in pull request #1351:
URL: https://github.com/apache/lucene-solr/pull/1351#discussion_r440675787



##########
File path: lucene/core/src/java/org/apache/lucene/search/FilteringNumericComparator.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+import java.io.IOException;
+
+/**
+ * A wrapper over {@code NumericComparator} that provides a leaf comparator that can filter non-competitive docs.
+ */
+class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> {
+  public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, boolean singleSort) {
+    super(in, reverse, singleSort);
+  }
+
+  @Override
+  public final FilteringLeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+    LeafFieldComparator inLeafComparator = in.getLeafComparator(context);
+    Class<?> comparatorClass = inLeafComparator.getClass();
+    if (comparatorClass == FieldComparator.LongComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringLongLeafComparator((FieldComparator.LongComparator) inLeafComparator, context,
+          ((LongComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } if (comparatorClass == FieldComparator.IntComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringIntLeafComparator((FieldComparator.IntComparator) inLeafComparator, context,
+          ((IntComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else if (comparatorClass == FieldComparator.DoubleComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringDoubleLeafComparator((FieldComparator.DoubleComparator) inLeafComparator, context,
+          ((DoubleComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else if (comparatorClass == FieldComparator.FloatComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringFloatLeafComparator((FieldComparator.FloatComparator) inLeafComparator, context,
+          ((FloatComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else {
+      assert false: "Unexpected class for [FieldComparator]!";

Review comment:
       add the class to the error message?

##########
File path: lucene/core/src/java/org/apache/lucene/search/FilteringNumericLeafComparator.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * A {@code FilteringLeafFieldComparator} that provides a functionality to skip over non-competitive documents
+ * for numeric fields indexed with points.
+ */
+abstract class FilteringNumericLeafComparator implements FilteringLeafFieldComparator {
+  protected final LeafFieldComparator in;
+  protected final boolean reverse;
+  protected final boolean singleSort;
+  private final boolean hasTopValue;
+  private final PointValues pointValues;
+  private final int bytesCount;
+  private final int maxDoc;
+  private final byte[] minValueAsBytes;
+  private final byte[] maxValueAsBytes;
+
+  private long iteratorCost;
+  private int maxDocVisited = 0;
+  private int updateCounter = 0;
+  private boolean canUpdateIterator = false; // set to true when queue becomes full and hitsThreshold is reached
+  private DocIdSetIterator competitiveIterator;
+
+  public FilteringNumericLeafComparator(LeafFieldComparator in, LeafReaderContext context, String field,
+        boolean reverse, boolean singleSort, boolean hasTopValue, int bytesCount) throws IOException {
+    this.in = in;
+    this.pointValues = context.reader().getPointValues(field);
+    this.reverse = reverse;
+    this.singleSort = singleSort;
+    this.hasTopValue = hasTopValue;
+    this.maxDoc = context.reader().maxDoc();
+    this.bytesCount = bytesCount;
+    this.maxValueAsBytes = reverse == false ? new byte[bytesCount] : hasTopValue ? new byte[bytesCount] : null;
+    this.minValueAsBytes = reverse ? new byte[bytesCount] : hasTopValue ? new byte[bytesCount] : null;
+
+    // TODO: optimize a case when pointValues are missing only on this segment
+    this.competitiveIterator = pointValues == null ? null : DocIdSetIterator.all(maxDoc);
+    this.iteratorCost = maxDoc;
+  }
+
+  @Override
+  public void setBottom(int slot) throws IOException {
+    in.setBottom(slot);
+    updateCompetitiveIterator(); // update an iterator if we set a new bottom
+  }
+
+  @Override
+  public int compareBottom(int doc) throws IOException {
+    return in.compareBottom(doc);
+  }
+
+  @Override
+  public int compareTop(int doc) throws IOException {
+    return in.compareTop(doc);
+  }
+
+  @Override
+  public void copy(int slot, int doc) throws IOException {
+    in.copy(slot, doc);
+    maxDocVisited = doc;
+  }
+
+  @Override
+  public void setScorer(Scorable scorer) throws IOException {
+    in.setScorer(scorer);
+    if (scorer instanceof Scorer) {
+      iteratorCost = ((Scorer) scorer).iterator().cost(); // starting iterator cost is the scorer's cost
+      updateCompetitiveIterator(); // update an iterator when we have a new segment
+    }
+  }
+
+  @Override
+  public void setCanUpdateIterator() throws IOException {
+    this.canUpdateIterator = true;
+    updateCompetitiveIterator();
+  }
+
+  @Override
+  public DocIdSetIterator competitiveIterator() {
+    if (competitiveIterator == null) return null;
+    return new DocIdSetIterator() {
+      private int doc;
+
+      @Override
+      public int nextDoc() throws IOException {
+        return doc = competitiveIterator.nextDoc();
+      }
+
+      @Override
+      public int docID() {
+        return doc;
+      }
+
+      @Override
+      public long cost() {
+        return competitiveIterator.cost();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        return doc = competitiveIterator.advance(target);
+      }
+    };
+  }
+
+  // update its iterator to include possibly only docs that are "stronger" than the current bottom entry
+  private void updateCompetitiveIterator() throws IOException {
+    if (canUpdateIterator == false) return;
+    if (pointValues == null) return;
+    // if some documents have missing points, check that missing values prohibits optimization
+    if ((pointValues.getDocCount() < maxDoc) && isMissingValueCompetitive()) {
+      return; // we can't filter out documents, as documents with missing values are competitive
+    }
+
+    updateCounter++;
+    if (updateCounter > 256 && (updateCounter & 0x1f) != 0x1f) { // Start sampling if we get called too much
+      return;
+    }
+    if (reverse == false) {
+      encodeBottom(maxValueAsBytes);
+      if (hasTopValue) {
+        encodeTop(minValueAsBytes);
+      }
+    } else {
+      encodeBottom(minValueAsBytes);
+      if (hasTopValue) {
+        encodeTop(maxValueAsBytes);
+      }
+    }
+
+    DocIdSetBuilder result = new DocIdSetBuilder(maxDoc);
+    PointValues.IntersectVisitor visitor = new PointValues.IntersectVisitor() {
+      DocIdSetBuilder.BulkAdder adder;
+
+      @Override
+      public void grow(int count) {
+        adder = result.grow(count);
+      }
+
+      @Override
+      public void visit(int docID) {
+        if (docID <= maxDocVisited) {
+          return; // Already visited or skipped
+        }
+        adder.add(docID);
+      }
+
+      @Override
+      public void visit(int docID, byte[] packedValue) {
+        if (docID <= maxDocVisited) {
+          return;  // already visited or skipped
+        }
+        if (maxValueAsBytes != null) {
+          int cmp = Arrays.compareUnsigned(packedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount);
+          // if doc's value is too high or for single sort even equal, it is not competitive and the doc can be skipped
+          if (cmp > 0 || (singleSort && cmp == 0)) return;
+        }
+        if (minValueAsBytes != null) {
+          int cmp = Arrays.compareUnsigned(packedValue, 0, bytesCount, minValueAsBytes, 0, bytesCount);
+          // if doc's value is too low or for single sort even equal, it is not competitive and the doc can be skipped
+          if (cmp < 0 || (singleSort && cmp == 0)) return;
+        }
+        adder.add(docID); // doc is competitive
+      }
+
+      @Override
+      public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+        boolean maxValueOutsideQuery = false;
+        boolean minValueOutsideQuery = false;
+        if (maxValueAsBytes != null) {
+          int cmp = Arrays.compareUnsigned(minPackedValue, 0, bytesCount, maxValueAsBytes, 0, bytesCount);
+          maxValueOutsideQuery = cmp > 0 || (singleSort && cmp == 0);

Review comment:
       should we return CELL_OUTSIDE_QUERY here when appropriate instead of delaying?

##########
File path: lucene/core/src/java/org/apache/lucene/search/FilteringNumericComparator.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+import java.io.IOException;
+
+/**
+ * A wrapper over {@code NumericComparator} that provides a leaf comparator that can filter non-competitive docs.
+ */
+class FilteringNumericComparator<T extends Number> extends FilteringFieldComparator<T> {
+  public FilteringNumericComparator(NumericComparator<T> in, boolean reverse, boolean singleSort) {
+    super(in, reverse, singleSort);
+  }
+
+  @Override
+  public final FilteringLeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+    LeafFieldComparator inLeafComparator = in.getLeafComparator(context);
+    Class<?> comparatorClass = inLeafComparator.getClass();
+    if (comparatorClass == FieldComparator.LongComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringLongLeafComparator((FieldComparator.LongComparator) inLeafComparator, context,
+          ((LongComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } if (comparatorClass == FieldComparator.IntComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringIntLeafComparator((FieldComparator.IntComparator) inLeafComparator, context,
+          ((IntComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else if (comparatorClass == FieldComparator.DoubleComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringDoubleLeafComparator((FieldComparator.DoubleComparator) inLeafComparator, context,
+          ((DoubleComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else if (comparatorClass == FieldComparator.FloatComparator.class) {
+      return new FilteringNumericLeafComparator.FilteringFloatLeafComparator((FieldComparator.FloatComparator) inLeafComparator, context,
+          ((FloatComparator) inLeafComparator).field, reverse, singleSort, hasTopValue);
+    } else {
+      assert false: "Unexpected class for [FieldComparator]!";
+      return null;

Review comment:
       should we throw an IllegalStateException instead of being lenient when assertions are disabled?

##########
File path: lucene/core/src/java/org/apache/lucene/search/FilteringLeafFieldComparator.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+
+/**
+ * Decorates a wrapped LeafFieldComparator to add a functionality to skip over non-competitive docs.
+ * FilteringLeafFieldComparator provides two additional functions to a LeafFieldComparator:
+ *  {@code competitiveIterator()} and {@code setCanUpdateIterator()}.
+ */
+public interface FilteringLeafFieldComparator extends LeafFieldComparator {
+  /**
+   * Returns a competitive iterator
+   * @return an iterator over competitive docs that are stronger than already collected docs
+   * or {@code null} if such an iterator is not available for the current segment.
+   */
+  DocIdSetIterator competitiveIterator();

Review comment:
       maybe we should throw IOException here to not force some implementations to catch/rethrow an an UncheckedIOException




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org