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/09/02 10:07:09 UTC

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

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