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/05/27 03:48:49 UTC

[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1537: LUCENE-9381: Add SortOrder interface

mikemccand commented on a change in pull request #1537:
URL: https://github.com/apache/lucene-solr/pull/1537#discussion_r430367067



##########
File path: lucene/core/src/java/org/apache/lucene/search/Sort.java
##########
@@ -22,80 +22,7 @@
 
 
 /**
- * Encapsulates sort criteria for returned hits.
- *
- * <p>The fields used to determine sort order must be carefully chosen.

Review comment:
       Wow, these docs were quite stale!  They hark from the `FieldCache` days.  Wow, 2004!
   
   I think instead of fully deleting them, we should just updating them to state that you index the corresponding typed doc values field, and then sort by that type?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
##########
@@ -463,16 +465,65 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
    * Set the {@link Sort} order to use for all (flushed and merged) segments.
    */
   public IndexWriterConfig setIndexSort(Sort sort) {
-    for (SortField sortField : sort.getSort()) {
+    for (SortOrder sortField : sort.getSort()) {
       if (sortField.getIndexSorter() == null) {
         throw new IllegalArgumentException("Cannot sort index with sort field " + sortField);
       }
     }
     this.indexSort = sort;
-    this.indexSortFields = Arrays.stream(sort.getSort()).map(SortField::getField).collect(Collectors.toSet());
+    this.indexSortFields = extractFields(sort);
     return this;
   }
 
+  private Set<String> extractFields(Sort sort) {
+    Set<String> fields = new HashSet<>();
+    for (SortOrder sortOrder : sort.getSort()) {
+      IndexSorter sorter = sortOrder.getIndexSorter();
+      assert sorter != null;
+      try {
+        sorter.getDocComparator(new DocValuesLeafReader() {

Review comment:
       Hmm this is a little bit scary hackity, compared to what we had before :)
   
   Are you wanting to not add a `SortOrder.getField()`?

##########
File path: lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWSegmentInfoFormat.java
##########
@@ -89,7 +89,10 @@ public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOE
       int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
       output.writeVInt(numSortFields);
       for (int i = 0; i < numSortFields; ++i) {
-        SortField sortField = indexSort.getSort()[i];
+        if (indexSort.getSort()[i] instanceof SortField == false) {

Review comment:
       Hmm, maybe `indexSort` should just continue to take `SortField` for now?  Else this is sort of trappy -- only when `IndexWriter` goes to flush a segment to disk, will we (late) hit this exception?
   
   Or, maybe we could do this check instead in `IndexWriterConfig`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -172,8 +180,9 @@ public SortField readSortField(DataInput in) throws IOException {
     }
 
     @Override
-    public void writeSortField(SortField sf, DataOutput out) throws IOException {
-      sf.serialize(out);
+    public void writeSortField(SortOrder sf, DataOutput out) throws IOException {
+      assert sf instanceof SortField;

Review comment:
       Shouldn't this be a real check?  Caller could legitimately mess this up?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortOrder.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.SortFieldProvider;
+
+/**
+ * Defines an ordering for documents within an index
+ */
+public interface SortOrder {
+
+  /**
+   * Returns whether the sort should be reversed.
+   */
+  boolean getReverse();
+
+  /**
+   * Returns the {@link FieldComparator} to use for sorting.
+   *
+   * @param numHits   number of top hits the queue will store
+   * @param sortPos   position of this SortField within {@link Sort}.  The comparator is primary
+   *                  if sortPos==0, secondary if sortPos==1, etc.  Some comparators can
+   *                  optimize themselves when they are the primary sort.
+   */
+  FieldComparator<?> getComparator(int numHits, int sortPos);
+
+  /**
+   * Whether the relevance score is needed to sort documents.
+   */
+  boolean needsScores();
+
+  /**
+   * A name for the sort order
+   */
+  default String name() {
+    return toString();
+  }
+
+  /**
+   * Rewrites this SortOrder, returning a new SortOrder if a change is made.
+   *
+   * @param searcher IndexSearcher to use during rewriting
+   * @return New rewritten SortOrder, or {@code this} if nothing has changed.
+   */
+  default SortOrder rewrite(IndexSearcher searcher) throws IOException {
+    return this;
+  }
+
+  /**
+   * Returns an {@link IndexSorter} used for sorting index segments by this SortField.

Review comment:
       s/`for sorting index segments`/`for sorting documents within a single index segment`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortOrder.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.SortFieldProvider;
+
+/**
+ * Defines an ordering for documents within an index
+ */
+public interface SortOrder {
+
+  /**
+   * Returns whether the sort should be reversed.
+   */
+  boolean getReverse();
+
+  /**
+   * Returns the {@link FieldComparator} to use for sorting.
+   *
+   * @param numHits   number of top hits the queue will store
+   * @param sortPos   position of this SortField within {@link Sort}.  The comparator is primary
+   *                  if sortPos==0, secondary if sortPos==1, etc.  Some comparators can
+   *                  optimize themselves when they are the primary sort.
+   */
+  FieldComparator<?> getComparator(int numHits, int sortPos);
+
+  /**
+   * Whether the relevance score is needed to sort documents.
+   */
+  boolean needsScores();
+
+  /**
+   * A name for the sort order
+   */
+  default String name() {
+    return toString();
+  }
+
+  /**
+   * Rewrites this SortOrder, returning a new SortOrder if a change is made.
+   *
+   * @param searcher IndexSearcher to use during rewriting
+   * @return New rewritten SortOrder, or {@code this} if nothing has changed.
+   */
+  default SortOrder rewrite(IndexSearcher searcher) throws IOException {
+    return this;
+  }
+
+  /**
+   * Returns an {@link IndexSorter} used for sorting index segments by this SortField.
+   *
+   * If the SortField cannot be used for index sorting (for example, if it uses scores or
+   * other query-dependent values) then this method should return {@code null}
+   *
+   * SortFields that implement this method should also implement a companion
+   * {@link SortFieldProvider} to serialize and deserialize the sort in index segment
+   * headers
+   *
+   * @lucene.experimental
+   */
+  IndexSorter getIndexSorter();
+
+  /**
+   * Sorts by document score
+   */
+  SortOrder SCORE = new SortOrder() {
+    @Override
+    public boolean getReverse() {
+      return false;
+    }
+
+    @Override
+    public FieldComparator<?> getComparator(int numHits, int sortPos) {
+      return new FieldComparator.RelevanceComparator(numHits);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+
+    @Override
+    public IndexSorter getIndexSorter() {
+      return null;
+    }
+
+    @Override
+    public String toString() {
+      return "<score>";

Review comment:
       Hmm so callers of `getField()` would think this `<score>` is a field name?
   
   What does `Sort.FIELD_SCORE.getField()` return today?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -226,6 +235,10 @@ else if (missingValue == STRING_FIRST) {
     }
   }
 
+  public String getField() {
+    return name();

Review comment:
       Hmm, why not return `field` like we do today?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -172,8 +180,9 @@ public SortField readSortField(DataInput in) throws IOException {
     }
 
     @Override
-    public void writeSortField(SortField sf, DataOutput out) throws IOException {
-      sf.serialize(out);
+    public void writeSortField(SortOrder sf, DataOutput out) throws IOException {

Review comment:
       Similarly, shouldn't this take a `SortField`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -138,7 +146,7 @@ public Provider() {
     }
 
     @Override
-    public SortField readSortField(DataInput in) throws IOException {
+    public SortOrder readSortField(DataInput in) throws IOException {

Review comment:
       Hmm, if `SortOrder` is not (necessarily) serializable, and `SortField` is, shouldn't this return a `SortField`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -172,8 +180,9 @@ public SortField readSortField(DataInput in) throws IOException {
     }
 
     @Override
-    public void writeSortField(SortField sf, DataOutput out) throws IOException {
-      sf.serialize(out);
+    public void writeSortField(SortOrder sf, DataOutput out) throws IOException {
+      assert sf instanceof SortField;
+      ((SortField)sf).serialize(out);

Review comment:
       And then these spooky `assert` / casts wouldn't be needed -- `javac` would do it for us?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortOrder.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.SortFieldProvider;
+
+/**
+ * Defines an ordering for documents within an index
+ */
+public interface SortOrder {
+
+  /**
+   * Returns whether the sort should be reversed.
+   */
+  boolean getReverse();
+
+  /**
+   * Returns the {@link FieldComparator} to use for sorting.
+   *
+   * @param numHits   number of top hits the queue will store
+   * @param sortPos   position of this SortField within {@link Sort}.  The comparator is primary
+   *                  if sortPos==0, secondary if sortPos==1, etc.  Some comparators can
+   *                  optimize themselves when they are the primary sort.
+   */
+  FieldComparator<?> getComparator(int numHits, int sortPos);
+
+  /**
+   * Whether the relevance score is needed to sort documents.
+   */
+  boolean needsScores();
+
+  /**
+   * A name for the sort order
+   */
+  default String name() {
+    return toString();
+  }
+
+  /**
+   * Rewrites this SortOrder, returning a new SortOrder if a change is made.
+   *
+   * @param searcher IndexSearcher to use during rewriting
+   * @return New rewritten SortOrder, or {@code this} if nothing has changed.
+   */
+  default SortOrder rewrite(IndexSearcher searcher) throws IOException {
+    return this;
+  }
+
+  /**
+   * Returns an {@link IndexSorter} used for sorting index segments by this SortField.
+   *
+   * If the SortField cannot be used for index sorting (for example, if it uses scores or
+   * other query-dependent values) then this method should return {@code null}

Review comment:
       s/`should`/`must`?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortOrder.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.SortFieldProvider;
+
+/**
+ * Defines an ordering for documents within an index
+ */
+public interface SortOrder {
+
+  /**
+   * Returns whether the sort should be reversed.
+   */
+  boolean getReverse();
+
+  /**
+   * Returns the {@link FieldComparator} to use for sorting.
+   *
+   * @param numHits   number of top hits the queue will store
+   * @param sortPos   position of this SortField within {@link Sort}.  The comparator is primary
+   *                  if sortPos==0, secondary if sortPos==1, etc.  Some comparators can
+   *                  optimize themselves when they are the primary sort.
+   */
+  FieldComparator<?> getComparator(int numHits, int sortPos);
+
+  /**
+   * Whether the relevance score is needed to sort documents.
+   */
+  boolean needsScores();
+
+  /**
+   * A name for the sort order
+   */
+  default String name() {
+    return toString();

Review comment:
       It's a little scary to me that `getField` comes down to `toString`...

##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -1225,6 +1227,16 @@ public SchemaField getFieldOrNull(String fieldName) {
     return f;
   }
 
+  public SchemaField getFieldOrNull(SortOrder sortOrder) {
+    if (sortOrder instanceof SortField) {
+      String field = sortOrder.name();

Review comment:
       Couldn't you cast to `SortField` and use `.getField()`?
   
   I think this `name`/`field`/`toString` triality is confusing...

##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -1225,6 +1227,16 @@ public SchemaField getFieldOrNull(String fieldName) {
     return f;
   }
 
+  public SchemaField getFieldOrNull(SortOrder sortOrder) {
+    if (sortOrder instanceof SortField) {

Review comment:
       This is another scary hackity code!




----------------------------------------------------------------
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