Break apart reverse index into a separate full index and priority index. It did this before using the same code. This will make the priority index about half as big since it no longer needs to keep metadata.

This commit is contained in:
Viktor Lofgren 2023-03-21 16:12:31 +01:00
parent ca22c287a5
commit 46f81aca2f
23 changed files with 645 additions and 164 deletions

View File

@ -8,4 +8,6 @@ This package contains common models to the search engine
* [EdgeUrl](src/main/java/nu/marginalia/model/EdgeUrl.java)
* [EdgeId](src/main/java/nu/marginalia/model/id/EdgeId.java)
* [DocumentMetadata](src/main/java/nu/marginalia/model/idx/DocumentMetadata.java)
* [DocumentFlags](src/main/java/nu/marginalia/model/idx/DocumentFlags.java)
* [WordMetadata](src/main/java/nu/marginalia/model/idx/WordMetadata.java)
* [WordFlags](src/main/java/nu/marginalia/model/idx/WordFlags.java)

View File

@ -7,6 +7,7 @@ functions based on [POS tags](https://www.ling.upenn.edu/courses/Fall_2003/ling0
## Central Classes
* [DocumentKeywordExtractor](src/main/java/nu/marginalia/keyword/DocumentKeywordExtractor.java)
* [KeywordMetadata](src/main/java/nu/marginalia/keyword/KeywordMetadata.java)
## See Also

View File

@ -1,9 +1,17 @@
# Reverse Index
The reverse index contains a mapping from word to document id. It also provides access to
term-level metadata.
The reverse index contains a mapping from word to document id.
There are two tiers of this index, one priority index which only indexes terms that are flagged with priority flags<sup>1</sup>,
and a full index that indexes all terms. The full index also provides access to term-level metadata, while the priority
index is a binary index.
[1] See WordFlags in [common/model](../../common/model/) and
KeywordMetadata in [features-convert/keyword-extraction](../../features-convert/keyword-extraction).
## Central Classes
* [ReverseIndexConverter](src/main/java/nu/marginalia/index/reverse/ReverseIndexConverter.java) constructs the index.
* [ReverseIndexReader](src/main/java/nu/marginalia/index/reverse/ReverseIndexReader.java) interrogates the index.
* [ReverseIndexFullConverter](src/main/java/nu/marginalia/index/full/ReverseIndexFullConverter.java) constructs the full index.
* [ReverseIndexFullReader](src/main/java/nu/marginalia/index/full/ReverseIndexFullReader.java) interrogates the full index.
* [ReverseIndexPriorityConverter](src/main/java/nu/marginalia/index/priority/ReverseIndexPriorityConverter.java) constructs the priority index.
* [ReverseIndexPriorityReader](src/main/java/nu/marginalia/index/priority/ReverseIndexPriorityReader.java) interrogates the priority index.

View File

@ -0,0 +1,25 @@
package nu.marginalia.index.construction;
import nu.marginalia.array.functional.LongTransformer;
/**
* Transforms an array of item-counts into an array of item-offsets such that the previous counts would fit into an
* array indexed by the generated array.<p>
*
* [ 1, 2, 3, 5, ... ] -> [ 0, 1, 3, 6, 11, ... ]
*
*/
public class CountToOffsetTransformer implements LongTransformer {
long offset = 0;
public final int entrySize;
public CountToOffsetTransformer(int entrySize) {
this.entrySize = entrySize;
}
@Override
public long transform(long pos, long count) {
return (offset += entrySize * count);
}
}

View File

@ -0,0 +1,29 @@
package nu.marginalia.index.construction;
import nu.marginalia.array.functional.LongBinaryOperation;
import nu.marginalia.btree.model.BTreeContext;
/** Calculates the necessary size of an index from an array of offsets (@see CountToOffsetTransformer)<p>
*
* Used with LongArray.fold()
* */
public class IndexSizeEstimator implements LongBinaryOperation {
private final BTreeContext bTreeContext;
private final int entrySize;
public long size = 0;
public IndexSizeEstimator(BTreeContext bTreeContext, int entrySize) {
this.bTreeContext = bTreeContext;
this.entrySize = entrySize;
}
@Override
public long apply(long start, long end) {
if (end == start) return end;
size += bTreeContext.calculateSize((int) (end - start) / entrySize);
return end;
}
}

View File

@ -0,0 +1,49 @@
package nu.marginalia.index.construction;
import nu.marginalia.array.LongArray;
import nu.marginalia.array.functional.LongIOTransformer;
import nu.marginalia.btree.BTreeWriter;
import nu.marginalia.btree.model.BTreeContext;
import nu.marginalia.index.priority.ReverseIndexPriorityParameters;
import java.io.IOException;
import java.nio.channels.FileChannel;
/** Constructs the BTrees in a reverse index */
public class ReverseIndexBTreeTransformer implements LongIOTransformer {
private final BTreeWriter writer;
private final FileChannel intermediateChannel;
private final int entrySize;
long start = 0;
long writeOffset = 0;
public ReverseIndexBTreeTransformer(LongArray urlsFileMap,
int entrySize,
BTreeContext bTreeContext,
FileChannel intermediateChannel) {
this.writer = new BTreeWriter(urlsFileMap, bTreeContext);
this.entrySize = entrySize;
this.intermediateChannel = intermediateChannel;
}
@Override
public long transform(long pos, long end) throws IOException {
final int size = (int) (end - start) / entrySize;
if (size == 0) {
return -1;
}
final long offsetForBlock = writeOffset;
writeOffset += writer.write(writeOffset, size,
mapRegion -> mapRegion.transferFrom(intermediateChannel, start, 0, end - start)
);
start = end;
return offsetForBlock;
}
}

View File

@ -1,6 +1,9 @@
package nu.marginalia.index.reverse;
package nu.marginalia.index.full;
import lombok.SneakyThrows;
import nu.marginalia.index.construction.CountToOffsetTransformer;
import nu.marginalia.index.construction.ReverseIndexBTreeTransformer;
import nu.marginalia.index.construction.IndexSizeEstimator;
import nu.marginalia.index.journal.model.IndexJournalEntryData;
import nu.marginalia.index.journal.model.IndexJournalStatistics;
import nu.marginalia.index.journal.reader.IndexJournalReader;
@ -9,10 +12,6 @@ import nu.marginalia.rwf.RandomWriteFunnel;
import nu.marginalia.array.IntArray;
import nu.marginalia.array.LongArray;
import nu.marginalia.array.algo.SortingContext;
import nu.marginalia.array.functional.LongBinaryIOOperation;
import nu.marginalia.array.functional.LongIOTransformer;
import nu.marginalia.array.functional.LongTransformer;
import nu.marginalia.btree.BTreeWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -22,7 +21,9 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
public class ReverseIndexConverter {
import static nu.marginalia.index.full.ReverseIndexFullParameters.bTreeContext;
public class ReverseIndexFullConverter {
private static final int RWF_BIN_SIZE = 10_000_000;
private final Path tmpFileDir;
@ -35,7 +36,7 @@ public class ReverseIndexConverter {
private final Path outputFileDocs;
private final SortingContext sortingContext;
public ReverseIndexConverter(Path tmpFileDir,
public ReverseIndexFullConverter(Path tmpFileDir,
IndexJournalReader journalReader,
DomainRankings domainRankings,
Path outputFileWords,
@ -70,7 +71,7 @@ public class ReverseIndexConverter {
logger.info("Gathering Offsets");
journalReader.forEachWordId(wordsOffsets::increment);
wordsOffsets.transformEach(0, wordsFileSize, new CountToOffsetTransformer());
wordsOffsets.transformEach(0, wordsFileSize, new CountToOffsetTransformer(ReverseIndexFullParameters.ENTRY_SIZE));
// Construct an intermediate representation of the reverse documents index
try (FileChannel intermediateDocChannel =
@ -95,7 +96,7 @@ public class ReverseIndexConverter {
{
LongArray intermediateDocs = LongArray.mmapForModifying(intermediateUrlsFile);
wordsOffsets.foldIO(0, 0, wordsFileSize, (s, e) -> {
intermediateDocs.sortLargeSpanN(sortingContext, ReverseIndexParameters.ENTRY_SIZE, s, e);
intermediateDocs.sortLargeSpanN(sortingContext, ReverseIndexFullParameters.ENTRY_SIZE, s, e);
return e;
});
intermediateDocs.force();
@ -104,14 +105,17 @@ public class ReverseIndexConverter {
logger.info("Sizing");
SizeEstimator sizeEstimator = new SizeEstimator();
wordsOffsets.foldIO(0, 0, wordsOffsets.size(), sizeEstimator);
IndexSizeEstimator sizeEstimator = new IndexSizeEstimator(
ReverseIndexFullParameters.bTreeContext,
ReverseIndexFullParameters.ENTRY_SIZE);
wordsOffsets.fold(0, 0, wordsOffsets.size(), sizeEstimator);
logger.info("Finalizing Docs File");
LongArray finalDocs = LongArray.mmapForWriting(outputFileDocs, sizeEstimator.size);
// Construct the proper reverse index
wordsOffsets.transformEachIO(0, wordsOffsets.size(), new CreateReverseIndexBTreeTransformer(finalDocs, intermediateDocChannel));
wordsOffsets.transformEachIO(0, wordsOffsets.size(), new ReverseIndexBTreeTransformer(finalDocs, ReverseIndexFullParameters.ENTRY_SIZE, bTreeContext, intermediateDocChannel));
wordsOffsets.write(outputFileWords);
// Attempt to clean up before forcing (important disk space preservation)
@ -130,66 +134,11 @@ public class ReverseIndexConverter {
}
}
private static class SizeEstimator implements LongBinaryIOOperation {
public long size = 0;
@Override
public long apply(long start, long end) {
if (end == start) return end;
size += ReverseIndexParameters.bTreeContext.calculateSize((int) (end - start) / ReverseIndexParameters.ENTRY_SIZE);
return end;
}
}
private void deleteOldFiles() throws IOException {
Files.deleteIfExists(outputFileWords);
Files.deleteIfExists(outputFileDocs);
}
private static class CountToOffsetTransformer implements LongTransformer {
long offset = 0;
@Override
public long transform(long pos, long count) {
return (offset += ReverseIndexParameters.ENTRY_SIZE * count);
}
}
private static class CreateReverseIndexBTreeTransformer implements LongIOTransformer {
private final BTreeWriter writer;
private final FileChannel intermediateChannel;
long start = 0;
long writeOffset = 0;
public CreateReverseIndexBTreeTransformer(LongArray urlsFileMap, FileChannel intermediateChannel) {
this.writer = new BTreeWriter(urlsFileMap, ReverseIndexParameters.bTreeContext);
this.intermediateChannel = intermediateChannel;
}
@Override
public long transform(long pos, long end) throws IOException {
assert (end - start) % ReverseIndexParameters.ENTRY_SIZE == 0;
final int size = (int)(end - start) / ReverseIndexParameters.ENTRY_SIZE;
if (size == 0) {
return -1;
}
final long offsetForBlock = writeOffset;
writeOffset += writer.write(writeOffset, size,
mapRegion -> mapRegion.transferFrom(intermediateChannel, start, 0, end - start)
);
start = end;
return offsetForBlock;
}
}
private class IntermediateIndexConstructor implements IndexJournalReader.LongObjectConsumer<IndexJournalEntryData.Record>, AutoCloseable {
private final LongArray wordRangeEnds;

View File

@ -1,27 +1,30 @@
package nu.marginalia.index.reverse.query;
package nu.marginalia.index.full;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.btree.BTreeReader;
import nu.marginalia.index.query.EntrySource;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import static java.lang.Math.min;
public class ReverseIndexEntrySource implements EntrySource {
public class ReverseIndexFullEntrySource implements EntrySource {
private final BTreeReader reader;
private static final int ENTRY_SIZE = 2;
int pos;
int endOffset;
final int entrySize;
private final ReverseIndexEntrySourceBehavior behavior;
public ReverseIndexEntrySource(BTreeReader reader, ReverseIndexEntrySourceBehavior behavior) {
public ReverseIndexFullEntrySource(BTreeReader reader,
int entrySize,
ReverseIndexEntrySourceBehavior behavior) {
this.reader = reader;
this.behavior = behavior;
this.entrySize = entrySize;
pos = 0;
endOffset = pos + ENTRY_SIZE*reader.numEntries();
endOffset = pos + entrySize * reader.numEntries();
}
@Override
@ -39,9 +42,7 @@ public class ReverseIndexEntrySource implements EntrySource {
}
buffer.end = min(buffer.end, endOffset - pos);
reader.readData(buffer.data, buffer.end, pos);
pos += buffer.end;
destagger(buffer);
@ -49,14 +50,14 @@ public class ReverseIndexEntrySource implements EntrySource {
}
private void destagger(LongQueryBuffer buffer) {
if (ENTRY_SIZE == 1)
if (entrySize == 1)
return;
for (int ri = ENTRY_SIZE, wi=1; ri < buffer.end ; ri+=ENTRY_SIZE, wi++) {
for (int ri = entrySize, wi=1; ri < buffer.end ; ri+=entrySize, wi++) {
buffer.data[wi] = buffer.data[ri];
}
buffer.end /= ENTRY_SIZE;
buffer.end /= entrySize;
}
@Override
@ -64,9 +65,4 @@ public class ReverseIndexEntrySource implements EntrySource {
return pos < endOffset;
}
@Override
public String toString() {
return String.format("BTreeRange.EntrySource(@" + pos + ": " + endOffset + ")");
}
}

View File

@ -1,16 +1,16 @@
package nu.marginalia.index.reverse;
package nu.marginalia.index.full;
import nu.marginalia.btree.model.BTreeBlockSize;
import nu.marginalia.btree.model.BTreeContext;
class ReverseIndexParameters {
public static final int ENTRY_SIZE = 2;
public class ReverseIndexFullParameters {
static final int ENTRY_SIZE = 2;
// This is the byte size per index page on disk, the data pages are twice as large due to ENTRY_SIZE = 2.
//
// Given a hardware limit of 4k reads, 2k block size should be optimal.
public static final BTreeBlockSize blockSize = BTreeBlockSize.BS_2048;
static final BTreeBlockSize blockSize = BTreeBlockSize.BS_2048;
public static final BTreeContext bTreeContext = new BTreeContext(5, ENTRY_SIZE, blockSize);
static final BTreeContext bTreeContext = new BTreeContext(5, ENTRY_SIZE, blockSize);
}

View File

@ -1,9 +1,8 @@
package nu.marginalia.index.reverse;
package nu.marginalia.index.full;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySource;
import nu.marginalia.index.reverse.query.ReverseIndexRejectFilter;
import nu.marginalia.index.reverse.query.ReverseIndexRetainFilter;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.query.ReverseIndexRejectFilter;
import nu.marginalia.index.query.ReverseIndexRetainFilter;
import nu.marginalia.array.LongArray;
import nu.marginalia.btree.BTreeReader;
import nu.marginalia.index.query.EmptyEntrySource;
@ -19,13 +18,13 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
public class ReverseIndexReader {
public class ReverseIndexFullReader {
private final LongArray words;
private final LongArray documents;
private final Logger logger = LoggerFactory.getLogger(getClass());
public ReverseIndexReader(Path words, Path documents) throws IOException {
public ReverseIndexFullReader(Path words, Path documents) throws IOException {
if (!Files.exists(words) || !Files.exists(documents)) {
this.words = null;
this.documents = null;
@ -64,7 +63,7 @@ public class ReverseIndexReader {
if (offset < 0) return new EmptyEntrySource();
return new ReverseIndexEntrySource(createReaderNew(offset), behavior);
return new ReverseIndexFullEntrySource(createReaderNew(offset), ReverseIndexFullParameters.ENTRY_SIZE, behavior);
}
public QueryFilterStepIf also(int wordId) {
@ -100,7 +99,7 @@ public class ReverseIndexReader {
}
private BTreeReader createReaderNew(long offset) {
return new BTreeReader(documents, ReverseIndexParameters.bTreeContext, offset);
return new BTreeReader(documents, ReverseIndexFullParameters.bTreeContext, offset);
}
public long[] getTermMeta(int wordId, long[] docIds) {

View File

@ -0,0 +1,197 @@
package nu.marginalia.index.priority;
import lombok.SneakyThrows;
import nu.marginalia.array.IntArray;
import nu.marginalia.array.LongArray;
import nu.marginalia.array.algo.SortingContext;
import nu.marginalia.index.construction.CountToOffsetTransformer;
import nu.marginalia.index.construction.ReverseIndexBTreeTransformer;
import nu.marginalia.index.construction.IndexSizeEstimator;
import nu.marginalia.index.journal.model.IndexJournalEntryData;
import nu.marginalia.index.journal.model.IndexJournalStatistics;
import nu.marginalia.index.journal.reader.IndexJournalReader;
import nu.marginalia.ranking.DomainRankings;
import nu.marginalia.rwf.RandomWriteFunnel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
public class ReverseIndexPriorityConverter {
private static final int RWF_BIN_SIZE = 10_000_000;
private final Path tmpFileDir;
private final Logger logger = LoggerFactory.getLogger(getClass());
private final IndexJournalReader journalReader;
private final DomainRankings domainRankings;
private final Path outputFileWords;
private final Path outputFileDocs;
private final SortingContext sortingContext;
public ReverseIndexPriorityConverter(Path tmpFileDir,
IndexJournalReader journalReader,
DomainRankings domainRankings,
Path outputFileWords,
Path outputFileDocs) {
this.tmpFileDir = tmpFileDir;
this.journalReader = journalReader;
this.domainRankings = domainRankings;
this.outputFileWords = outputFileWords;
this.outputFileDocs = outputFileDocs;
this.sortingContext = new SortingContext(tmpFileDir, 64_000);
}
public void convert() throws IOException {
deleteOldFiles();
if (journalReader.fileHeader().fileSize() <= IndexJournalReader.FILE_HEADER_SIZE_BYTES) {
logger.warn("Bailing: Journal is empty!");
return;
}
final IndexJournalStatistics statistics = journalReader.getStatistics();
final Path intermediateUrlsFile = Files.createTempFile(tmpFileDir, "urls-sorted", ".dat");
try {
final long wordsFileSize = statistics.highestWord() + 1;
logger.debug("Words file size: {}", wordsFileSize);
// Create a count of how many documents has contains each word
final LongArray wordsOffsets = LongArray.allocate(wordsFileSize);
logger.info("Gathering Offsets");
journalReader.forEachWordId(wordsOffsets::increment);
wordsOffsets.transformEach(0, wordsFileSize, new CountToOffsetTransformer(ReverseIndexPriorityParameters.ENTRY_SIZE));
// Construct an intermediate representation of the reverse documents index
try (FileChannel intermediateDocChannel =
(FileChannel) Files.newByteChannel(intermediateUrlsFile,
StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE))
{
logger.info("Creating Intermediate Docs File");
// Construct intermediate index
try (RandomWriteFunnel intermediateDocumentWriteFunnel = new RandomWriteFunnel(tmpFileDir, RWF_BIN_SIZE);
IntermediateIndexConstructor intermediateIndexConstructor = new IntermediateIndexConstructor(tmpFileDir, wordsOffsets, intermediateDocumentWriteFunnel)
)
{
journalReader.forEachDocIdRecord(intermediateIndexConstructor);
intermediateDocumentWriteFunnel.write(intermediateDocChannel);
}
intermediateDocChannel.force(false);
logger.info("Sorting Intermediate Docs File");
// Sort each segment of the intermediate file
{
LongArray intermediateDocs = LongArray.mmapForModifying(intermediateUrlsFile);
wordsOffsets.foldIO(0, 0, wordsFileSize, (s, e) -> {
intermediateDocs.sortLargeSpan(sortingContext, s, e);
return e;
});
intermediateDocs.force();
}
logger.info("Sizing");
IndexSizeEstimator indexSizeEstimator = new IndexSizeEstimator(
ReverseIndexPriorityParameters.bTreeContext,
ReverseIndexPriorityParameters.ENTRY_SIZE);
wordsOffsets.fold(0, 0, wordsOffsets.size(), indexSizeEstimator);
logger.info("Finalizing Docs File");
LongArray finalDocs = LongArray.mmapForWriting(outputFileDocs, indexSizeEstimator.size);
// Construct the proper reverse index
wordsOffsets.transformEachIO(0, wordsOffsets.size(),
new ReverseIndexBTreeTransformer(finalDocs,
ReverseIndexPriorityParameters.ENTRY_SIZE,
ReverseIndexPriorityParameters.bTreeContext,
intermediateDocChannel));
wordsOffsets.write(outputFileWords);
// Attempt to clean up before forcing (important disk space preservation)
Files.deleteIfExists(intermediateUrlsFile);
wordsOffsets.force();
finalDocs.force();
logger.info("Done");
}
} catch (IOException ex) {
logger.error("Failed to convert", ex);
throw ex;
} finally {
Files.deleteIfExists(intermediateUrlsFile);
}
}
private void deleteOldFiles() throws IOException {
Files.deleteIfExists(outputFileWords);
Files.deleteIfExists(outputFileDocs);
}
private class IntermediateIndexConstructor implements IndexJournalReader.LongObjectConsumer<IndexJournalEntryData.Record>, AutoCloseable {
private final LongArray wordRangeEnds;
private final IntArray wordRangeOffset;
private final RandomWriteFunnel documentsFile;
private final Path tempFile;
public IntermediateIndexConstructor(Path tempDir, LongArray wordRangeEnds, RandomWriteFunnel documentsFile) throws IOException {
tempFile = Files.createTempFile(tempDir, "iic", "dat");
this.wordRangeEnds = wordRangeEnds;
this.wordRangeOffset = IntArray.mmapForWriting(tempFile, wordRangeEnds.size());
this.documentsFile = documentsFile;
}
@SneakyThrows
@Override
public void accept(long docId, IndexJournalEntryData.Record record) {
/* Encode the ID as
*
* 32 bits 32 bits
* [ ranking | url-id ]
*
* in order to get low-ranking documents to be considered first
* when sorting the items.
*/
int domainId = (int) (docId >>> 32);
long rankingId = (long) domainRankings.getRanking(domainId) << 32;
int urlId = (int) (docId & 0xFFFF_FFFFL);
long rankEncodedId = rankingId | urlId;
final int wordId = record.wordId();
long offset = startOfRange(wordId);
documentsFile.put(offset + wordRangeOffset.getAndIncrement(wordId), rankEncodedId);
}
private long startOfRange(int wordId) {
if (wordId == 0) return 0;
return wordRangeEnds.get(wordId - 1);
}
public void close() throws IOException {
Files.delete(tempFile);
}
}
}

View File

@ -0,0 +1,52 @@
package nu.marginalia.index.priority;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.btree.BTreeReader;
import nu.marginalia.index.query.EntrySource;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import static java.lang.Math.min;
public class ReverseIndexPriorityEntrySource implements EntrySource {
private final BTreeReader reader;
int pos;
int endOffset;
private final ReverseIndexEntrySourceBehavior behavior;
public ReverseIndexPriorityEntrySource(BTreeReader reader, ReverseIndexEntrySourceBehavior behavior) {
this.reader = reader;
this.behavior = behavior;
pos = 0;
endOffset = pos + reader.numEntries();
}
@Override
public void skip(int n) {
pos += n;
}
@Override
public void read(LongQueryBuffer buffer) {
if (behavior == ReverseIndexEntrySourceBehavior.DO_NOT_PREFER
&& buffer.hasRetainedData())
{
pos = endOffset;
return;
}
buffer.end = min(buffer.end, endOffset - pos);
reader.readData(buffer.data, buffer.end, pos);
pos += buffer.end;
buffer.uniq();
}
@Override
public boolean hasMore() {
return pos < endOffset;
}
}

View File

@ -1,9 +1,16 @@
package nu.marginalia.index.reverse;
package nu.marginalia.index.priority;
import nu.marginalia.btree.model.BTreeBlockSize;
import nu.marginalia.btree.model.BTreeContext;
import nu.marginalia.index.journal.model.IndexJournalEntryData;
import nu.marginalia.model.idx.WordFlags;
public class ReverseIndexPriorityParameters {
static final int ENTRY_SIZE = 1;
static final BTreeBlockSize blockSize = BTreeBlockSize.BS_4096;
static final BTreeContext bTreeContext = new BTreeContext(5, ENTRY_SIZE, blockSize);
private static final long highPriorityFlags =
WordFlags.Title.asBit()
| WordFlags.Subjects.asBit()

View File

@ -1,7 +1,6 @@
package nu.marginalia.index.reverse;
package nu.marginalia.index.priority;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySource;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.query.EntrySource;
import nu.marginalia.array.LongArray;
import nu.marginalia.btree.BTreeReader;
@ -13,13 +12,13 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
public class ReverseIndexPrioReader {
public class ReverseIndexPriorityReader {
private final LongArray words;
private final LongArray documents;
private final Logger logger = LoggerFactory.getLogger(getClass());
public ReverseIndexPrioReader(Path words, Path documents) throws IOException {
public ReverseIndexPriorityReader(Path words, Path documents) throws IOException {
if (!Files.exists(words) || !Files.exists(documents)) {
this.words = null;
this.documents = null;
@ -44,10 +43,10 @@ public class ReverseIndexPrioReader {
if (offset < 0) return new EmptyEntrySource();
return new ReverseIndexEntrySource(createReaderNew(offset), ReverseIndexEntrySourceBehavior.DO_PREFER);
return new ReverseIndexPriorityEntrySource(createReaderNew(offset), ReverseIndexEntrySourceBehavior.DO_PREFER);
}
private BTreeReader createReaderNew(long offset) {
return new BTreeReader(documents, ReverseIndexParameters.bTreeContext, offset);
return new BTreeReader(documents, ReverseIndexPriorityParameters.bTreeContext, offset);
}
}

View File

@ -1,4 +1,4 @@
package nu.marginalia.index.reverse.query;
package nu.marginalia.index.query;
public enum ReverseIndexEntrySourceBehavior {
/** Eagerly read from this entry source */

View File

@ -1,4 +1,4 @@
package nu.marginalia.index.reverse.query;
package nu.marginalia.index.query;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.btree.BTreeReader;

View File

@ -1,4 +1,4 @@
package nu.marginalia.index.reverse.query;
package nu.marginalia.index.query;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.btree.BTreeReader;

View File

@ -2,11 +2,13 @@ package nu.marginalia.index.reverse;
import lombok.SneakyThrows;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.index.full.ReverseIndexFullConverter;
import nu.marginalia.index.full.ReverseIndexFullReader;
import nu.marginalia.index.journal.model.IndexJournalEntry;
import nu.marginalia.index.journal.reader.IndexJournalReaderSingleCompressedFile;
import nu.marginalia.index.journal.writer.IndexJournalWriterImpl;
import nu.marginalia.index.journal.writer.IndexJournalWriter;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.ranking.DomainRankings;
import nu.marginalia.lexicon.KeywordLexicon;
import nu.marginalia.lexicon.journal.KeywordLexiconJournal;
@ -25,7 +27,7 @@ import java.util.stream.LongStream;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
class ReverseIndexConverterTest {
class ReverseIndexFullConverterTest {
KeywordLexicon keywordLexicon;
Path indexFile;
@ -82,10 +84,10 @@ class ReverseIndexConverterTest {
var docsFile = dataDir.resolve("docs.dat");
var journalReader = new IndexJournalReaderSingleCompressedFile(indexFile);
new ReverseIndexConverter(tmpDir, journalReader, new DomainRankings(), wordsFile, docsFile)
new ReverseIndexFullConverter(tmpDir, journalReader, new DomainRankings(), wordsFile, docsFile)
.convert();
var reverseIndexReader = new ReverseIndexReader(wordsFile, docsFile);
var reverseIndexReader = new ReverseIndexFullReader(wordsFile, docsFile);
System.out.println(reverseIndexReader.numDocuments(keywordLexicon.getReadOnly("1")));
System.out.println(reverseIndexReader.numDocuments(keywordLexicon.getReadOnly("2")));

View File

@ -2,12 +2,15 @@ package nu.marginalia.index.reverse;
import lombok.SneakyThrows;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.index.full.ReverseIndexFullConverter;
import nu.marginalia.index.full.ReverseIndexFullReader;
import nu.marginalia.index.journal.model.IndexJournalEntryData;
import nu.marginalia.index.journal.model.IndexJournalEntryHeader;
import nu.marginalia.index.journal.reader.IndexJournalReaderSingleCompressedFile;
import nu.marginalia.index.journal.writer.IndexJournalWriterImpl;
import nu.marginalia.index.journal.writer.IndexJournalWriter;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.priority.ReverseIndexPriorityParameters;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.ranking.DomainRankings;
import nu.marginalia.lexicon.KeywordLexicon;
import nu.marginalia.lexicon.journal.KeywordLexiconJournal;
@ -25,7 +28,7 @@ import java.util.Arrays;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
class ReverseIndexConverterTest2 {
class ReverseIndexFullConverterTest2 {
KeywordLexicon keywordLexicon;
IndexJournalWriter writer;
@ -114,9 +117,9 @@ class ReverseIndexConverterTest2 {
Path tmpDir = Path.of("/tmp");
new ReverseIndexConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile), new DomainRankings(), wordsFile, docsFile).convert();
new ReverseIndexFullConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile), new DomainRankings(), wordsFile, docsFile).convert();
var reverseReader = new ReverseIndexReader(wordsFile, docsFile);
var reverseReader = new ReverseIndexFullReader(wordsFile, docsFile);
for (int i = workSetStart; i < workSetSize; i++) {
@ -139,9 +142,9 @@ class ReverseIndexConverterTest2 {
Path tmpDir = Path.of("/tmp");
new ReverseIndexConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile, null, ReverseIndexPriorityParameters::filterPriorityRecord), new DomainRankings(), wordsFile, docsFile).convert();
new ReverseIndexFullConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile, null, ReverseIndexPriorityParameters::filterPriorityRecord), new DomainRankings(), wordsFile, docsFile).convert();
var reverseReader = new ReverseIndexReader(wordsFile, docsFile);
var reverseReader = new ReverseIndexFullReader(wordsFile, docsFile);
for (int i = workSetStart; i < workSetSize; i++) {

View File

@ -0,0 +1,163 @@
package nu.marginalia.index.reverse;
import lombok.SneakyThrows;
import nu.marginalia.array.buffer.LongQueryBuffer;
import nu.marginalia.index.journal.model.IndexJournalEntryData;
import nu.marginalia.index.journal.model.IndexJournalEntryHeader;
import nu.marginalia.index.journal.reader.IndexJournalReaderSingleCompressedFile;
import nu.marginalia.index.journal.writer.IndexJournalWriter;
import nu.marginalia.index.journal.writer.IndexJournalWriterImpl;
import nu.marginalia.index.priority.ReverseIndexPriorityReader;
import nu.marginalia.index.priority.ReverseIndexPriorityConverter;
import nu.marginalia.index.priority.ReverseIndexPriorityParameters;
import nu.marginalia.lexicon.KeywordLexicon;
import nu.marginalia.lexicon.journal.KeywordLexiconJournal;
import nu.marginalia.ranking.DomainRankings;
import nu.marginalia.test.TestUtil;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
class ReverseIndexPriorityConverterTest2 {
KeywordLexicon keywordLexicon;
IndexJournalWriter writer;
Path indexFile;
Path wordsFile1;
Path urlsFile1;
Path dictionaryFile;
private final Logger logger = LoggerFactory.getLogger(getClass());
Path dataDir;
private Path wordsFile;
private Path docsFile;
int workSetSize = 8192;
int workSetStart = 8000;
@BeforeEach
@SneakyThrows
void setUp() {
dictionaryFile = Files.createTempFile("tmp", ".dict");
dictionaryFile.toFile().deleteOnExit();
keywordLexicon = new KeywordLexicon(new KeywordLexiconJournal(dictionaryFile.toFile()));
keywordLexicon.getOrInsert("0");
indexFile = Files.createTempFile("tmp", ".idx");
indexFile.toFile().deleteOnExit();
writer = new IndexJournalWriterImpl(keywordLexicon, indexFile);
wordsFile1 = Files.createTempFile("words1", ".idx");
urlsFile1 = Files.createTempFile("urls1", ".idx");
dataDir = Files.createTempDirectory(getClass().getSimpleName());
for (int i = 1; i < workSetSize; i++) {
if (i < workSetStart) {
keywordLexicon.getOrInsert(Integer.toString(i));
}
else {
createEntry(writer, keywordLexicon, i);
}
}
keywordLexicon.commitToDisk();
Thread.sleep(1000);
writer.forceWrite();
var reader = new IndexJournalReaderSingleCompressedFile(indexFile);
wordsFile = dataDir.resolve("words.dat");
docsFile = dataDir.resolve("docs.dat");
}
@AfterEach
public void tearDown() {
TestUtil.clearTempDir(dataDir);
}
public int[] getFactorsI(int id) {
return IntStream.rangeClosed(1, id-1).toArray();
}
public long[] getFactorsL(int id) {
return LongStream.rangeClosed(1, id).filter(v -> (id % v) == 0).toArray();
}
long createId(long url, long domain) {
return (domain << 32) | url;
}
public void createEntry(IndexJournalWriter writer, KeywordLexicon keywordLexicon, int id) {
int[] factors = getFactorsI(id);
var header = new IndexJournalEntryHeader(factors.length, createId(id, id/20), id % 5);
long[] data = new long[factors.length*2];
for (int i = 0; i < factors.length; i++) {
data[2*i] = keywordLexicon.getOrInsert(Integer.toString(factors[i]));
data[2*i + 1] = (i % 21 != 0) ? 0 : -factors[i];
}
writer.put(header, new IndexJournalEntryData(data));
}
@Test
void testRev2() throws IOException {
Path tmpDir = Path.of("/tmp");
new ReverseIndexPriorityConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile), new DomainRankings(), wordsFile, docsFile).convert();
var reverseReader = new ReverseIndexPriorityReader(wordsFile, docsFile);
for (int i = workSetStart; i < workSetSize; i++) {
var es = reverseReader.priorityDocuments(i);
LongQueryBuffer lqb = new LongQueryBuffer(100);
while (es.hasMore()) {
lqb.reset();
es.read(lqb);
System.out.println(Arrays.toString(Arrays.copyOf(lqb.data, lqb.end)));
}
System.out.println("--");
}
TestUtil.clearTempDir(dataDir);
}
@Test
void testRevP() throws IOException {
Path tmpDir = Path.of("/tmp");
new ReverseIndexPriorityConverter(tmpDir, new IndexJournalReaderSingleCompressedFile(indexFile, null, ReverseIndexPriorityParameters::filterPriorityRecord), new DomainRankings(), wordsFile, docsFile).convert();
var reverseReader = new ReverseIndexPriorityReader(wordsFile, docsFile);
for (int i = workSetStart; i < workSetSize; i++) {
var es = reverseReader.priorityDocuments(i);
LongQueryBuffer lqb = new LongQueryBuffer(100);
while (es.hasMore()) {
lqb.reset();
es.read(lqb);
System.out.println(Arrays.toString(Arrays.copyOf(lqb.data, lqb.end)));
}
System.out.println("--");
}
TestUtil.clearTempDir(dataDir);
}
}

View File

@ -8,10 +8,11 @@ import nu.marginalia.index.forward.ForwardIndexReader;
import nu.marginalia.index.journal.reader.IndexJournalReaderSingleCompressedFile;
import nu.marginalia.index.journal.writer.IndexJournalWriter;
import nu.marginalia.index.journal.writer.IndexJournalWriterImpl;
import nu.marginalia.index.reverse.ReverseIndexConverter;
import nu.marginalia.index.reverse.ReverseIndexPrioReader;
import nu.marginalia.index.reverse.ReverseIndexPriorityParameters;
import nu.marginalia.index.reverse.ReverseIndexReader;
import nu.marginalia.index.priority.ReverseIndexPriorityConverter;
import nu.marginalia.index.full.ReverseIndexFullConverter;
import nu.marginalia.index.priority.ReverseIndexPriorityReader;
import nu.marginalia.index.priority.ReverseIndexPriorityParameters;
import nu.marginalia.index.full.ReverseIndexFullReader;
import nu.marginalia.lexicon.KeywordLexicon;
import nu.marginalia.ranking.DomainRankings;
import nu.marginalia.index.index.SearchIndexReader;
@ -110,7 +111,7 @@ public class IndexServicesFactory {
logger.info("Converting full reverse index {}", source);
var journalReader = new IndexJournalReaderSingleCompressedFile(source);
var converter = new ReverseIndexConverter(tmpFileDir,
var converter = new ReverseIndexFullConverter(tmpFileDir,
journalReader,
domainRankings,
revIndexWords.get(NEXT_PART).toPath(),
@ -127,9 +128,10 @@ public class IndexServicesFactory {
logger.info("Converting priority reverse index {}", source);
var journalReader = new IndexJournalReaderSingleCompressedFile(source, null, ReverseIndexPriorityParameters::filterPriorityRecord);
var journalReader = new IndexJournalReaderSingleCompressedFile(source, null,
ReverseIndexPriorityParameters::filterPriorityRecord);
var converter = new ReverseIndexConverter(tmpFileDir,
var converter = new ReverseIndexPriorityConverter(tmpFileDir,
journalReader,
domainRankings,
revPrioIndexWords.get(NEXT_PART).toPath(),
@ -164,17 +166,16 @@ public class IndexServicesFactory {
throw new RuntimeException(e);
}
System.runFinalization();
System.gc();
}
public ReverseIndexReader getReverseIndexReader() throws IOException {
return new ReverseIndexReader(
public ReverseIndexFullReader getReverseIndexReader() throws IOException {
return new ReverseIndexFullReader(
revIndexWords.get(LIVE_PART).toPath(),
revIndexDoc.get(LIVE_PART).toPath());
}
public ReverseIndexPrioReader getReverseIndexPrioReader() throws IOException {
return new ReverseIndexPrioReader(
public ReverseIndexPriorityReader getReverseIndexPrioReader() throws IOException {
return new ReverseIndexPriorityReader(
revPrioIndexWords.get(LIVE_PART).toPath(),
revPrioIndexDoc.get(LIVE_PART).toPath());
}

View File

@ -3,27 +3,27 @@ package nu.marginalia.index.index;
import nu.marginalia.index.query.IndexQuery;
import nu.marginalia.index.query.IndexQueryBuilder;
import nu.marginalia.index.query.filter.QueryFilterStepIf;
import nu.marginalia.index.reverse.ReverseIndexReader;
import nu.marginalia.index.full.ReverseIndexFullReader;
public class SearchIndexQueryBuilder implements IndexQueryBuilder {
private final IndexQuery query;
private final ReverseIndexReader reverseIndexReader;
private final ReverseIndexFullReader reverseIndexFullReader;
SearchIndexQueryBuilder(ReverseIndexReader reverseIndexReader, IndexQuery query) {
SearchIndexQueryBuilder(ReverseIndexFullReader reverseIndexFullReader, IndexQuery query) {
this.query = query;
this.reverseIndexReader = reverseIndexReader;
this.reverseIndexFullReader = reverseIndexFullReader;
}
public IndexQueryBuilder also(int termId) {
query.addInclusionFilter(reverseIndexReader.also(termId));
query.addInclusionFilter(reverseIndexFullReader.also(termId));
return this;
}
public IndexQueryBuilder not(int termId) {
query.addInclusionFilter(reverseIndexReader.not(termId));
query.addInclusionFilter(reverseIndexFullReader.not(termId));
return this;
}

View File

@ -1,6 +1,5 @@
package nu.marginalia.index.index;
import lombok.SneakyThrows;
import nu.marginalia.index.forward.ForwardIndexReader;
import nu.marginalia.index.forward.ParamMatchingQueryFilter;
import nu.marginalia.index.query.EntrySource;
@ -8,9 +7,9 @@ import nu.marginalia.index.query.IndexQuery;
import nu.marginalia.index.query.IndexQueryBuilder;
import nu.marginalia.index.query.IndexQueryParams;
import nu.marginalia.index.query.filter.QueryFilterStepIf;
import nu.marginalia.index.reverse.ReverseIndexPrioReader;
import nu.marginalia.index.reverse.ReverseIndexReader;
import nu.marginalia.index.reverse.query.ReverseIndexEntrySourceBehavior;
import nu.marginalia.index.priority.ReverseIndexPriorityReader;
import nu.marginalia.index.full.ReverseIndexFullReader;
import nu.marginalia.index.query.ReverseIndexEntrySourceBehavior;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -21,33 +20,33 @@ public class SearchIndexReader {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final ForwardIndexReader forwardIndexReader;
private final ReverseIndexReader reverseIndexReader;
private final ReverseIndexPrioReader reverseIndexPrioReader;
private final ReverseIndexFullReader reverseIndexFullReader;
private final ReverseIndexPriorityReader reverseIndexPriorityReader;
public SearchIndexReader(ForwardIndexReader forwardIndexReader,
ReverseIndexReader reverseIndexReader,
ReverseIndexPrioReader reverseIndexPrioReader) {
ReverseIndexFullReader reverseIndexFullReader,
ReverseIndexPriorityReader reverseIndexPriorityReader) {
this.forwardIndexReader = forwardIndexReader;
this.reverseIndexReader = reverseIndexReader;
this.reverseIndexPrioReader = reverseIndexPrioReader;
this.reverseIndexFullReader = reverseIndexFullReader;
this.reverseIndexPriorityReader = reverseIndexPriorityReader;
}
public IndexQueryBuilder findWordAsSentence(int[] wordIdsByFrequency) {
List<EntrySource> entrySources = new ArrayList<>(1);
entrySources.add(reverseIndexReader.documents(wordIdsByFrequency[0], ReverseIndexEntrySourceBehavior.DO_PREFER));
entrySources.add(reverseIndexFullReader.documents(wordIdsByFrequency[0], ReverseIndexEntrySourceBehavior.DO_PREFER));
return new SearchIndexQueryBuilder(reverseIndexReader, new IndexQuery(entrySources));
return new SearchIndexQueryBuilder(reverseIndexFullReader, new IndexQuery(entrySources));
}
public IndexQueryBuilder findWordAsTopic(int[] wordIdsByFrequency) {
List<EntrySource> entrySources = new ArrayList<>(wordIdsByFrequency.length);
for (int wordId : wordIdsByFrequency) {
entrySources.add(reverseIndexPrioReader.priorityDocuments(wordId));
entrySources.add(reverseIndexPriorityReader.priorityDocuments(wordId));
}
return new SearchIndexQueryBuilder(reverseIndexReader, new IndexQuery(entrySources));
return new SearchIndexQueryBuilder(reverseIndexFullReader, new IndexQuery(entrySources));
}
public IndexQueryBuilder findWordTopicDynamicMode(int[] wordIdsByFrequency) {
@ -58,12 +57,12 @@ public class SearchIndexReader {
List<EntrySource> entrySources = new ArrayList<>(wordIdsByFrequency.length + 1);
for (int wordId : wordIdsByFrequency) {
entrySources.add(reverseIndexPrioReader.priorityDocuments(wordId));
entrySources.add(reverseIndexPriorityReader.priorityDocuments(wordId));
}
entrySources.add(reverseIndexReader.documents(wordIdsByFrequency[0], ReverseIndexEntrySourceBehavior.DO_NOT_PREFER));
entrySources.add(reverseIndexFullReader.documents(wordIdsByFrequency[0], ReverseIndexEntrySourceBehavior.DO_NOT_PREFER));
return new SearchIndexQueryBuilder(reverseIndexReader, new IndexQuery(entrySources));
return new SearchIndexQueryBuilder(reverseIndexFullReader, new IndexQuery(entrySources));
}
QueryFilterStepIf filterForParams(IndexQueryParams params) {
@ -71,11 +70,11 @@ public class SearchIndexReader {
}
public long numHits(int word) {
return reverseIndexReader.numDocuments(word);
return reverseIndexFullReader.numDocuments(word);
}
public long[] getMetadata(int wordId, long[] docIds) {
return reverseIndexReader.getTermMeta(wordId, docIds);
return reverseIndexFullReader.getTermMeta(wordId, docIds);
}
public long getDocumentMetadata(long docId) {