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/08/26 11:37:38 UTC

[GitHub] [lucene-solr] s1monw opened a new pull request #1789: LUCENE-9484: Allow sorting an index after the fact

s1monw opened a new pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789


   Today we need to decide on an index sorting before we create the index.
   In some situations it might make a lot of sense to sort an index afterwards
   when the index doesn't change anymore or to compress older indices.
   This comit adds the ability to wrap readers from an unsorted index and merge it
   into a sorted index by using IW#addIndices.
   


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


[GitHub] [lucene-solr] s1monw commented on pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#issuecomment-681983796


   > It's nice that this change is so straightforward. It makes me realize I don't know what happens today if we specify an index Sort and then open an existing index that is not sorted. Do we throw an error? Should we instead provide a sorted view on the index that can be used to rewrite it?
   
   yes we fail if you do that. I don't think we should do any magic here. rewriting can be very costly and taking lots of space. I think failing is the right thing to do.


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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r482434175



##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      throw new UnsupportedOperationException("use nextDoc instead");

Review comment:
       I just realized reading other comments that this method might be required when configuring index sorting on a binary field.




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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r482772181



##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      throw new UnsupportedOperationException("use nextDoc instead");

Review comment:
       :+1:




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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r482419163



##########
File path: lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
##########
@@ -165,4 +166,62 @@ public long longValue() {
       return value;
     }
   }
+
+  static class SortingNumericDocValues extends NumericDocValues {
+
+    private final CachedNumericDVs dvs;
+    private int docID = -1;
+
+    SortingNumericDocValues(CachedNumericDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = dvs.docsWithField.nextSetBit(target);
+      return docID;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      docID = target;
+      return dvs.docsWithField.get(target);
+    }
+
+    @Override
+    public long longValue() {
+      return dvs.values[docID];
+    }
+
+    @Override
+    public long cost() {
+      return dvs.docsWithField.cardinality();

Review comment:
       cache the cost?




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


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r482740267



##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      throw new UnsupportedOperationException("use nextDoc instead");

Review comment:
       many others are needed but afaik if you wanna sort on a binary field it needs to be `SortedDocValues` and not `BinaryDocValues` - at least I wasn't able to make any test fail here..




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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r482418961



##########
File path: lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
##########
@@ -165,4 +166,62 @@ public long longValue() {
       return value;
     }
   }
+
+  static class SortingNumericDocValues extends NumericDocValues {
+
+    private final CachedNumericDVs dvs;
+    private int docID = -1;
+
+    SortingNumericDocValues(CachedNumericDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = dvs.docsWithField.nextSetBit(target);
+      return docID;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      docID = target;
+      return dvs.docsWithField.get(target);

Review comment:
       throw UOE here too?

##########
File path: lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
##########
@@ -165,4 +166,62 @@ public long longValue() {
       return value;
     }
   }
+
+  static class SortingNumericDocValues extends NumericDocValues {
+
+    private final CachedNumericDVs dvs;
+    private int docID = -1;
+
+    SortingNumericDocValues(CachedNumericDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = dvs.docsWithField.nextSetBit(target);
+      return docID;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      docID = target;
+      return dvs.docsWithField.get(target);
+    }
+
+    @Override
+    public long longValue() {
+      return dvs.values[docID];
+    }
+
+    @Override
+    public long cost() {
+      return dvs.docsWithField.cardinality();

Review comment:
       and cache the cost?

##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      throw new UnsupportedOperationException("use nextDoc instead");

Review comment:
       I just realized reading other comments that this method might be required when configuring index sorting on a binary field?




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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r480385432



##########
File path: lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
##########
@@ -1090,20 +1150,64 @@ public Bits getLiveDocs() {
   }
 
   @Override
-  public PointValues getPointValues(String fieldName) throws IOException {
-    final PointValues inPointValues = in.getPointValues(fieldName);
-    if (inPointValues == null) {
-      return null;
-    } else {
-      return new SortingPointValues(inPointValues, docMap);
-    }
+  public PointsReader getPointsReader() {
+    return new PointsReader() {
+      @Override
+      public void checkIntegrity() throws IOException {
+        in.getPointsReader().checkIntegrity();
+      }
+
+      @Override
+      public PointValues getValues(String field) throws IOException {
+        final PointValues inPointValues = in.getPointValues(field);
+        if (inPointValues == null) {
+          return null;
+        } else {
+          return new SortingPointValues(inPointValues, docMap);
+        }
+      }
+
+      @Override
+      public void close() throws IOException {
+        in.getPointsReader().close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return in.getPointsReader().ramBytesUsed();
+      }
+    };
   }
 
   private final Map<String,CachedNumericDVs> cachedNorms = new HashMap<>();
 
   @Override
-  public NumericDocValues getNormValues(String field) throws IOException {
-    final NumericDocValues oldNorms = in.getNormValues(field);
+  public NormsProducer getNormsReader() {
+    return new NormsProducer() {
+      @Override
+      public NumericDocValues getNorms(FieldInfo field) throws IOException {
+        final NumericDocValues oldNorms = in.getNormValues(field.name);

Review comment:
       likewise, delegate to in#getNormsReader here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
##########
@@ -1090,20 +1150,64 @@ public Bits getLiveDocs() {
   }
 
   @Override
-  public PointValues getPointValues(String fieldName) throws IOException {
-    final PointValues inPointValues = in.getPointValues(fieldName);
-    if (inPointValues == null) {
-      return null;
-    } else {
-      return new SortingPointValues(inPointValues, docMap);
-    }
+  public PointsReader getPointsReader() {
+    return new PointsReader() {
+      @Override
+      public void checkIntegrity() throws IOException {
+        in.getPointsReader().checkIntegrity();
+      }
+
+      @Override
+      public PointValues getValues(String field) throws IOException {
+        final PointValues inPointValues = in.getPointValues(field);

Review comment:
       IMO we should delegate to `in#getPointsReader()` as the contract is slightly different on the reader and the points reader (the reader returns null if the field was not indexed with points while the points reader fails).

##########
File path: lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
##########
@@ -1090,20 +1150,64 @@ public Bits getLiveDocs() {
   }
 
   @Override
-  public PointValues getPointValues(String fieldName) throws IOException {
-    final PointValues inPointValues = in.getPointValues(fieldName);
-    if (inPointValues == null) {
-      return null;
-    } else {
-      return new SortingPointValues(inPointValues, docMap);
-    }
+  public PointsReader getPointsReader() {
+    return new PointsReader() {
+      @Override
+      public void checkIntegrity() throws IOException {
+        in.getPointsReader().checkIntegrity();
+      }
+
+      @Override
+      public PointValues getValues(String field) throws IOException {
+        final PointValues inPointValues = in.getPointValues(field);
+        if (inPointValues == null) {

Review comment:
       it can never be null since getValues may only be called on fields that were indexed with points on points readers




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


[GitHub] [lucene-solr] s1monw merged pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
s1monw merged pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789


   


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


[GitHub] [lucene-solr] jpountz commented on a change in pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#discussion_r481823967



##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = dvs.docsWithField.nextSetBit(target);
+      return docID;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      docID = target;
+      return dvs.docsWithField.get(target);
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return dvs.values[docID];
+    }
+
+    @Override
+    public long cost() {
+      return dvs.docsWithField.cardinality();

Review comment:
       I think we should either throw UOE here or cache the cardinality.

##########
File path: lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
##########
@@ -202,4 +203,63 @@ public BytesRef binaryValue() {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = dvs.docsWithField.nextSetBit(target);
+      return docID;

Review comment:
       I think this will fail if target is > maxDoc? Maybe this class should just throw UOE for advance and advanceExact since codecs are only supposed to consume these doc values via nextDoc.

##########
File path: lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
##########
@@ -0,0 +1,514 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * An {@link org.apache.lucene.index.CodecReader} which supports sorting documents by a given
+ * {@link Sort}. This can be used to re-sort and index after it's been created by wrapping all
+ * readers of the index with this reader and adding it to a fresh IndexWriter via
+ * {@link IndexWriter#addIndexes(CodecReader...)}.
+ *
+ * @lucene.experimental
+ */
+public final class SortingCodecReader extends FilterCodecReader {
+
+  private final Map<String, NumericDocValuesWriter.CachedNumericDVs> cachedNumericDVs = new HashMap<>();
+
+  private final Map<String, BinaryDocValuesWriter.CachedBinaryDVs> cachedBinaryDVs = new HashMap<>();
+
+  private final Map<String, int[]> cachedSortedDVs = new HashMap<>();
+
+  // TODO: pack long[][] into an int[] (offset) and long[] instead:
+  private final Map<String, long[][]> cachedSortedSetDVs = new HashMap<>();
+
+  private final Map<String, long[][]> cachedSortedNumericDVs = new HashMap<>();
+
+  private static class SortingBits implements Bits {
+
+    private final Bits in;
+    private final Sorter.DocMap docMap;
+
+    SortingBits(final Bits in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public boolean get(int index) {
+      return in.get(docMap.newToOld(index));
+    }
+
+    @Override
+    public int length() {
+      return in.length();
+    }
+  }
+
+  private static class SortingPointValues extends PointValues {
+
+    private final PointValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingPointValues(final PointValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public void intersect(IntersectVisitor visitor) throws IOException {
+      in.intersect(new IntersectVisitor() {
+                     @Override
+                     public void visit(int docID) throws IOException {
+                       visitor.visit(docMap.oldToNew(docID));
+                     }
+
+                     @Override
+                     public void visit(int docID, byte[] packedValue) throws IOException {
+                       visitor.visit(docMap.oldToNew(docID), packedValue);
+                     }
+
+                     @Override
+                     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                       return visitor.compare(minPackedValue, maxPackedValue);
+                     }
+                   });
+    }
+
+    @Override
+    public long estimatePointCount(IntersectVisitor visitor) {
+      return in.estimatePointCount(visitor);
+    }
+
+    @Override
+    public byte[] getMinPackedValue() throws IOException {
+      return in.getMinPackedValue();
+    }
+
+    @Override
+    public byte[] getMaxPackedValue() throws IOException {
+      return in.getMaxPackedValue();
+    }
+
+    @Override
+    public int getNumDimensions() throws IOException {
+      return in.getNumDimensions();
+    }
+
+    @Override
+    public int getNumIndexDimensions() throws IOException {
+      return in.getNumIndexDimensions();
+    }
+
+    @Override
+    public int getBytesPerDimension() throws IOException {
+      return in.getBytesPerDimension();
+    }
+
+    @Override
+    public long size() {
+      return in.size();
+    }
+
+    @Override
+    public int getDocCount() {
+      return in.getDocCount();
+    }
+  }
+
+
+
+
+
+  /** Return a sorted view of <code>reader</code> according to the order
+   *  defined by <code>sort</code>. If the reader is already sorted, this
+   *  method might return the reader as-is. */
+  public static CodecReader wrap(CodecReader reader, Sort sort) throws IOException {
+    return wrap(reader, new Sorter(sort).sort(reader), sort);
+  }
+
+  /** Expert: same as {@link #wrap(org.apache.lucene.index.CodecReader, Sort)} but operates directly on a {@link Sorter.DocMap}. */
+  static CodecReader wrap(CodecReader reader, Sorter.DocMap docMap, Sort sort) {
+    LeafMetaData metaData = reader.getMetaData();
+    LeafMetaData newMetaData = new LeafMetaData(metaData.getCreatedVersionMajor(), metaData.getMinVersion(), sort);
+    if (docMap == null) {
+      // the reader is already sorted
+      return new FilterCodecReader(reader) {
+        @Override
+        public CacheHelper getCoreCacheHelper() {
+          return null;
+        }
+
+        @Override
+        public CacheHelper getReaderCacheHelper() {
+          return null;
+        }
+
+        @Override
+        public LeafMetaData getMetaData() {
+          return newMetaData;
+        }
+
+        @Override
+        public String toString() {
+          return "SortingCodecReader(" + in + ")";
+        }
+      };
+    }
+    if (reader.maxDoc() != docMap.size()) {
+      throw new IllegalArgumentException("reader.maxDoc() should be equal to docMap.size(), got" + reader.maxDoc() + " != " + docMap.size());
+    }
+    assert Sorter.isConsistent(docMap);
+    return new SortingCodecReader(reader, docMap, newMetaData);
+  }
+
+  final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
+  final LeafMetaData metaData;
+
+  private SortingCodecReader(final CodecReader in, final Sorter.DocMap docMap, LeafMetaData metaData) {
+    super(in);
+    this.docMap = docMap;
+    this.metaData = metaData;
+  }
+
+
+  @Override
+  public FieldsProducer getPostingsReader() {
+    FieldsProducer postingsReader = in.getPostingsReader();
+    return new FieldsProducer() {
+      @Override
+      public void close() throws IOException {
+        postingsReader.close();
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        postingsReader.checkIntegrity();
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        return postingsReader.iterator();
+      }
+
+      @Override
+      public Terms terms(String field) throws IOException {
+        Terms terms = postingsReader.terms(field);
+        return terms == null ? null : new FreqProxTermsWriter.SortingTerms(terms,
+            in.getFieldInfos().fieldInfo(field).getIndexOptions(), docMap);
+      }
+
+      @Override
+      public int size() {
+        return postingsReader.size();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return postingsReader.ramBytesUsed();
+      }
+    };
+  }
+
+  @Override
+  public StoredFieldsReader getFieldsReader() {
+    StoredFieldsReader delegate = in.getFieldsReader();
+    return newStoredFieldsReader(delegate);
+  }
+
+  private StoredFieldsReader newStoredFieldsReader(StoredFieldsReader delegate) {
+    return new StoredFieldsReader() {
+      @Override
+      public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException {
+        in.document(docMap.newToOld(docID), visitor);

Review comment:
       can you call `in.visitDocument` instead?




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


[GitHub] [lucene-solr] msokolov commented on pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#issuecomment-681978792


   It's nice that this change is so straightforward. It makes me realize I don't know what happens today if we specify an index Sort and then open an existing index that is not sorted. Do we throw an error? Should we instead provide a sorted view on the index that can be used to rewrite it?


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


[GitHub] [lucene-solr] s1monw commented on pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#issuecomment-684658664


   @jpountz I applied your comments and moved the inner classes from the reader to the classes where they are mainly used. We should have done that earlier then we would have not kept this unused class. Now that reader only holds private inner classes.


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


[GitHub] [lucene-solr] s1monw commented on pull request #1789: LUCENE-9484: Allow sorting an index after the fact

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1789:
URL: https://github.com/apache/lucene-solr/pull/1789#issuecomment-683654109


   @jpountz I pushed some improvements and inherited from CodecReader, can you take another look


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