LUCENE-10371 Make IndexRearranger able to arrange segment in a determined order (#630)

This commit is contained in:
Patrick Zhai 2022-02-15 10:52:40 -08:00 committed by GitHub
parent 70c152bf32
commit 6157854523
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 51 additions and 11 deletions

View File

@ -175,6 +175,9 @@ Improvements
* LUCENE-10054 Make HnswGraph hierarchical (Mayya Sharipova, Julie Tibshirani, Mike Sokolov,
Adrien Grand)
* LUCENE-10371: Make IndexRearranger able to arrange segment in a determined order.
(Patrick Zhai)
Optimizations
---------------------

View File

@ -32,6 +32,10 @@ import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
@ -44,12 +48,23 @@ import org.apache.lucene.util.NamedThreadFactory;
* <p>TODO: another possible (faster) approach to do this is to manipulate FlushPolicy and
* MergePolicy at indexing time to create small desired segments first and merge them accordingly
* for details please see: https://markmail.org/message/lbtdntclpnocmfuf
*
* @lucene.experimental
*/
public class IndexRearranger {
protected final Directory input, output;
protected final IndexWriterConfig config;
protected final List<DocumentSelector> documentSelectors;
/**
* Constructor
*
* @param input input dir
* @param output output dir
* @param config index writer config
* @param documentSelectors specify what document is desired in the rearranged index segments,
* each selector correspond to one segment
*/
public IndexRearranger(
Directory input,
Directory output,
@ -84,6 +99,34 @@ public class IndexRearranger {
}
executor.shutdown();
}
List<SegmentCommitInfo> ordered = new ArrayList<>();
try (IndexReader reader = DirectoryReader.open(output)) {
for (DocumentSelector ds : documentSelectors) {
int foundLeaf = -1;
for (LeafReaderContext context : reader.leaves()) {
SegmentReader sr = (SegmentReader) context.reader();
int docFound = ds.getFilteredLiveDocs(sr).nextSetBit(0);
if (docFound != DocIdSetIterator.NO_MORE_DOCS) {
if (foundLeaf != -1) {
throw new IllegalStateException(
"Document selector "
+ ds
+ " has matched more than 1 segments. Matched segments order: "
+ foundLeaf
+ ", "
+ context.ord);
}
foundLeaf = context.ord;
ordered.add(sr.getSegmentInfo());
}
}
assert foundLeaf != -1;
}
}
SegmentInfos sis = SegmentInfos.readLatestCommit(output);
sis.clear();
sis.addAll(ordered);
sis.commit(output);
}
private static void addOneSegment(

View File

@ -18,7 +18,6 @@
package org.apache.lucene.misc.index;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
@ -105,24 +104,19 @@ public class TestIndexRearranger extends LuceneTestCase {
private static void assertSequentialIndex(Directory dir, int docNum, int segNum)
throws IOException {
HashSet<Long> seenOrds = new HashSet<>();
IndexReader reader = DirectoryReader.open(dir);
long lastOrd = -1;
for (int i = 0; i < segNum; i++) {
LeafReader leafReader = reader.leaves().get(i).reader();
NumericDocValues numericDocValues = leafReader.getNumericDocValues("ord");
assertTrue(numericDocValues.advanceExact(0));
long lastOrd = numericDocValues.longValue();
seenOrds.add(lastOrd);
for (int doc = 1; doc < leafReader.numDocs(); doc++) {
for (int doc = 0; doc < leafReader.numDocs(); doc++) {
assertTrue(numericDocValues.advanceExact(doc));
assertEquals(numericDocValues.longValue(), lastOrd + 1);
lastOrd = numericDocValues.longValue();
seenOrds.add(lastOrd);
}
}
assertEquals(docNum, seenOrds.size());
assertEquals(docNum, lastOrd + 1);
reader.close();
}
@ -151,7 +145,7 @@ public class TestIndexRearranger extends LuceneTestCase {
w.close();
}
private class OddDocSelector implements IndexRearranger.DocumentSelector {
private static class OddDocSelector implements IndexRearranger.DocumentSelector {
@Override
public BitSet getFilteredLiveDocs(CodecReader reader) throws IOException {
@ -170,7 +164,7 @@ public class TestIndexRearranger extends LuceneTestCase {
}
}
private class EvenDocSelector implements IndexRearranger.DocumentSelector {
private static class EvenDocSelector implements IndexRearranger.DocumentSelector {
@Override
public BitSet getFilteredLiveDocs(CodecReader reader) throws IOException {