You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/07/01 11:40:01 UTC

svn commit: r1688611 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/ lucene/join/ lucene/join/src/java/org/apache/lucene/search/join/ lucene/join/src/test/org/apache/lucene/search/join/

Author: jpountz
Date: Wed Jul  1 09:40:00 2015
New Revision: 1688611

URL: http://svn.apache.org/r1688611
Log:
LUCENE-6554: Removed illegal ToBlockJoinFieldComparator in favor of doc values selectors.

Added:
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
      - copied, changed from r1688599, lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
    lucene/dev/branches/branch_5x/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
      - copied unchanged from r1688599, lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
Removed:
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
    lucene/dev/branches/branch_5x/lucene/join/   (props changed)
    lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1688611&r1=1688610&r2=1688611&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed Jul  1 09:40:00 2015
@@ -119,6 +119,12 @@ API Changes
 * LUCENE-6643: GroupingSearch from lucene/grouping was changed to take a Query
   object to define groups instead of a Filter. (Adrien Grand)
 
+* LUCENE-6554: ToParentBlockJoinFieldComparator was removed because of a bug
+  with missing values that could not be fixed. ToParentBlockJoinSortField now
+  works with string or numeric doc values selectors. Sorting on anything else
+  than a string or numeric field would require to implement a custom selector.
+  (Adrien Grand)
+
 Bug fixes
 
 * LUCENE-6500: ParallelCompositeReader did not always call

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1688611&r1=1688610&r2=1688611&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Wed Jul  1 09:40:00 2015
@@ -152,7 +152,7 @@ public abstract class FieldComparator<T>
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
       currentReaderValues = getNumericDocValues(context, field);
       if (missingValue != null) {
-        docsWithField = DocValues.getDocsWithField(context.reader(), field);
+        docsWithField = getDocsWithValue(context, field);
         // optimization to remove unneeded checks on the bit interface:
         if (docsWithField instanceof Bits.MatchAllBits) {
           docsWithField = null;
@@ -166,6 +166,11 @@ public abstract class FieldComparator<T>
     protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
       return DocValues.getNumeric(context.reader(), field);
     }
+
+    /** Retrieves a {@link Bits} instance representing documents that have a value in this segment. */
+    protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
+      return DocValues.getDocsWithField(context.reader(), field);
+    }
   }
 
   /** Parses field's values as double (using {@link

Copied: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java (from r1688599, lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java?p2=lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java&p1=lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java&r1=1688599&r2=1688611&rev=1688611&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java Wed Jul  1 09:40:00 2015
@@ -33,15 +33,20 @@ import org.apache.lucene.util.BytesRef;
  *  @lucene.internal */
 public class BlockJoinSelector {
 
+  private BlockJoinSelector() {}
+
   /** Type of selection to perform. If none of the documents in the block have
    *  a value then no value will be selected. */
   public enum Type {
-    MIN, MAX;
+    /** Only consider the minimum value from the block when sorting. */
+    MIN,
+    /** Only consider the maximum value from the block when sorting. */
+    MAX;
   }
 
   /** Return a {@link Bits} instance that returns true if, and only if, any of
    *  the children of the given parent document has a value. */
-  public static Bits wrap(final Bits docsWithValue, BitSet parents, BitSet children) {
+  public static Bits wrap(final Bits docsWithValue, final BitSet parents, final BitSet children) {
     return new Bits() {
 
       @Override
@@ -91,7 +96,7 @@ public class BlockJoinSelector {
   /** Wraps the provided {@link SortedDocValues} in order to only select
    *  one value per parent among its {@code children} using the configured
    *  {@code selection} type. */
-  public static SortedDocValues wrap(final SortedDocValues values, Type selection, BitSet parents, BitSet children) {
+  public static SortedDocValues wrap(final SortedDocValues values, final Type selection, final BitSet parents, final BitSet children) {
     return new SortedDocValues() {
 
       @Override
@@ -157,7 +162,10 @@ public class BlockJoinSelector {
     return wrap(values, DocValues.docsWithValue(sortedNumerics, parents.length()), selection, parents, children);
   }
 
-  public static NumericDocValues wrap(final NumericDocValues values, Bits docsWithValue, Type selection, BitSet parents, BitSet children) {
+  /** Wraps the provided {@link NumericDocValues} in order to only select
+   *  one value per parent among its {@code children} using the configured
+   *  {@code selection} type. */
+  public static NumericDocValues wrap(final NumericDocValues values, final Bits docsWithValue, final Type selection, final BitSet parents, final BitSet children) {
     return new NumericDocValues() {
 
       @Override

Modified: lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java?rev=1688611&r1=1688610&r2=1688611&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java Wed Jul  1 09:40:00 2015
@@ -17,9 +17,17 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.NumericUtils;
 
 import java.io.IOException;
 
@@ -46,6 +54,17 @@ public class ToParentBlockJoinSortField
    */
   public ToParentBlockJoinSortField(String field, Type type, boolean reverse, BitDocIdSetFilter parentFilter, BitDocIdSetFilter childFilter) {
     super(field, type, reverse);
+    switch (getType()) {
+      case STRING:
+      case DOUBLE:
+      case FLOAT:
+      case LONG:
+      case INT:
+        // ok
+        break;
+      default:
+        throw new UnsupportedOperationException("Sort type " + type + " is not supported");
+    }
     this.order = reverse;
     this.parentFilter = parentFilter;
     this.childFilter = childFilter;
@@ -69,14 +88,156 @@ public class ToParentBlockJoinSortField
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public FieldComparator<?> getComparator(int numHits, int sortPos) throws IOException {
-    FieldComparator<Object> wrappedFieldComparator = (FieldComparator) super.getComparator(numHits + 1, sortPos);
-    if (order) {
-      return new ToParentBlockJoinFieldComparator.Highest(wrappedFieldComparator, parentFilter, childFilter, numHits);
-    } else {
-      return new ToParentBlockJoinFieldComparator.Lowest(wrappedFieldComparator, parentFilter, childFilter, numHits);
+    switch (getType()) {
+      case STRING:
+        return getStringComparator(numHits);
+      case DOUBLE:
+        return getDoubleComparator(numHits);
+      case FLOAT:
+        return getFloatComparator(numHits);
+      case LONG:
+        return getLongComparator(numHits);
+      case INT:
+        return getIntComparator(numHits);
+      default:
+        throw new UnsupportedOperationException("Sort type " + getType() + " is not supported");
     }
   }
 
+  private FieldComparator<?> getStringComparator(int numHits) {
+    return new FieldComparator.TermOrdValComparator(numHits, getField(), missingValue == STRING_LAST) {
+
+      @Override
+      protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
+        SortedSetDocValues sortedSet = DocValues.getSortedSet(context.reader(), field);
+        final BlockJoinSelector.Type type = order
+            ? BlockJoinSelector.Type.MAX
+            : BlockJoinSelector.Type.MIN;
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return DocValues.emptySorted();
+        }
+        return BlockJoinSelector.wrap(sortedSet, type, parents.bits(), children.bits());
+      }
+
+    };
+  }
+
+  private FieldComparator<?> getIntComparator(int numHits) {
+    return new FieldComparator.IntComparator(numHits, getField(), (Integer) missingValue) {
+      @Override
+      protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+        SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(context.reader(), field);
+        final BlockJoinSelector.Type type = order
+            ? BlockJoinSelector.Type.MAX
+            : BlockJoinSelector.Type.MIN;
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return DocValues.emptyNumeric();
+        }
+        return BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+      }
+      @Override
+      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
+        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return new Bits.MatchNoBits(context.reader().maxDoc());
+        }
+        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+      }
+    };
+  }
+
+  private FieldComparator<?> getLongComparator(int numHits) {
+    return new FieldComparator.LongComparator(numHits, getField(), (Long) missingValue) {
+      @Override
+      protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+        SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(context.reader(), field);
+        final BlockJoinSelector.Type type = order
+            ? BlockJoinSelector.Type.MAX
+            : BlockJoinSelector.Type.MIN;
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return DocValues.emptyNumeric();
+        }
+        return BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+      }
+      @Override
+      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
+        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return new Bits.MatchNoBits(context.reader().maxDoc());
+        }
+        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+      }
+    };
+  }
+
+  private FieldComparator<?> getFloatComparator(int numHits) {
+    return new FieldComparator.FloatComparator(numHits, getField(), (Float) missingValue) {
+      @Override
+      protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+        SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(context.reader(), field);
+        final BlockJoinSelector.Type type = order
+            ? BlockJoinSelector.Type.MAX
+            : BlockJoinSelector.Type.MIN;
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return DocValues.emptyNumeric();
+        }
+        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        // undo the numericutils sortability
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return NumericUtils.sortableFloatBits((int) view.get(docID));
+          }
+        };
+      }
+    };
+  }
+
+  private FieldComparator<?> getDoubleComparator(int numHits) {
+    return new FieldComparator.DoubleComparator(numHits, getField(), (Double) missingValue) {
+      @Override
+      protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+        SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(context.reader(), field);
+        final BlockJoinSelector.Type type = order
+            ? BlockJoinSelector.Type.MAX
+            : BlockJoinSelector.Type.MIN;
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return DocValues.emptyNumeric();
+        }
+        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents.bits(), children.bits());
+        // undo the numericutils sortability
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return NumericUtils.sortableDoubleBits(view.get(docID));
+          }
+        };
+      }
+      @Override
+      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
+        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
+        final BitDocIdSet parents = parentFilter.getDocIdSet(context);
+        final BitDocIdSet children = childFilter.getDocIdSet(context);
+        if (children == null) {
+          return new Bits.MatchNoBits(context.reader().maxDoc());
+        }
+        return BlockJoinSelector.wrap(docsWithValue, parents.bits(), children.bits());
+      }
+    };
+  }
 }