diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index fd2e8d02e15..b6a600dda48 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -243,6 +243,9 @@ Optimizations * LUCENE-8607: MatchAllDocsQuery can shortcut when total hit count is not required (Alan Woodward, Adrien Grand) +* LUCENE-8585: Index-time jump-tables for DocValues, for O(1) advance when retrieving doc values. + (Toke Eskildsen, Adrien Grand) + ======================= Lucene 7.7.0 ======================= Changes in Runtime Behavior diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat index 4a812de77e5..20463c5dc45 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat @@ -13,3 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java similarity index 100% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java similarity index 100% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java similarity index 100% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java index 9170c69bb3d..e63873a63c6 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java @@ -17,7 +17,7 @@ /** * Components from the Lucene 5.0 index format - * See {@link org.apache.lucene.codecs.lucene50} for an overview + * See {@link org.apache.lucene.codecs.lucene80} for an overview * of the index format. */ package org.apache.lucene.codecs.lucene50; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java index a914001d9d2..b7145ccea94 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java @@ -16,7 +16,7 @@ */ /** - * Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene70} + * Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene80} * for an overview of the current index format. */ package org.apache.lucene.codecs.lucene60; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java index 08dd72d3100..e1913a0c496 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java @@ -16,390 +16,7 @@ */ /** - * Lucene 7.0 file format. - * - *

Apache Lucene - Index File Formats

- *
- * - *
- * - *

Introduction

- *
- *

This document defines the index file formats used in this version of Lucene. - * If you are using a different version of Lucene, please consult the copy of - * docs/ that was distributed with - * the version you are using.

- *

This document attempts to provide a high-level definition of the Apache - * Lucene file formats.

- *
- * - *

Definitions

- *
- *

The fundamental concepts in Lucene are index, document, field and term.

- *

An index contains a sequence of documents.

- * - *

The same sequence of bytes in two different fields is considered a different - * term. Thus terms are represented as a pair: the string naming the field, and the - * bytes within the field.

- * - *

Inverted Indexing

- *

The index stores statistics about terms in order to make term-based search - * more efficient. Lucene's index falls into the family of indexes known as an - * inverted index. This is because it can list, for a term, the documents - * that contain it. This is the inverse of the natural relationship, in which - * documents list terms.

- * - *

Types of Fields

- *

In Lucene, fields may be stored, in which case their text is stored - * in the index literally, in a non-inverted manner. Fields that are inverted are - * called indexed. A field may be both stored and indexed.

- *

The text of a field may be tokenized into terms to be indexed, or the - * text of a field may be used literally as a term to be indexed. Most fields are - * tokenized, but sometimes it is useful for certain identifier fields to be - * indexed literally.

- *

See the {@link org.apache.lucene.document.Field Field} - * java docs for more information on Fields.

- * - *

Segments

- *

Lucene indexes may be composed of multiple sub-indexes, or segments. - * Each segment is a fully independent index, which could be searched separately. - * Indexes evolve by:

- *
    - *
  1. Creating new segments for newly added documents.
  2. - *
  3. Merging existing segments.
  4. - *
- *

Searches may involve multiple segments and/or multiple indexes, each index - * potentially composed of a set of segments.

- * - *

Document Numbers

- *

Internally, Lucene refers to documents by an integer document number. - * The first document added to an index is numbered zero, and each subsequent - * document added gets a number one greater than the previous.

- *

Note that a document's number may change, so caution should be taken when - * storing these numbers outside of Lucene. In particular, numbers may change in - * the following situations:

- * - *
- * - *

Index Structure Overview

- *
- *

Each segment index maintains the following:

- * - *

Details on each of these are provided in their linked pages.

- *
- * - *

File Naming

- *
- *

All files belonging to a segment have the same name with varying extensions. - * The extensions correspond to the different file formats described below. When - * using the Compound File format (default for small segments) these files (except - * for the Segment info file, the Lock file, and Deleted documents file) are collapsed - * into a single .cfs file (see below for details)

- *

Typically, all segments in an index are stored in a single directory, - * although this is not required.

- *

File names are never re-used. That is, when any file is saved - * to the Directory it is given a never before used filename. This is achieved - * using a simple generations approach. For example, the first segments file is - * segments_1, then segments_2, etc. The generation is a sequential long integer - * represented in alpha-numeric (base 36) form.

- *
- * - *

Summary of File Extensions

- *
- *

The following table summarizes the names and extensions of the files in - * Lucene:

- * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
{@link org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data}.tvdContains term vector data.
{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}.livInfo about what documents are live
{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.dii, .dimHolds indexed points, if any
- *
- * - *

Lock File

- * The write lock, which is stored in the index directory by default, is named - * "write.lock". If the lock directory is different from the index directory then - * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix - * derived from the full path to the index directory. When this file is present, a - * writer is currently modifying the index (adding or removing documents). This - * lock file ensures that only one writer is modifying the index at a time. - * - *

History

- *

Compatibility notes are provided in this document, describing how file - * formats have changed from prior versions:

- * - * - *

Limitations

- *
- *

Lucene uses a Java int to refer to - * document numbers, and the index file format uses an Int32 - * on-disk to store document numbers. This is a limitation - * of both the index file format and the current implementation. Eventually these - * should be replaced with either UInt64 values, or - * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.

- *
+ * Components from the Lucene 7.0 index format. See {@link org.apache.lucene.codecs.lucene80} + * for an overview of the current index format. */ package org.apache.lucene.codecs.lucene70; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java new file mode 100644 index 00000000000..8ddb93e5a1b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/IndexedDISI.java @@ -0,0 +1,632 @@ +/* + * 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.codecs.lucene80; + +import java.io.DataInput; +import java.io.IOException; + +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.RoaringDocIdSet; + +/** + * Disk-based implementation of a {@link DocIdSetIterator} which can return + * the index of the current document, i.e. the ordinal of the current document + * among the list of documents that this iterator can return. This is useful + * to implement sparse doc values by only having to encode values for documents + * that actually have a value. + *

Implementation-wise, this {@link DocIdSetIterator} is inspired of + * {@link RoaringDocIdSet roaring bitmaps} and encodes ranges of {@code 65536} + * documents independently and picks between 3 encodings depending on the + * density of the range:

+ *

Only ranges that contain at least one value are encoded. + *

This implementation uses 6 bytes per document in the worst-case, which happens + * in the case that all ranges contain exactly one document. + * + * + * To avoid O(n) lookup time complexity, with n being the number of documents, two lookup + * tables are used: A lookup table for block offset and index, and a rank structure + * for DENSE block index lookups. + * + * The lookup table is an array of {@code int}-pairs, with a pair for each block. It allows for + * direct jumping to the block, as opposed to iteration from the current position and forward + * one block at a time. + * + * Each int-pair entry consists of 2 logical parts: + * + * The first 32 bit int holds the index (number of set bits in the blocks) up to just before the + * wanted block. The maximum number of set bits is the maximum number of documents, which is < 2^31. + * + * The next int holds the offset in bytes into the underlying slice. As there is a maximum of 2^16 + * blocks, it follows that the maximum size of any block must not exceed 2^15 bytes to avoid + * overflow (2^16 bytes if the int is treated as unsigned). This is currently the case, with the + * largest block being DENSE and using 2^13 + 36 bytes. + * + * The cache overhead is numDocs/1024 bytes. + * + * Note: There are 4 types of blocks: ALL, DENSE, SPARSE and non-existing (0 set bits). + * In the case of non-existing blocks, the entry in the lookup table has index equal to the + * previous entry and offset equal to the next non-empty block. + * + * The block lookup table is stored at the end of the total block structure. + * + * + * The rank structure for DENSE blocks is an array of byte-pairs with an entry for each + * sub-block (default 512 bits) out of the 65536 bits in the outer DENSE block. + * + * Each rank-entry states the number of set bits within the block up to the bit before the + * bit positioned at the start of the sub-block. + * Note that that the rank entry of the first sub-block is always 0 and that the last entry can + * at most be 65536-2 = 65634 and thus will always fit into an byte-pair of 16 bits. + * + * The rank structure for a given DENSE block is stored at the beginning of the DENSE block. + * This ensures locality and keeps logistics simple. + * + * @lucene.internal + */ +final class IndexedDISI extends DocIdSetIterator { + + // jump-table time/space trade-offs to consider: + // The block offsets and the block indexes could be stored in more compressed form with + // two PackedInts or two MonotonicDirectReaders. + // The DENSE ranks (default 128 shorts = 256 bytes) could likewise be compressed. But as there is + // at least 4096 set bits in DENSE blocks, there will be at least one rank with 2^12 bits, so it + // is doubtful if there is much to gain here. + + private static final int BLOCK_SIZE = 65536; // The number of docIDs that a single block represents + + private static final int DENSE_BLOCK_LONGS = BLOCK_SIZE/Long.SIZE; // 1024 + public static final byte DEFAULT_DENSE_RANK_POWER = 9; // Every 512 docIDs / 8 longs + + static final int MAX_ARRAY_LENGTH = (1 << 12) - 1; + + private static void flush( + int block, FixedBitSet buffer, int cardinality, byte denseRankPower, IndexOutput out) throws IOException { + assert block >= 0 && block < 65536; + out.writeShort((short) block); + assert cardinality > 0 && cardinality <= 65536; + out.writeShort((short) (cardinality - 1)); + if (cardinality > MAX_ARRAY_LENGTH) { + if (cardinality != 65536) { // all docs are set + if (denseRankPower != -1) { + final byte[] rank = createRank(buffer, denseRankPower); + out.writeBytes(rank, rank.length); + } + for (long word : buffer.getBits()) { + out.writeLong(word); + } + } + } else { + BitSetIterator it = new BitSetIterator(buffer, cardinality); + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + out.writeShort((short) doc); + } + } + } + + // Creates a DENSE rank-entry (the number of set bits up to a given point) for the buffer. + // One rank-entry for every {@code 2^denseRankPower} bits, with each rank-entry using 2 bytes. + // Represented as a byte[] for fast flushing and mirroring of the retrieval representation. + private static byte[] createRank(FixedBitSet buffer, byte denseRankPower) { + final int longsPerRank = 1 << (denseRankPower-6); + final int rankMark = longsPerRank-1; + final int rankIndexShift = denseRankPower-7; // 6 for the long (2^6) + 1 for 2 bytes/entry + final byte[] rank = new byte[DENSE_BLOCK_LONGS >> rankIndexShift]; + final long[] bits = buffer.getBits(); + int bitCount = 0; + for (int word = 0 ; word < DENSE_BLOCK_LONGS ; word++) { + if ((word & rankMark) == 0) { // Every longsPerRank longs + rank[word >> rankIndexShift] = (byte)(bitCount>>8); + rank[(word >> rankIndexShift)+1] = (byte)(bitCount & 0xFF); + } + bitCount += Long.bitCount(bits[word]); + } + return rank; + } + + /** + * Writes the docIDs from it to out, in logical blocks, one for each 65536 docIDs in monotonically increasing + * gap-less order. DENSE blocks uses {@link #DEFAULT_DENSE_RANK_POWER} of 9 (every 512 docIDs / 8 longs). + * The caller must keep track of the number of jump-table entries (returned by this method) as well as the + * denseRankPower (9 for this method) and provide them when constructing an IndexedDISI for reading. + * @param it the document IDs. + * @param out destination for the blocks. + * @throws IOException if there was an error writing to out. + * @return the number of jump-table entries following the blocks, -1 for no entries. + * This should be stored in meta and used when creating an instance of IndexedDISI. + */ + static short writeBitSet(DocIdSetIterator it, IndexOutput out) throws IOException { + return writeBitSet(it, out, DEFAULT_DENSE_RANK_POWER); + } + + /** + * Writes the docIDs from it to out, in logical blocks, one for each 65536 docIDs in monotonically + * increasing gap-less order. + * The caller must keep track of the number of jump-table entries (returned by this method) as well as the + * denseRankPower and provide them when constructing an IndexedDISI for reading. + * @param it the document IDs. + * @param out destination for the blocks. + * @param denseRankPower for {@link Method#DENSE} blocks, a rank will be written every {@code 2^denseRankPower} docIDs. + * Values < 7 (every 128 docIDs) or > 15 (every 32768 docIDs) disables DENSE rank. + * Recommended values are 8-12: Every 256-4096 docIDs or 4-64 longs. + * {@link #DEFAULT_DENSE_RANK_POWER} is 9: Every 512 docIDs. + * This should be stored in meta and used when creating an instance of IndexedDISI. + * @throws IOException if there was an error writing to out. + * @return the number of jump-table entries following the blocks, -1 for no entries. + * This should be stored in meta and used when creating an instance of IndexedDISI. + */ + static short writeBitSet(DocIdSetIterator it, IndexOutput out, byte denseRankPower) throws IOException { + final long origo = out.getFilePointer(); // All jumps are relative to the origo + if ((denseRankPower < 7 || denseRankPower > 15) && denseRankPower != -1) { + throw new IllegalArgumentException("Acceptable values for denseRankPower are 7-15 (every 128-32768 docIDs). " + + "The provided power was " + denseRankPower + " (every " + (int)Math.pow(2, denseRankPower) + " docIDs)"); + } + int totalCardinality = 0; + int blockCardinality = 0; + final FixedBitSet buffer = new FixedBitSet(1<<16); + int[] jumps = new int[ArrayUtil.oversize(1, Integer.BYTES*2)]; + int prevBlock = -1; + int jumpBlockIndex = 0; + + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + final int block = doc >>> 16; + if (prevBlock != -1 && block != prevBlock) { + // Track offset+index from previous block up to current + jumps = addJumps(jumps, out.getFilePointer()-origo, totalCardinality, jumpBlockIndex, prevBlock+1); + jumpBlockIndex = prevBlock+1; + // Flush block + flush(prevBlock, buffer, blockCardinality, denseRankPower, out); + // Reset for next block + buffer.clear(0, buffer.length()); + totalCardinality += blockCardinality; + blockCardinality = 0; + } + buffer.set(doc & 0xFFFF); + blockCardinality++; + prevBlock = block; + } + if (blockCardinality > 0) { + jumps = addJumps(jumps, out.getFilePointer()-origo, totalCardinality, jumpBlockIndex, prevBlock+1); + totalCardinality += blockCardinality; + flush(prevBlock, buffer, blockCardinality, denseRankPower, out); + buffer.clear(0, buffer.length()); + prevBlock++; + } + final int lastBlock = prevBlock == -1 ? 0 : prevBlock; // There will always be at least 1 block (NO_MORE_DOCS) + // Last entry is a SPARSE with blockIndex == 32767 and the single entry 65535, which becomes the docID NO_MORE_DOCS + // To avoid creating 65K jump-table entries, only a single entry is created pointing to the offset of the + // NO_MORE_DOCS block, with the jumpBlockIndex set to the logical EMPTY block after all real blocks. + jumps = addJumps(jumps, out.getFilePointer()-origo, totalCardinality, lastBlock, lastBlock+1); + buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF); + flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, denseRankPower, out); + // offset+index jump-table stored at the end + return flushBlockJumps(jumps, lastBlock+1, out, origo); + } + + // Adds entries to the offset & index jump-table for blocks + private static int[] addJumps(int[] jumps, long offset, int index, int startBlock, int endBlock) { + assert offset < Integer.MAX_VALUE : "Logically the offset should not exceed 2^30 but was >= Integer.MAX_VALUE"; + jumps = ArrayUtil.grow(jumps, (endBlock+1)*2); + for (int b = startBlock; b < endBlock; b++) { + jumps[b*2] = index; + jumps[b*2+1] = (int) offset; + } + return jumps; + } + + // Flushes the offet & index jump-table for blocks. This should be the last data written to out + // This method returns the blockCount for the blocks reachable for the jump_table or -1 for no jump-table + private static short flushBlockJumps(int[] jumps, int blockCount, IndexOutput out, long origo) throws IOException { + if (blockCount == 2) { // Jumps with a single real entry + NO_MORE_DOCS is just wasted space so we ignore that + blockCount = 0; + } + for (int i = 0 ; i < blockCount ; i++) { + out.writeInt(jumps[i*2]); // index + out.writeInt(jumps[i*2+1]); // offset + } + // As there are at most 32k blocks, the count is a short + // The jumpTableOffset will be at lastPos - (blockCount * Long.BYTES) + return (short)blockCount; + } + + /** The slice that stores the {@link DocIdSetIterator}. */ + private final IndexInput slice; + private final int jumpTableEntryCount; + private final byte denseRankPower; + private final RandomAccessInput jumpTable; // Skip blocks of 64K bits + private final byte[] denseRankTable; + private final long cost; + + /** + * This constructor always creates a new blockSlice and a new jumpTable from in, to ensure that operations are + * independent from the caller. + * See {@link #IndexedDISI(IndexInput, RandomAccessInput, int, byte, long)} for re-use of blockSlice and jumpTable. + * @param in backing data. + * @param offset starting offset for blocks in the backing data. + * @param length the number of bytes holding blocks and jump-table in the backing data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * This must match the number returned by {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)}. + * @param denseRankPower the number of docIDs covered by each rank entry in DENSE blocks, expressed as {@code 2^denseRankPower}. + * This must match the power given in {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)} + * @param cost normally the number of logical docIDs. + */ + IndexedDISI(IndexInput in, long offset, long length, int jumpTableEntryCount, byte denseRankPower, long cost) throws IOException { + this(createBlockSlice(in,"docs", offset, length, jumpTableEntryCount), + createJumpTable(in, offset, length, jumpTableEntryCount), + jumpTableEntryCount, denseRankPower, cost); + } + + /** + * This constructor allows to pass the slice and jumpTable directly in case it helps reuse. + * see eg. Lucene80 norms producer's merge instance. + * @param blockSlice data blocks, normally created by {@link #createBlockSlice}. + * @param jumpTable table holding jump-data for block-skips, normally created by {@link #createJumpTable}. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * This must match the number returned by {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)}. + * @param denseRankPower the number of docIDs covered by each rank entry in DENSE blocks, expressed as {@code 2^denseRankPower}. + * This must match the power given in {@link #writeBitSet(DocIdSetIterator, IndexOutput, byte)} + * @param cost normally the number of logical docIDs. + */ + IndexedDISI(IndexInput blockSlice, RandomAccessInput jumpTable, int jumpTableEntryCount, byte denseRankPower, long cost) throws IOException { + if ((denseRankPower < 7 || denseRankPower > 15) && denseRankPower != -1) { + throw new IllegalArgumentException("Acceptable values for denseRankPower are 7-15 (every 128-32768 docIDs). " + + "The provided power was " + denseRankPower + " (every " + (int)Math.pow(2, denseRankPower) + " docIDs). "); + } + + this.slice = blockSlice; + this.jumpTable = jumpTable; + this.jumpTableEntryCount = jumpTableEntryCount; + this.denseRankPower = denseRankPower; + final int rankIndexShift = denseRankPower-7; + this.denseRankTable = denseRankPower == -1 ? null : new byte[DENSE_BLOCK_LONGS >> rankIndexShift]; + this.cost = cost; + } + + /** + * Helper method for using {@link #IndexedDISI(IndexInput, RandomAccessInput, int, byte, long)}. + * Creates a disiSlice for the IndexedDISI data blocks, without the jump-table. + * @param slice backing data, holding both blocks and jump-table. + * @param sliceDescription human readable slice designation. + * @param offset relative to the backing data. + * @param length full length of the IndexedDISI, including blocks and jump-table data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * @return a jumpTable containing the block jump-data or null if no such table exists. + * @throws IOException if a RandomAccessInput could not be created from slice. + */ + public static IndexInput createBlockSlice( + IndexInput slice, String sliceDescription, long offset, long length, int jumpTableEntryCount) throws IOException { + long jumpTableBytes = jumpTableEntryCount < 0 ? 0 : jumpTableEntryCount*Integer.BYTES*2; + return slice.slice(sliceDescription, offset, length - jumpTableBytes); + } + + /** + * Helper method for using {@link #IndexedDISI(IndexInput, RandomAccessInput, int, byte, long)}. + * Creates a RandomAccessInput covering only the jump-table data or null. + * @param slice backing data, holding both blocks and jump-table. + * @param offset relative to the backing data. + * @param length full length of the IndexedDISI, including blocks and jump-table data. + * @param jumpTableEntryCount the number of blocks covered by the jump-table. + * @return a jumpTable containing the block jump-data or null if no such table exists. + * @throws IOException if a RandomAccessInput could not be created from slice. + */ + public static RandomAccessInput createJumpTable( + IndexInput slice, long offset, long length, int jumpTableEntryCount) throws IOException { + if (jumpTableEntryCount <= 0) { + return null; + } else { + int jumpTableBytes = jumpTableEntryCount*Integer.BYTES*2; + return slice.randomAccessSlice(offset + length - jumpTableBytes, jumpTableBytes); + } + } + + private int block = -1; + private long blockEnd; + private long denseBitmapOffset = -1; // Only used for DENSE blocks + private int nextBlockIndex = -1; + Method method; + + private int doc = -1; + private int index = -1; + + // SPARSE variables + private boolean exists; + + // DENSE variables + private long word; + private int wordIndex = -1; + // number of one bits encountered so far, including those of `word` + private int numberOfOnes; + // Used with rank for jumps inside of DENSE as they are absolute instead of relative + private int denseOrigoIndex; + + // ALL variables + private int gap; + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + final int targetBlock = target & 0xFFFF0000; + if (block < targetBlock) { + advanceBlock(targetBlock); + } + if (block == targetBlock) { + if (method.advanceWithinBlock(this, target)) { + return doc; + } + readBlockHeader(); + } + boolean found = method.advanceWithinBlock(this, block); + assert found; + return doc; + } + + public boolean advanceExact(int target) throws IOException { + final int targetBlock = target & 0xFFFF0000; + if (block < targetBlock) { + advanceBlock(targetBlock); + } + boolean found = block == targetBlock && method.advanceExactWithinBlock(this, target); + this.doc = target; + return found; + } + + private void advanceBlock(int targetBlock) throws IOException { + final int blockIndex = targetBlock >> 16; + // If the destination block is 2 blocks or more ahead, we use the jump-table. + if (jumpTable != null && blockIndex >= (block >> 16)+2) { + // If the jumpTableEntryCount is exceeded, there are no further bits. Last entry is always NO_MORE_DOCS + final int inRangeBlockIndex = blockIndex < jumpTableEntryCount ? blockIndex : jumpTableEntryCount-1; + final int index = jumpTable.readInt(inRangeBlockIndex*Integer.BYTES*2); + final int offset = jumpTable.readInt(inRangeBlockIndex*Integer.BYTES*2+Integer.BYTES); + this.nextBlockIndex = index-1; // -1 to compensate for the always-added 1 in readBlockHeader + slice.seek(offset); + readBlockHeader(); + return; + } + + // Fallback to iteration of blocks + do { + slice.seek(blockEnd); + readBlockHeader(); + } while (block < targetBlock); + } + + private void readBlockHeader() throws IOException { + block = Short.toUnsignedInt(slice.readShort()) << 16; + assert block >= 0; + final int numValues = 1 + Short.toUnsignedInt(slice.readShort()); + index = nextBlockIndex; + nextBlockIndex = index + numValues; + if (numValues <= MAX_ARRAY_LENGTH) { + method = Method.SPARSE; + blockEnd = slice.getFilePointer() + (numValues << 1); + } else if (numValues == 65536) { + method = Method.ALL; + blockEnd = slice.getFilePointer(); + gap = block - index - 1; + } else { + method = Method.DENSE; + denseBitmapOffset = slice.getFilePointer() + (denseRankTable == null ? 0 : denseRankTable.length); + blockEnd = denseBitmapOffset + (1 << 13); + // Performance consideration: All rank (default 128 * 16 bits) are loaded up front. This should be fast with the + // reusable byte[] buffer, but it is still wasted if the DENSE block is iterated in small steps. + // If this results in too great a performance regression, a heuristic strategy might work where the rank data + // are loaded on first in-block advance, if said advance is > X docIDs. The hope being that a small first + // advance means that subsequent advances will be small too. + // Another alternative is to maintain an extra slice for DENSE rank, but IndexedDISI is already slice-heavy. + if (denseRankPower != -1) { + slice.readBytes(denseRankTable, 0, denseRankTable.length); + } + wordIndex = -1; + numberOfOnes = index + 1; + denseOrigoIndex = numberOfOnes; + } + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + public int index() { + return index; + } + + @Override + public long cost() { + return cost; + } + + enum Method { + SPARSE { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + // TODO: binary search + for (; disi.index < disi.nextBlockIndex;) { + int doc = Short.toUnsignedInt(disi.slice.readShort()); + disi.index++; + if (doc >= targetInBlock) { + disi.doc = disi.block | doc; + disi.exists = true; + return true; + } + } + return false; + } + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + // TODO: binary search + if (target == disi.doc) { + return disi.exists; + } + for (; disi.index < disi.nextBlockIndex;) { + int doc = Short.toUnsignedInt(disi.slice.readShort()); + disi.index++; + if (doc >= targetInBlock) { + if (doc != targetInBlock) { + disi.index--; + disi.slice.seek(disi.slice.getFilePointer() - Short.BYTES); + break; + } + disi.exists = true; + return true; + } + } + disi.exists = false; + return false; + } + }, + DENSE { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + final int targetWordIndex = targetInBlock >>> 6; + + // If possible, skip ahead using the rank cache + rankSkip(disi, target); + + for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) { + disi.word = disi.slice.readLong(); + disi.numberOfOnes += Long.bitCount(disi.word); + } + disi.wordIndex = targetWordIndex; + + long leftBits = disi.word >>> target; + if (leftBits != 0L) { + disi.doc = target + Long.numberOfTrailingZeros(leftBits); + disi.index = disi.numberOfOnes - Long.bitCount(leftBits); + return true; + } + + // There were no set bits at the wanted position. Move forward until one is reached + while (++disi.wordIndex < 1024) { + // This could use the rank cache to skip empty spaces >= 512 bits, but it seems unrealistic + // that such blocks would be DENSE + disi.word = disi.slice.readLong(); + if (disi.word != 0) { + disi.index = disi.numberOfOnes; + disi.numberOfOnes += Long.bitCount(disi.word); + disi.doc = disi.block | (disi.wordIndex << 6) | Long.numberOfTrailingZeros(disi.word); + return true; + } + } + // No set bits in the block at or after the wanted position. + return false; + } + + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException { + final int targetInBlock = target & 0xFFFF; + final int targetWordIndex = targetInBlock >>> 6; + + rankSkip(disi, target); + + for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) { + disi.word = disi.slice.readLong(); + disi.numberOfOnes += Long.bitCount(disi.word); + } + disi.wordIndex = targetWordIndex; + + long leftBits = disi.word >>> target; + disi.index = disi.numberOfOnes - Long.bitCount(leftBits); + return (leftBits & 1L) != 0; + } + + + }, + ALL { + @Override + boolean advanceWithinBlock(IndexedDISI disi, int target) { + disi.doc = target; + disi.index = target - disi.gap; + return true; + } + @Override + boolean advanceExactWithinBlock(IndexedDISI disi, int target) { + disi.index = target - disi.gap; + return true; + } + }; + + /** Advance to the first doc from the block that is equal to or greater than {@code target}. + * Return true if there is such a doc and false otherwise. */ + abstract boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException; + + /** Advance the iterator exactly to the position corresponding to the given {@code target} + * and return whether this document exists. */ + abstract boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException; + } + + /** + * If the distance between the current position and the target is > 8 words, the rank cache will + * be used to guarantee a worst-case of 1 rank-lookup and 7 word-read-and-count-bits operations. + * Note: This does not guarantee a skip up to target, only up to nearest rank boundary. It is the + * responsibility of the caller to iterate further to reach target. + * @param disi standard DISI. + * @param target the wanted docID for which to calculate set-flag and index. + * @throws IOException if a DISI seek failed. + */ + private static void rankSkip(IndexedDISI disi, int target) throws IOException { + if (disi.denseRankPower == -1) { // No rank for the current structure + return; + } + + final int targetInBlock = target & 0xFFFF; // Lower 16 bits + final int targetWordIndex = targetInBlock >>> 6; // long: 2^6 = 64 + + // If the distance between the current position and the target is < rank-longs + // there is no sense in using rank + if (targetWordIndex - disi.wordIndex < (1 << (disi.denseRankPower-6) )) { + return; + } + + // Resolve the rank as close to targetInBlock as possible (maximum distance is 8 longs) + // Note: rankOrigoOffset is tracked on block open, so it is absolute (e.g. don't add origo) + final int rankIndex = targetInBlock >> disi.denseRankPower; // Default is 9 (8 longs: 2^3 * 2^6 = 512 docIDs) + + final int rank = + (disi.denseRankTable[rankIndex<<1] & 0xFF) << 8 | + (disi.denseRankTable[(rankIndex<<1)+1] & 0xFF); + + // Position the counting logic just after the rank point + final int rankAlignedWordIndex = rankIndex << disi.denseRankPower >> 6; + disi.slice.seek(disi.denseBitmapOffset + rankAlignedWordIndex*Long.BYTES); + long rankWord = disi.slice.readLong(); + int denseNOO = rank + Long.bitCount(rankWord); + + disi.wordIndex = rankAlignedWordIndex; + disi.word = rankWord; + disi.numberOfOnes = disi.denseOrigoIndex + denseNOO; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80Codec.java index 1905b701c31..93e91ea02fc 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80Codec.java @@ -37,7 +37,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat; import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat; -import org.apache.lucene.codecs.lucene70.Lucene70NormsFormat; import org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -150,7 +149,7 @@ public class Lucene80Codec extends Codec { /** Returns the docvalues format that should be used for writing * new segments of field. * - * The default implementation always returns "Lucene70". + * The default implementation always returns "Lucene80". *

* WARNING: if you subclass, you are responsible for index * backwards compatibility: future version of Lucene are only @@ -166,9 +165,9 @@ public class Lucene80Codec extends Codec { } private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50"); - private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70"); + private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene80"); - private final NormsFormat normsFormat = new Lucene70NormsFormat(); + private final NormsFormat normsFormat = new Lucene80NormsFormat(); @Override public final NormsFormat normsFormat() { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java new file mode 100644 index 00000000000..38b9fc012d9 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesConsumer.java @@ -0,0 +1,677 @@ +/* + * 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.codecs.lucene80; + + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.EmptyDocValuesProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.SortedSetSelector; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.MathUtil; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.packed.DirectMonotonicWriter; +import org.apache.lucene.util.packed.DirectWriter; + +import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE; + +/** writer for {@link Lucene80DocValuesFormat} */ +final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Closeable { + + IndexOutput data, meta; + final int maxDoc; + + /** expert: Creates a new writer */ + public Lucene80DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader(data, dataCodec, Lucene80DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader(meta, metaCodec, Lucene80DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + maxDoc = state.segmentInfo.maxDoc(); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); // write checksum + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + meta = data = null; + } + } + + @Override + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene80DocValuesFormat.NUMERIC); + + writeValues(field, new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return DocValues.singleton(valuesProducer.getNumeric(field)); + } + }); + } + + private static class MinMaxTracker { + long min, max, numValues, spaceInBits; + + MinMaxTracker() { + reset(); + spaceInBits = 0; + } + + private void reset() { + min = Long.MAX_VALUE; + max = Long.MIN_VALUE; + numValues = 0; + } + + /** Accumulate a new value. */ + void update(long v) { + min = Math.min(min, v); + max = Math.max(max, v); + ++numValues; + } + + /** Update the required space. */ + void finish() { + if (max > min) { + spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues; + } + } + + /** Update space usage and get ready for accumulating values for the next block. */ + void nextBlock() { + finish(); + reset(); + } + } + + private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + int numDocsWithValue = 0; + MinMaxTracker minMax = new MinMaxTracker(); + MinMaxTracker blockMinMax = new MinMaxTracker(); + long gcd = 0; + Set uniqueValues = new HashSet<>(); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + long v = values.nextValue(); + + if (gcd != 1) { + if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) { + // in that case v - minValue might overflow and make the GCD computation return + // wrong results. Since these extreme values are unlikely, we just discard + // GCD computation for them + gcd = 1; + } else if (minMax.numValues != 0) { // minValue needs to be set first + gcd = MathUtil.gcd(gcd, v - minMax.min); + } + } + + minMax.update(v); + blockMinMax.update(v); + if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) { + blockMinMax.nextBlock(); + } + + if (uniqueValues != null + && uniqueValues.add(v) + && uniqueValues.size() > 256) { + uniqueValues = null; + } + } + + numDocsWithValue++; + } + + minMax.finish(); + blockMinMax.finish(); + + final long numValues = minMax.numValues; + long min = minMax.min; + final long max = minMax.max; + assert blockMinMax.spaceInBits <= minMax.spaceInBits; + + if (numDocsWithValue == 0) { // meta[-2, 0]: No documents with values + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithValue == maxDoc) { // meta[-1, 0]: All documents has values + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { // meta[data.offset, data.length]: IndexedDISI structure for documents with values + long offset = data.getFilePointer(); + meta.writeLong(offset);// docsWithFieldOffset + values = valuesProducer.getSortedNumeric(field); + final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeLong(numValues); + final int numBitsPerValue; + boolean doBlocks = false; + Map encode = null; + if (min >= max) { // meta[-1]: All values are 0 + numBitsPerValue = 0; + meta.writeInt(-1); // tablesize + } else { + if (uniqueValues != null + && uniqueValues.size() > 1 + && DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1) < DirectWriter.unsignedBitsRequired((max - min) / gcd)) { + numBitsPerValue = DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1); + final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]); + Arrays.sort(sortedUniqueValues); + meta.writeInt(sortedUniqueValues.length); // tablesize + for (Long v : sortedUniqueValues) { + meta.writeLong(v); // table[] entry + } + encode = new HashMap<>(); + for (int i = 0; i < sortedUniqueValues.length; ++i) { + encode.put(sortedUniqueValues[i], i); + } + min = 0; + gcd = 1; + } else { + uniqueValues = null; + // we do blocks if that appears to save 10+% storage + doBlocks = minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9; + if (doBlocks) { + numBitsPerValue = 0xFF; + meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT); // tablesize + } else { + numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd); + if (gcd == 1 && min > 0 + && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) { + min = 0; + } + meta.writeInt(-1); // tablesize + } + } + } + + meta.writeByte((byte) numBitsPerValue); + meta.writeLong(min); + meta.writeLong(gcd); + long startOffset = data.getFilePointer(); + meta.writeLong(startOffset); // valueOffset + long jumpTableOffset = -1; + if (doBlocks) { + jumpTableOffset = writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd); + } else if (numBitsPerValue != 0) { + writeValuesSingleBlock(valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode); + } + meta.writeLong(data.getFilePointer() - startOffset); // valuesLength + meta.writeLong(jumpTableOffset); + return new long[] {numDocsWithValue, numValues}; + } + + private void writeValuesSingleBlock(SortedNumericDocValues values, long numValues, int numBitsPerValue, + long min, long gcd, Map encode) throws IOException { + DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + long v = values.nextValue(); + if (encode == null) { + writer.add((v - min) / gcd); + } else { + writer.add(encode.get(v)); + } + } + } + writer.finish(); + } + + // Returns the offset to the jump-table for vBPV + private long writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException { + long[] offsets = new long[ArrayUtil.oversize(1, Long.BYTES)]; + int offsetsIndex = 0; + final long[] buffer = new long[NUMERIC_BLOCK_SIZE]; + final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance(); + int upTo = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, count = values.docValueCount(); i < count; ++i) { + buffer[upTo++] = values.nextValue(); + if (upTo == NUMERIC_BLOCK_SIZE) { + offsets = ArrayUtil.grow(offsets, offsetsIndex+1); + offsets[offsetsIndex++] = data.getFilePointer(); + writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer); + upTo = 0; + } + } + } + if (upTo > 0) { + offsets = ArrayUtil.grow(offsets, offsetsIndex+1); + offsets[offsetsIndex++] = data.getFilePointer(); + writeBlock(buffer, upTo, gcd, encodeBuffer); + } + + // All blocks has been written. Flush the offset jump-table + final long offsetsOrigo = data.getFilePointer(); + for (int i = 0 ; i < offsetsIndex ; i++) { + data.writeLong(offsets[i]); + } + data.writeLong(offsetsOrigo); + return offsetsOrigo; + } + + private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer) throws IOException { + assert length > 0; + long min = values[0]; + long max = values[0]; + for (int i = 1; i < length; ++i) { + final long v = values[i]; + assert Math.floorMod(values[i] - min, gcd) == 0; + min = Math.min(min, v); + max = Math.max(max, v); + } + if (min == max) { + data.writeByte((byte) 0); + data.writeLong(min); + } else { + final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min); + buffer.reset(); + assert buffer.size() == 0; + final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue); + for (int i = 0; i < length; ++i) { + w.add((values[i] - min) / gcd); + } + w.finish(); + data.writeByte((byte) bitsPerValue); + data.writeLong(min); + data.writeInt(Math.toIntExact(buffer.size())); + buffer.copyTo(data); + } + } + + @Override + public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene80DocValuesFormat.BINARY); + + BinaryDocValues values = valuesProducer.getBinary(field); + long start = data.getFilePointer(); + meta.writeLong(start); // dataOffset + int numDocsWithField = 0; + int minLength = Integer.MAX_VALUE; + int maxLength = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + BytesRef v = values.binaryValue(); + int length = v.length; + data.writeBytes(v.bytes, v.offset, v.length); + minLength = Math.min(length, minLength); + maxLength = Math.max(length, maxLength); + } + assert numDocsWithField <= maxDoc; + meta.writeLong(data.getFilePointer() - start); // dataLength + + if (numDocsWithField == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getBinary(field); + final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithField); + meta.writeInt(minLength); + meta.writeInt(maxLength); + if (maxLength > minLength) { + start = data.getFilePointer(); + meta.writeLong(start); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + writer.add(addr); + values = valuesProducer.getBinary(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + addr += values.binaryValue().length; + writer.add(addr); + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); + } + } + + @Override + public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene80DocValuesFormat.SORTED); + doAddSortedField(field, valuesProducer); + } + + private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + SortedDocValues values = valuesProducer.getSorted(field); + int numDocsWithField = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + } + + if (numDocsWithField == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getSorted(field); + final short jumpTableentryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableentryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithField); + if (values.getValueCount() <= 1) { + meta.writeByte((byte) 0); // bitsPerValue + meta.writeLong(0L); // ordsOffset + meta.writeLong(0L); // ordsLength + } else { + int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1); + meta.writeByte((byte) numberOfBitsPerOrd); // bitsPerValue + long start = data.getFilePointer(); + meta.writeLong(start); // ordsOffset + DirectWriter writer = DirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd); + values = valuesProducer.getSorted(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + writer.add(values.ordValue()); + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); // ordsLength + } + + addTermsDict(DocValues.singleton(valuesProducer.getSorted(field))); + } + + private void addTermsDict(SortedSetDocValues values) throws IOException { + final long size = values.getValueCount(); + meta.writeVLong(size); + meta.writeInt(Lucene80DocValuesFormat.TERMS_DICT_BLOCK_SHIFT); + + ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput(); + ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp"); + meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + long numBlocks = (size + Lucene80DocValuesFormat.TERMS_DICT_BLOCK_MASK) >>> Lucene80DocValuesFormat.TERMS_DICT_BLOCK_SHIFT; + DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT); + + BytesRefBuilder previous = new BytesRefBuilder(); + long ord = 0; + long start = data.getFilePointer(); + int maxLength = 0; + TermsEnum iterator = values.termsEnum(); + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if ((ord & Lucene80DocValuesFormat.TERMS_DICT_BLOCK_MASK) == 0) { + writer.add(data.getFilePointer() - start); + data.writeVInt(term.length); + data.writeBytes(term.bytes, term.offset, term.length); + } else { + final int prefixLength = StringHelper.bytesDifference(previous.get(), term); + final int suffixLength = term.length - prefixLength; + assert suffixLength > 0; // terms are unique + + data.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4))); + if (prefixLength >= 15) { + data.writeVInt(prefixLength - 15); + } + if (suffixLength >= 16) { + data.writeVInt(suffixLength - 16); + } + data.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength); + } + maxLength = Math.max(maxLength, term.length); + previous.copyBytes(term); + ++ord; + } + writer.finish(); + meta.writeInt(maxLength); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + start = data.getFilePointer(); + addressBuffer.copyTo(data); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + + // Now write the reverse terms index + writeTermsIndex(values); + } + + private void writeTermsIndex(SortedSetDocValues values) throws IOException { + final long size = values.getValueCount(); + meta.writeInt(Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT); + long start = data.getFilePointer(); + + long numBlocks = 1L + ((size + Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) >>> Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT); + ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput(); + DirectMonotonicWriter writer; + try (ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp")) { + writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT); + TermsEnum iterator = values.termsEnum(); + BytesRefBuilder previous = new BytesRefBuilder(); + long offset = 0; + long ord = 0; + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if ((ord & Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) { + writer.add(offset); + final int sortKeyLength; + if (ord == 0) { + // no previous term: no bytes to write + sortKeyLength = 0; + } else { + sortKeyLength = StringHelper.sortKeyLength(previous.get(), term); + } + offset += sortKeyLength; + data.writeBytes(term.bytes, term.offset, sortKeyLength); + } else if ((ord & Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == Lucene80DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) { + previous.copyBytes(term); + } + ++ord; + } + writer.add(offset); + writer.finish(); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + start = data.getFilePointer(); + addressBuffer.copyTo(data); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } + } + + @Override + public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene80DocValuesFormat.SORTED_NUMERIC); + + long[] stats = writeValues(field, valuesProducer); + int numDocsWithField = Math.toIntExact(stats[0]); + long numValues = stats[1]; + assert numValues >= numDocsWithField; + + meta.writeInt(numDocsWithField); + if (numValues > numDocsWithField) { + long start = data.getFilePointer(); + meta.writeLong(start); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter addressesWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + addressesWriter.add(addr); + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + addr += values.docValueCount(); + addressesWriter.add(addr); + } + addressesWriter.finish(); + meta.writeLong(data.getFilePointer() - start); + } + } + + @Override + public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeInt(field.number); + meta.writeByte(Lucene80DocValuesFormat.SORTED_SET); + + SortedSetDocValues values = valuesProducer.getSortedSet(field); + int numDocsWithField = 0; + long numOrds = 0; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithField++; + for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) { + numOrds++; + } + } + + if (numDocsWithField == numOrds) { + meta.writeByte((byte) 0); // multiValued (0 = singleValued) + doAddSortedField(field, new EmptyDocValuesProducer() { + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return SortedSetSelector.wrap(valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN); + } + }); + return; + } + meta.writeByte((byte) 1); // multiValued (1 = multiValued) + + assert numDocsWithField != 0; + if (numDocsWithField == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = valuesProducer.getSortedSet(field); + final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1); + meta.writeByte((byte) numberOfBitsPerOrd); // bitsPerValue + long start = data.getFilePointer(); + meta.writeLong(start); // ordsOffset + DirectWriter writer = DirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd); + values = valuesProducer.getSortedSet(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) { + writer.add(ord); + } + } + writer.finish(); + meta.writeLong(data.getFilePointer() - start); // ordsLength + + meta.writeInt(numDocsWithField); + start = data.getFilePointer(); + meta.writeLong(start); // addressesOffset + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter addressesWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT); + long addr = 0; + addressesWriter.add(addr); + values = valuesProducer.getSortedSet(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + values.nextOrd(); + addr++; + while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) { + addr++; + } + addressesWriter.add(addr); + } + addressesWriter.finish(); + meta.writeLong(data.getFilePointer() - start); // addressesLength + + addTermsDict(values); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java new file mode 100644 index 00000000000..029980f6ba9 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesFormat.java @@ -0,0 +1,175 @@ +/* + * 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.codecs.lucene80; + + +import java.io.IOException; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.SmallFloat; +import org.apache.lucene.util.packed.DirectWriter; + +/** + * Lucene 8.0 DocValues format. + *

+ * Documents that have a value for the field are encoded in a way that it is always possible to + * know the ordinal of the current document in the set of documents that have a value. For instance, + * say the set of documents that have a value for the field is {1, 5, 6, 11}. When the + * iterator is on 6, it knows that this is the 3rd item of the set. This way, values can + * be stored densely and accessed based on their index at search time. If all documents in a segment + * have a value for the field, the index is the same as the doc ID, so this case is encoded implicitly + * and is very fast at query time. On the other hand if some documents are missing a value for the + * field then the set of documents that have a value is encoded into blocks. All doc IDs that share + * the same upper 16 bits are encoded into the same block with the following strategies: + *

+ *

+ * Skipping blocks to arrive at a wanted document is either done on an iterative basis or by using the + * jump-table stored at the end of the chain of blocks. The jump-table holds the offset as well as the + * index for all blocks, packed in a single long per block. + *

+ *

+ * Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are + * encoded using the following strategies: + *

+ * {@link DocValuesType#NUMERIC NUMERIC}: + *

+ *

+ * Depending on calculated gains, the numbers might be split into blocks of 16384 values. In that case, + * a jump-table with block offsets is appended to the blocks for O(1) access to the needed block. + *

+ *

+ * {@link DocValuesType#BINARY BINARY}: + *

+ *

+ * {@link DocValuesType#SORTED SORTED}: + *

+ *

+ * {@link DocValuesType#SORTED_SET SORTED_SET}: + *

+ *

+ * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}: + *

+ *

+ * Files: + *

    + *
  1. .dvd: DocValues data
  2. + *
  3. .dvm: DocValues metadata
  4. + *
+ * @lucene.experimental + */ +public final class Lucene80DocValuesFormat extends DocValuesFormat { + + /** Sole Constructor */ + public Lucene80DocValuesFormat() { + super("Lucene80"); + } + + @Override + public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + return new Lucene80DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION); + } + + @Override + public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException { + return new Lucene80DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION); + } + + static final String DATA_CODEC = "Lucene80DocValuesData"; + static final String DATA_EXTENSION = "dvd"; + static final String META_CODEC = "Lucene80DocValuesMetadata"; + static final String META_EXTENSION = "dvm"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + // indicates docvalues type + static final byte NUMERIC = 0; + static final byte BINARY = 1; + static final byte SORTED = 2; + static final byte SORTED_SET = 3; + static final byte SORTED_NUMERIC = 4; + + static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16; + + static final int NUMERIC_BLOCK_SHIFT = 14; + static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT; + + static final int TERMS_DICT_BLOCK_SHIFT = 4; + static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT; + static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1; + + static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10; + static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT; + static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java new file mode 100644 index 00000000000..e0ece521d5e --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java @@ -0,0 +1,1427 @@ +/* + * 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.codecs.lucene80; + +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.LongValues; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.packed.DirectMonotonicReader; +import org.apache.lucene.util.packed.DirectReader; + +/** reader for {@link Lucene80DocValuesFormat} */ +final class Lucene80DocValuesProducer extends DocValuesProducer implements Closeable { + private final Map numerics = new HashMap<>(); + private final Map binaries = new HashMap<>(); + private final Map sorted = new HashMap<>(); + private final Map sortedSets = new HashMap<>(); + private final Map sortedNumerics = new HashMap<>(); + private long ramBytesUsed; + private final IndexInput data; + private final int maxDoc; + + /** expert: instantiates a new reader */ + Lucene80DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + this.maxDoc = state.segmentInfo.maxDoc(); + ramBytesUsed = RamUsageEstimator.shallowSizeOfInstance(getClass()); + + int version = -1; + + // read in the entries from the metadata file. + try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) { + Throwable priorE = null; + try { + version = CodecUtil.checkIndexHeader(in, metaCodec, + Lucene80DocValuesFormat.VERSION_START, + Lucene80DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + readFields(in, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(in, priorE); + } + } + + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + this.data = state.directory.openInput(dataName, state.context); + boolean success = false; + try { + final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, + Lucene80DocValuesFormat.VERSION_START, + Lucene80DocValuesFormat.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + if (version != version2) { + throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data); + } + + // NOTE: data file is too costly to verify checksum against all the bytes on open, + // but for now we at least verify proper structure of the checksum footer: which looks + // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption + // such as file truncation. + CodecUtil.retrieveChecksum(data); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this.data); + } + } + } + + private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } + byte type = meta.readByte(); + if (type == Lucene80DocValuesFormat.NUMERIC) { + numerics.put(info.name, readNumeric(meta)); + } else if (type == Lucene80DocValuesFormat.BINARY) { + binaries.put(info.name, readBinary(meta)); + } else if (type == Lucene80DocValuesFormat.SORTED) { + sorted.put(info.name, readSorted(meta)); + } else if (type == Lucene80DocValuesFormat.SORTED_SET) { + sortedSets.put(info.name, readSortedSet(meta)); + } else if (type == Lucene80DocValuesFormat.SORTED_NUMERIC) { + sortedNumerics.put(info.name, readSortedNumeric(meta)); + } else { + throw new CorruptIndexException("invalid type: " + type, meta); + } + } + } + + private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException { + NumericEntry entry = new NumericEntry(); + readNumeric(meta, entry); + return entry; + } + + private void readNumeric(ChecksumIndexInput meta, NumericEntry entry) throws IOException { + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numValues = meta.readLong(); + int tableSize = meta.readInt(); + if (tableSize > 256) { + throw new CorruptIndexException("invalid table size: " + tableSize, meta); + } + if (tableSize >= 0) { + entry.table = new long[tableSize]; + ramBytesUsed += RamUsageEstimator.sizeOf(entry.table); + for (int i = 0; i < tableSize; ++i) { + entry.table[i] = meta.readLong(); + } + } + if (tableSize < -1) { + entry.blockShift = -2 - tableSize; + } else { + entry.blockShift = -1; + } + entry.bitsPerValue = meta.readByte(); + entry.minValue = meta.readLong(); + entry.gcd = meta.readLong(); + entry.valuesOffset = meta.readLong(); + entry.valuesLength = meta.readLong(); + entry.valueJumpTableOffset = meta.readLong(); + } + + private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException { + BinaryEntry entry = new BinaryEntry(); + entry.dataOffset = meta.readLong(); + entry.dataLength = meta.readLong(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.minLength = meta.readInt(); + entry.maxLength = meta.readInt(); + if (entry.minLength < entry.maxLength) { + entry.addressesOffset = meta.readLong(); + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1L, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + } + return entry; + } + + private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException { + SortedEntry entry = new SortedEntry(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.bitsPerValue = meta.readByte(); + entry.ordsOffset = meta.readLong(); + entry.ordsLength = meta.readLong(); + readTermDict(meta, entry); + return entry; + } + + private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException { + SortedSetEntry entry = new SortedSetEntry(); + byte multiValued = meta.readByte(); + switch (multiValued) { + case 0: // singlevalued + entry.singleValueEntry = readSorted(meta); + return entry; + case 1: // multivalued + break; + default: + throw new CorruptIndexException("Invalid multiValued flag: " + multiValued, meta); + } + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.bitsPerValue = meta.readByte(); + entry.ordsOffset = meta.readLong(); + entry.ordsLength = meta.readLong(); + entry.numDocsWithField = meta.readInt(); + entry.addressesOffset = meta.readLong(); + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + readTermDict(meta, entry); + return entry; + } + + private static void readTermDict(ChecksumIndexInput meta, TermsDictEntry entry) throws IOException { + entry.termsDictSize = meta.readVLong(); + entry.termsDictBlockShift = meta.readInt(); + final int blockShift = meta.readInt(); + final long addressesSize = (entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift; + entry.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift); + entry.maxTermLength = meta.readInt(); + entry.termsDataOffset = meta.readLong(); + entry.termsDataLength = meta.readLong(); + entry.termsAddressesOffset = meta.readLong(); + entry.termsAddressesLength = meta.readLong(); + entry.termsDictIndexShift = meta.readInt(); + final long indexSize = (entry.termsDictSize + (1L << entry.termsDictIndexShift) - 1) >>> entry.termsDictIndexShift; + entry.termsIndexAddressesMeta = DirectMonotonicReader.loadMeta(meta, 1 + indexSize, blockShift); + entry.termsIndexOffset = meta.readLong(); + entry.termsIndexLength = meta.readLong(); + entry.termsIndexAddressesOffset = meta.readLong(); + entry.termsIndexAddressesLength = meta.readLong(); + } + + private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException { + SortedNumericEntry entry = new SortedNumericEntry(); + readNumeric(meta, entry); + entry.numDocsWithField = meta.readInt(); + if (entry.numDocsWithField != entry.numValues) { + entry.addressesOffset = meta.readLong(); + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift); + ramBytesUsed += entry.addressesMeta.ramBytesUsed(); + entry.addressesLength = meta.readLong(); + } + return entry; + } + + @Override + public void close() throws IOException { + data.close(); + } + + private static class NumericEntry { + long[] table; + int blockShift; + byte bitsPerValue; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + long numValues; + long minValue; + long gcd; + long valuesOffset; + long valuesLength; + long valueJumpTableOffset; // -1 if no jump-table + } + + private static class BinaryEntry { + long dataOffset; + long dataLength; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + int minLength; + int maxLength; + long addressesOffset; + long addressesLength; + DirectMonotonicReader.Meta addressesMeta; + } + + private static class TermsDictEntry { + long termsDictSize; + int termsDictBlockShift; + DirectMonotonicReader.Meta termsAddressesMeta; + int maxTermLength; + long termsDataOffset; + long termsDataLength; + long termsAddressesOffset; + long termsAddressesLength; + int termsDictIndexShift; + DirectMonotonicReader.Meta termsIndexAddressesMeta; + long termsIndexOffset; + long termsIndexLength; + long termsIndexAddressesOffset; + long termsIndexAddressesLength; + } + + private static class SortedEntry extends TermsDictEntry { + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + byte bitsPerValue; + long ordsOffset; + long ordsLength; + } + + private static class SortedSetEntry extends TermsDictEntry { + SortedEntry singleValueEntry; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + byte denseRankPower; + int numDocsWithField; + byte bitsPerValue; + long ordsOffset; + long ordsLength; + DirectMonotonicReader.Meta addressesMeta; + long addressesOffset; + long addressesLength; + } + + private static class SortedNumericEntry extends NumericEntry { + int numDocsWithField; + DirectMonotonicReader.Meta addressesMeta; + long addressesOffset; + long addressesLength; + } + + @Override + public long ramBytesUsed() { + return ramBytesUsed; + } + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + NumericEntry entry = numerics.get(field.name); + return getNumeric(entry); + } + + private static abstract class DenseNumericDocValues extends NumericDocValues { + + final int maxDoc; + int doc = -1; + + DenseNumericDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) { + doc = target; + return true; + } + + @Override + public long cost() { + return maxDoc; + } + + } + + private static abstract class SparseNumericDocValues extends NumericDocValues { + + final IndexedDISI disi; + + SparseNumericDocValues(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + } + + private NumericDocValues getNumeric(NumericEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -2) { + // empty + return DocValues.emptyNumeric(); + } else if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.bitsPerValue == 0) { + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + // dense but split into blocks of different bits per value + return new DenseNumericDocValues(maxDoc) { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + + @Override + public long longValue() throws IOException { + return vBPVReader.getLongValue(doc); + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return table[(int) values.get(doc)]; + } + }; + } else { + final long mul = entry.gcd; + final long delta = entry.minValue; + return new DenseNumericDocValues(maxDoc) { + @Override + public long longValue() throws IOException { + return mul * values.get(doc) + delta; + } + }; + } + } + } + } else { + // sparse + final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, + entry.jumpTableEntryCount, entry.denseRankPower, entry.numValues); + if (entry.bitsPerValue == 0) { + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + // sparse and split into blocks of different bits per value + return new SparseNumericDocValues(disi) { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + + @Override + public long longValue() throws IOException { + final int index = disi.index(); + return vBPVReader.getLongValue(index); + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return table[(int) values.get(disi.index())]; + } + }; + } else { + final long mul = entry.gcd; + final long delta = entry.minValue; + return new SparseNumericDocValues(disi) { + @Override + public long longValue() throws IOException { + return mul * values.get(disi.index()) + delta; + } + }; + } + } + } + } + } + + private LongValues getNumericValues(NumericEntry entry) throws IOException { + if (entry.bitsPerValue == 0) { + return new LongValues() { + @Override + public long get(long index) { + return entry.minValue; + } + }; + } else { + final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength); + if (entry.blockShift >= 0) { + return new LongValues() { + final VaryingBPVReader vBPVReader = new VaryingBPVReader(entry, slice); + @Override + public long get(long index) { + try { + return vBPVReader.getLongValue(index); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + } else { + final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue); + if (entry.table != null) { + final long[] table = entry.table; + return new LongValues() { + @Override + public long get(long index) { + return table[(int) values.get(index)]; + } + }; + } else if (entry.gcd != 1) { + final long gcd = entry.gcd; + final long minValue = entry.minValue; + return new LongValues() { + @Override + public long get(long index) { + return values.get(index) * gcd + minValue; + } + }; + } else if (entry.minValue != 0) { + final long minValue = entry.minValue; + return new LongValues() { + @Override + public long get(long index) { + return values.get(index) + minValue; + } + }; + } else { + return values; + } + } + } + } + + private static abstract class DenseBinaryDocValues extends BinaryDocValues { + + final int maxDoc; + int doc = -1; + + DenseBinaryDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + doc = target; + return true; + } + } + + private static abstract class SparseBinaryDocValues extends BinaryDocValues { + + final IndexedDISI disi; + + SparseBinaryDocValues(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + BinaryEntry entry = binaries.get(field.name); + if (entry.docsWithFieldOffset == -2) { + return DocValues.emptyBinary(); + } + + final IndexInput bytesSlice = data.slice("fixed-binary", entry.dataOffset, entry.dataLength); + + if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.minLength == entry.maxLength) { + // fixed length + final int length = entry.maxLength; + return new DenseBinaryDocValues(maxDoc) { + final BytesRef bytes = new BytesRef(new byte[length], 0, length); + + @Override + public BytesRef binaryValue() throws IOException { + bytesSlice.seek((long) doc * length); + bytesSlice.readBytes(bytes.bytes, 0, length); + return bytes; + } + }; + } else { + // variable length + final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new DenseBinaryDocValues(maxDoc) { + final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength); + + @Override + public BytesRef binaryValue() throws IOException { + long startOffset = addresses.get(doc); + bytes.length = (int) (addresses.get(doc + 1L) - startOffset); + bytesSlice.seek(startOffset); + bytesSlice.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + }; + } + } else { + // sparse + final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, + entry.jumpTableEntryCount, entry.denseRankPower, entry.numDocsWithField); + if (entry.minLength == entry.maxLength) { + // fixed length + final int length = entry.maxLength; + return new SparseBinaryDocValues(disi) { + final BytesRef bytes = new BytesRef(new byte[length], 0, length); + + @Override + public BytesRef binaryValue() throws IOException { + bytesSlice.seek((long) disi.index() * length); + bytesSlice.readBytes(bytes.bytes, 0, length); + return bytes; + } + }; + } else { + // variable length + final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new SparseBinaryDocValues(disi) { + final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength); + + @Override + public BytesRef binaryValue() throws IOException { + final int index = disi.index(); + long startOffset = addresses.get(index); + bytes.length = (int) (addresses.get(index + 1L) - startOffset); + bytesSlice.seek(startOffset); + bytesSlice.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + }; + } + } + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + SortedEntry entry = sorted.get(field.name); + return getSorted(entry); + } + + private SortedDocValues getSorted(SortedEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -2) { + return DocValues.emptySorted(); + } + + final LongValues ords; + if (entry.bitsPerValue == 0) { + ords = new LongValues() { + @Override + public long get(long index) { + return 0L; + } + }; + } else { + final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength); + ords = DirectReader.getInstance(slice, entry.bitsPerValue); + } + + if (entry.docsWithFieldOffset == -1) { + // dense + return new BaseSortedDocValues(entry, data) { + + int doc = -1; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) { + doc = target; + return true; + } + + @Override + public int ordValue() { + return (int) ords.get(doc); + } + }; + } else { + // sparse + final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, + entry.jumpTableEntryCount, entry.denseRankPower, entry.numDocsWithField); + return new BaseSortedDocValues(entry, data) { + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public int ordValue() { + return (int) ords.get(disi.index()); + } + }; + } + } + + private static abstract class BaseSortedDocValues extends SortedDocValues { + + final SortedEntry entry; + final IndexInput data; + final TermsEnum termsEnum; + + BaseSortedDocValues(SortedEntry entry, IndexInput data) throws IOException { + this.entry = entry; + this.data = data; + this.termsEnum = termsEnum(); + } + + @Override + public int getValueCount() { + return Math.toIntExact(entry.termsDictSize); + } + + @Override + public BytesRef lookupOrd(int ord) throws IOException { + termsEnum.seekExact(ord); + return termsEnum.term(); + } + + @Override + public int lookupTerm(BytesRef key) throws IOException { + SeekStatus status = termsEnum.seekCeil(key); + switch (status) { + case FOUND: + return Math.toIntExact(termsEnum.ord()); + default: + return Math.toIntExact(-1L - termsEnum.ord()); + } + } + + @Override + public TermsEnum termsEnum() throws IOException { + return new TermsDict(entry, data); + } + } + + private static abstract class BaseSortedSetDocValues extends SortedSetDocValues { + + final SortedSetEntry entry; + final IndexInput data; + final TermsEnum termsEnum; + + BaseSortedSetDocValues(SortedSetEntry entry, IndexInput data) throws IOException { + this.entry = entry; + this.data = data; + this.termsEnum = termsEnum(); + } + + @Override + public long getValueCount() { + return entry.termsDictSize; + } + + @Override + public BytesRef lookupOrd(long ord) throws IOException { + termsEnum.seekExact(ord); + return termsEnum.term(); + } + + @Override + public long lookupTerm(BytesRef key) throws IOException { + SeekStatus status = termsEnum.seekCeil(key); + switch (status) { + case FOUND: + return termsEnum.ord(); + default: + return -1L - termsEnum.ord(); + } + } + + @Override + public TermsEnum termsEnum() throws IOException { + return new TermsDict(entry, data); + } + } + + private static class TermsDict extends TermsEnum { + + final TermsDictEntry entry; + final LongValues blockAddresses; + final IndexInput bytes; + final long blockMask; + final LongValues indexAddresses; + final IndexInput indexBytes; + final BytesRef term; + long ord = -1; + + TermsDict(TermsDictEntry entry, IndexInput data) throws IOException { + this.entry = entry; + RandomAccessInput addressesSlice = data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength); + blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice); + bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength); + blockMask = (1L << entry.termsDictBlockShift) - 1; + RandomAccessInput indexAddressesSlice = data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength); + indexAddresses = DirectMonotonicReader.getInstance(entry.termsIndexAddressesMeta, indexAddressesSlice); + indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength); + term = new BytesRef(entry.maxTermLength); + } + + @Override + public BytesRef next() throws IOException { + if (++ord >= entry.termsDictSize) { + return null; + } + if ((ord & blockMask) == 0L) { + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + } else { + final int token = Byte.toUnsignedInt(bytes.readByte()); + int prefixLength = token & 0x0F; + int suffixLength = 1 + (token >>> 4); + if (prefixLength == 15) { + prefixLength += bytes.readVInt(); + } + if (suffixLength == 16) { + suffixLength += bytes.readVInt(); + } + term.length = prefixLength + suffixLength; + bytes.readBytes(term.bytes, prefixLength, suffixLength); + } + return term; + } + + @Override + public void seekExact(long ord) throws IOException { + if (ord < 0 || ord >= entry.termsDictSize) { + throw new IndexOutOfBoundsException(); + } + final long blockIndex = ord >>> entry.termsDictBlockShift; + final long blockAddress = blockAddresses.get(blockIndex); + bytes.seek(blockAddress); + this.ord = (blockIndex << entry.termsDictBlockShift) - 1; + do { + next(); + } while (this.ord < ord); + } + + private BytesRef getTermFromIndex(long index) throws IOException { + assert index >= 0 && index <= (entry.termsDictSize - 1) >>> entry.termsDictIndexShift; + final long start = indexAddresses.get(index); + term.length = (int) (indexAddresses.get(index + 1) - start); + indexBytes.seek(start); + indexBytes.readBytes(term.bytes, 0, term.length); + return term; + } + + private long seekTermsIndex(BytesRef text) throws IOException { + long lo = 0L; + long hi = (entry.termsDictSize - 1) >>> entry.termsDictIndexShift; + while (lo <= hi) { + final long mid = (lo + hi) >>> 1; + getTermFromIndex(mid); + final int cmp = term.compareTo(text); + if (cmp <= 0) { + lo = mid + 1; + } else { + hi = mid - 1; + } + } + + assert hi < 0 || getTermFromIndex(hi).compareTo(text) <= 0; + assert hi == ((entry.termsDictSize - 1) >>> entry.termsDictIndexShift) || getTermFromIndex(hi + 1).compareTo(text) > 0; + + return hi; + } + + private BytesRef getFirstTermFromBlock(long block) throws IOException { + assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift; + final long blockAddress = blockAddresses.get(block); + bytes.seek(blockAddress); + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + return term; + } + + private long seekBlock(BytesRef text) throws IOException { + long index = seekTermsIndex(text); + if (index == -1L) { + return -1L; + } + + long ordLo = index << entry.termsDictIndexShift; + long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L; + + long blockLo = ordLo >>> entry.termsDictBlockShift; + long blockHi = ordHi >>> entry.termsDictBlockShift; + + while (blockLo <= blockHi) { + final long blockMid = (blockLo + blockHi) >>> 1; + getFirstTermFromBlock(blockMid); + final int cmp = term.compareTo(text); + if (cmp <= 0) { + blockLo = blockMid + 1; + } else { + blockHi = blockMid - 1; + } + } + + assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0; + assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift) || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0; + + return blockHi; + } + + @Override + public SeekStatus seekCeil(BytesRef text) throws IOException { + final long block = seekBlock(text); + if (block == -1) { + // before the first term + seekExact(0L); + return SeekStatus.NOT_FOUND; + } + final long blockAddress = blockAddresses.get(block); + this.ord = block << entry.termsDictBlockShift; + bytes.seek(blockAddress); + term.length = bytes.readVInt(); + bytes.readBytes(term.bytes, 0, term.length); + while (true) { + int cmp = term.compareTo(text); + if (cmp == 0) { + return SeekStatus.FOUND; + } else if (cmp > 0) { + return SeekStatus.NOT_FOUND; + } + if (next() == null) { + return SeekStatus.END; + } + } + } + + @Override + public BytesRef term() throws IOException { + return term; + } + + @Override + public long ord() throws IOException { + return ord; + } + + @Override + public long totalTermFreq() throws IOException { + return -1L; + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public ImpactsEnum impacts(int flags) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() throws IOException { + throw new UnsupportedOperationException(); + } + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + SortedNumericEntry entry = sortedNumerics.get(field.name); + if (entry.numValues == entry.numDocsWithField) { + return DocValues.singleton(getNumeric(entry)); + } + + final RandomAccessInput addressesInput = data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput); + + final LongValues values = getNumericValues(entry); + + if (entry.docsWithFieldOffset == -1) { + // dense + return new SortedNumericDocValues() { + + int doc = -1; + long start, end; + int count; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + start = addresses.get(target); + end = addresses.get(target + 1L); + count = (int) (end - start); + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + start = addresses.get(target); + end = addresses.get(target + 1L); + count = (int) (end - start); + doc = target; + return true; + } + + @Override + public long nextValue() throws IOException { + return values.get(start++); + } + + @Override + public int docValueCount() { + return count; + } + }; + } else { + // sparse + final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, + entry.jumpTableEntryCount, entry.denseRankPower, entry.numDocsWithField); + return new SortedNumericDocValues() { + + boolean set; + long start, end; + int count; + + @Override + public int nextDoc() throws IOException { + set = false; + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + set = false; + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + set = false; + return disi.advanceExact(target); + } + + @Override + public long nextValue() throws IOException { + set(); + return values.get(start++); + } + + @Override + public int docValueCount() { + set(); + return count; + } + + private void set() { + if (set == false) { + final int index = disi.index(); + start = addresses.get(index); + end = addresses.get(index + 1L); + count = (int) (end - start); + set = true; + } + } + + }; + } + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + SortedSetEntry entry = sortedSets.get(field.name); + if (entry.singleValueEntry != null) { + return DocValues.singleton(getSorted(entry.singleValueEntry)); + } + + final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength); + final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue); + + final RandomAccessInput addressesInput = data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput); + + if (entry.docsWithFieldOffset == -1) { + // dense + return new BaseSortedSetDocValues(entry, data) { + + int doc = -1; + long start; + long end; + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int docID() { + return doc; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + start = addresses.get(target); + end = addresses.get(target + 1L); + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + start = addresses.get(target); + end = addresses.get(target + 1L); + doc = target; + return true; + } + + @Override + public long nextOrd() throws IOException { + if (start == end) { + return NO_MORE_ORDS; + } + return ords.get(start++); + } + + }; + } else { + // sparse + final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, + entry.jumpTableEntryCount, entry.denseRankPower, entry.numDocsWithField); + return new BaseSortedSetDocValues(entry, data) { + + boolean set; + long start; + long end = 0; + + @Override + public int nextDoc() throws IOException { + set = false; + return disi.nextDoc(); + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public long cost() { + return disi.cost(); + } + + @Override + public int advance(int target) throws IOException { + set = false; + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + set = false; + return disi.advanceExact(target); + } + + @Override + public long nextOrd() throws IOException { + if (set == false) { + final int index = disi.index(); + final long start = addresses.get(index); + this.start = start + 1; + end = addresses.get(index + 1L); + set = true; + return ords.get(start); + } else if (start == end) { + return NO_MORE_ORDS; + } else { + return ords.get(start++); + } + } + + }; + } + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(data); + } + + /** + * Reader for longs split into blocks of different bits per values. + * The longs are requested by index and must be accessed in monotonically increasing order. + */ + // Note: The order requirement could be removed as the jump-tables allow for backwards iteration + // Note 2: The rankSlice is only used if an advance of > 1 block is called. Its construction could be lazy + private class VaryingBPVReader { + final RandomAccessInput slice; // 2 slices to avoid cache thrashing when using rank + final RandomAccessInput rankSlice; + final NumericEntry entry; + final int shift; + final long mul; + final int mask; + + long block = -1; + long delta; + long offset; + long blockEndOffset; + LongValues values; + + VaryingBPVReader(NumericEntry entry, RandomAccessInput slice) throws IOException { + this.entry = entry; + this.slice = slice; + this.rankSlice = entry.valueJumpTableOffset == -1 ? null : + data.randomAccessSlice(entry.valueJumpTableOffset, data.length()-entry.valueJumpTableOffset); + shift = entry.blockShift; + mul = entry.gcd; + mask = (1 << shift) - 1; + } + + long getLongValue(long index) throws IOException { + final long block = index >>> shift; + if (this.block != block) { + int bitsPerValue; + do { + // If the needed block is the one directly following the current block, it is cheaper to avoid the cache + if (rankSlice != null && block != this.block+1) { + blockEndOffset = rankSlice.readLong(block*Long.BYTES)-entry.valuesOffset; + this.block = block-1; + } + offset = blockEndOffset; + bitsPerValue = slice.readByte(offset++); + delta = slice.readLong(offset); + offset += Long.BYTES; + if (bitsPerValue == 0) { + blockEndOffset = offset; + } else { + final int length = slice.readInt(offset); + offset += Integer.BYTES; + blockEndOffset = offset + length; + } + this.block++; + } while (this.block != block); + values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset); + } + return mul * values.get(index & mask) + delta; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java new file mode 100644 index 00000000000..ae2b445617c --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsConsumer.java @@ -0,0 +1,165 @@ +/* + * 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.codecs.lucene80; + +import java.io.IOException; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsConsumer; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; + +import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; + +/** + * Writer for {@link Lucene80NormsFormat} + */ +final class Lucene80NormsConsumer extends NormsConsumer { + IndexOutput data, meta; + final int maxDoc; + + Lucene80NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + maxDoc = state.segmentInfo.maxDoc(); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); // write checksum + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + meta = data = null; + } + } + + @Override + public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException { + NumericDocValues values = normsProducer.getNorms(field); + int numDocsWithValue = 0; + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + numDocsWithValue++; + long v = values.longValue(); + min = Math.min(min, v); + max = Math.max(max, v); + } + assert numDocsWithValue <= maxDoc; + + meta.writeInt(field.number); + + if (numDocsWithValue == 0) { + meta.writeLong(-2); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else if (numDocsWithValue == maxDoc) { + meta.writeLong(-1); // docsWithFieldOffset + meta.writeLong(0L); // docsWithFieldLength + meta.writeShort((short) -1); // jumpTableEntryCount + meta.writeByte((byte) -1); // denseRankPower + } else { + long offset = data.getFilePointer(); + meta.writeLong(offset); // docsWithFieldOffset + values = normsProducer.getNorms(field); + final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); + meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength + meta.writeShort(jumpTableEntryCount); + meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER); + } + + meta.writeInt(numDocsWithValue); + int numBytesPerValue = numBytesPerValue(min, max); + + meta.writeByte((byte) numBytesPerValue); + if (numBytesPerValue == 0) { + meta.writeLong(min); + } else { + meta.writeLong(data.getFilePointer()); // normsOffset + values = normsProducer.getNorms(field); + writeValues(values, numBytesPerValue, data); + } + } + + private int numBytesPerValue(long min, long max) { + if (min >= max) { + return 0; + } else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) { + return 1; + } else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) { + return 2; + } else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) { + return 4; + } else { + return 8; + } + } + + private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out) throws IOException, AssertionError { + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + long value = values.longValue(); + switch (numBytesPerValue) { + case 1: + out.writeByte((byte) value); + break; + case 2: + out.writeShort((short) value); + break; + case 4: + out.writeInt((int) value); + break; + case 8: + out.writeLong(value); + break; + default: + throw new AssertionError(); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java new file mode 100644 index 00000000000..915116b529e --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsFormat.java @@ -0,0 +1,99 @@ +/* + * 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.codecs.lucene80; + +import java.io.IOException; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsConsumer; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; + +/** + * Lucene 8.0 Score normalization format. + *

+ * Encodes normalization values by encoding each value with the minimum + * number of bytes needed to represent the range (which can be zero). + *

+ * Files: + *

    + *
  1. .nvd: Norms data
  2. + *
  3. .nvm: Norms metadata
  4. + *
+ *
    + *
  1. + *

    The Norms metadata or .nvm file.

    + *

    For each norms field, this stores metadata, such as the offset into the + * Norms data (.nvd)

    + *

    Norms metadata (.dvm) --> Header,<Entry>NumFields,Footer

    + *
      + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
    • + *
    • Entry --> FieldNumber, DocsWithFieldAddress, DocsWithFieldLength, NumDocsWithField, BytesPerNorm, NormsAddress
    • + *
    • FieldNumber --> {@link DataOutput#writeInt Int32}
    • + *
    • DocsWithFieldAddress --> {@link DataOutput#writeLong Int64}
    • + *
    • DocsWithFieldLength --> {@link DataOutput#writeLong Int64}
    • + *
    • NumDocsWithField --> {@link DataOutput#writeInt Int32}
    • + *
    • BytesPerNorm --> {@link DataOutput#writeByte byte}
    • + *
    • NormsAddress --> {@link DataOutput#writeLong Int64}
    • + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • + *
    + *

    FieldNumber of -1 indicates the end of metadata.

    + *

    NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the singleton value + * when BytesPerValue = 0. If BytesPerValue is different from 0 then there are NumDocsWithField values + * to read at that offset.

    + *

    DocsWithFieldAddress is the pointer to the start of the bit set containing documents that have a norm + * in the norms data (.nvd), or -2 if no documents have a norm value, or -1 if all documents have a norm + * value.

    + *

    DocsWithFieldLength is the number of bytes used to encode the set of documents that have a norm.

    + *
  2. + *

    The Norms data or .nvd file.

    + *

    For each Norms field, this stores the actual per-document data (the heavy-lifting)

    + *

    Norms data (.nvd) --> Header,< Data >NumFields,Footer

    + *
      + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
    • + *
    • DocsWithFieldData --> {@link IndexedDISI#writeBitSet Bit set of MaxDoc bits}
    • + *
    • NormsData --> {@link DataOutput#writeByte(byte) byte}NumDocsWithField * BytesPerValue
    • + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • + *
    + *
+ * @lucene.experimental + */ +public class Lucene80NormsFormat extends NormsFormat { + + /** Sole Constructor */ + public Lucene80NormsFormat() {} + + @Override + public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException { + return new Lucene80NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + @Override + public NormsProducer normsProducer(SegmentReadState state) throws IOException { + return new Lucene80NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION); + } + + private static final String DATA_CODEC = "Lucene80NormsData"; + private static final String DATA_EXTENSION = "nvd"; + private static final String METADATA_CODEC = "Lucene80NormsMetadata"; + private static final String METADATA_EXTENSION = "nvm"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java new file mode 100644 index 00000000000..66126a23405 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80NormsProducer.java @@ -0,0 +1,386 @@ +/* + * 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.codecs.lucene80; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.IOUtils; + +import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene80.Lucene80NormsFormat.VERSION_START; + +/** + * Reader for {@link Lucene80NormsFormat} + */ +final class Lucene80NormsProducer extends NormsProducer implements Cloneable { + // metadata maps (just file pointers and minimal stuff) + private final Map norms = new HashMap<>(); + private final int maxDoc; + private IndexInput data; + private boolean merging; + private Map disiInputs; + private Map disiJumpTables; + private Map dataInputs; + + Lucene80NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.maxDoc(); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + int version = -1; + + // read in the entries from the metadata file. + try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) { + Throwable priorE = null; + try { + version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + readFields(in, state.fieldInfos); + } catch (Throwable exception) { + priorE = exception; + } finally { + CodecUtil.checkFooter(in, priorE); + } + } + + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.openInput(dataName, state.context); + boolean success = false; + try { + final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + if (version != version2) { + throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data); + } + + // NOTE: data file is too costly to verify checksum against all the bytes on open, + // but for now we at least verify proper structure of the checksum footer: which looks + // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption + // such as file truncation. + CodecUtil.retrieveChecksum(data); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this.data); + } + } + } + + @Override + public NormsProducer getMergeInstance() throws IOException { + Lucene80NormsProducer clone; + try { + clone = (Lucene80NormsProducer) super.clone(); + } catch (CloneNotSupportedException e) { + // cannot happen + throw new RuntimeException(e); + } + clone.data = data.clone(); + clone.disiInputs = new HashMap<>(); + clone.disiJumpTables = new HashMap<>(); + clone.dataInputs = new HashMap<>(); + clone.merging = true; + return clone; + } + + static class NormsEntry { + byte denseRankPower; + byte bytesPerNorm; + long docsWithFieldOffset; + long docsWithFieldLength; + short jumpTableEntryCount; + int numDocsWithField; + long normsOffset; + } + + static abstract class DenseNormsIterator extends NumericDocValues { + + final int maxDoc; + int doc = -1; + + DenseNormsIterator(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target >= maxDoc) { + return doc = NO_MORE_DOCS; + } + return doc = target; + } + + @Override + public boolean advanceExact(int target) throws IOException { + this.doc = target; + return true; + } + + @Override + public long cost() { + return maxDoc; + } + + } + + static abstract class SparseNormsIterator extends NumericDocValues { + + final IndexedDISI disi; + + SparseNormsIterator(IndexedDISI disi) { + this.disi = disi; + } + + @Override + public int docID() { + return disi.docID(); + } + + @Override + public int nextDoc() throws IOException { + return disi.nextDoc(); + } + + @Override + public int advance(int target) throws IOException { + return disi.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return disi.advanceExact(target); + } + + @Override + public long cost() { + return disi.cost(); + } + } + + private void readFields(IndexInput meta, FieldInfos infos) throws IOException { + for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { + FieldInfo info = infos.fieldInfo(fieldNumber); + if (info == null) { + throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); + } else if (!info.hasNorms()) { + throw new CorruptIndexException("Invalid field: " + info.name, meta); + } + NormsEntry entry = new NormsEntry(); + entry.docsWithFieldOffset = meta.readLong(); + entry.docsWithFieldLength = meta.readLong(); + entry.jumpTableEntryCount = meta.readShort(); + entry.denseRankPower = meta.readByte(); + entry.numDocsWithField = meta.readInt(); + entry.bytesPerNorm = meta.readByte(); + switch (entry.bytesPerNorm) { + case 0: case 1: case 2: case 4: case 8: + break; + default: + throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta); + } + entry.normsOffset = meta.readLong(); + norms.put(info.number, entry); + } + } + + private RandomAccessInput getDataInput(FieldInfo field, NormsEntry entry) throws IOException { + RandomAccessInput slice = null; + if (merging) { + slice = dataInputs.get(field.number); + } + if (slice == null) { + slice = data.randomAccessSlice(entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm); + if (merging) { + dataInputs.put(field.number, slice); + } + } + return slice; + } + + private IndexInput getDisiInput(FieldInfo field, NormsEntry entry) throws IOException { + IndexInput slice = null; + if (merging) { + slice = disiInputs.get(field.number); + } + if (slice == null) { + slice = IndexedDISI.createBlockSlice( + data, "docs", entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.jumpTableEntryCount); + if (merging) { + disiInputs.put(field.number, slice); + } + } + return slice; + } + + private RandomAccessInput getDisiJumpTable(FieldInfo field, NormsEntry entry) throws IOException { + RandomAccessInput jumpTable = null; + if (merging) { + jumpTable = disiJumpTables.get(field.number); + } + if (jumpTable == null) { + jumpTable = IndexedDISI.createJumpTable( + data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.jumpTableEntryCount); + if (merging) { + disiJumpTables.put(field.number, jumpTable); + } + } + return jumpTable; + } + + + @Override + public NumericDocValues getNorms(FieldInfo field) throws IOException { + final NormsEntry entry = norms.get(field.number); + if (entry.docsWithFieldOffset == -2) { + // empty + return DocValues.emptyNumeric(); + } else if (entry.docsWithFieldOffset == -1) { + // dense + if (entry.bytesPerNorm == 0) { + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return entry.normsOffset; + } + }; + } + final RandomAccessInput slice = getDataInput(field, entry); + switch (entry.bytesPerNorm) { + case 1: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readByte(doc); + } + }; + case 2: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readShort(((long) doc) << 1); + } + }; + case 4: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readInt(((long) doc) << 2); + } + }; + case 8: + return new DenseNormsIterator(maxDoc) { + @Override + public long longValue() throws IOException { + return slice.readLong(((long) doc) << 3); + } + }; + default: + // should not happen, we already validate bytesPerNorm in readFields + throw new AssertionError(); + } + } else { + // sparse + final IndexInput disiInput = getDisiInput(field, entry); + final RandomAccessInput disiJumpTable = getDisiJumpTable(field, entry); + final IndexedDISI disi = new IndexedDISI(disiInput, disiJumpTable, entry.jumpTableEntryCount, entry.denseRankPower, entry.numDocsWithField); + + if (entry.bytesPerNorm == 0) { + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return entry.normsOffset; + } + }; + } + final RandomAccessInput slice = data.randomAccessSlice(entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm); + switch (entry.bytesPerNorm) { + case 1: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readByte(disi.index()); + } + }; + case 2: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readShort(((long) disi.index()) << 1); + } + }; + case 4: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readInt(((long) disi.index()) << 2); + } + }; + case 8: + return new SparseNormsIterator(disi) { + @Override + public long longValue() throws IOException { + return slice.readLong(((long) disi.index()) << 3); + } + }; + default: + // should not happen, we already validate bytesPerNorm in readFields + throw new AssertionError(); + } + } + } + + @Override + public void close() throws IOException { + data.close(); + } + + @Override + public long ramBytesUsed() { + return 64L * norms.size(); // good enough + } + + @Override + public void checkIntegrity() throws IOException { + CodecUtil.checksumEntireFile(data); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "(fields=" + norms.size() + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/package-info.java index 2b8a7e10737..82b5689638c 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/package-info.java @@ -163,7 +163,7 @@ * all documents omit position data. * *
  • - * {@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Normalization factors}. + * {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}. * For each field in each document, a value is stored * that is multiplied into the score for hits on that field. *
  • @@ -175,7 +175,7 @@ * {@link org.apache.lucene.document.Field Field} constructors * *
  • - * {@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-document values}. + * {@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-document values}. * Like stored values, these are also keyed by document * number, but are generally intended to be loaded into main memory for fast * access. Whereas stored values are generally intended for summary results from @@ -284,12 +284,12 @@ * Stores additional per-position metadata information such as character offsets and user payloads * * - * {@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Norms} + * {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Norms} * .nvd, .nvm * Encodes length and boost factors for docs and fields * * - * {@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-Document Values} + * {@link org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat Per-Document Values} * .dvd, .dvm * Encodes additional scoring factors or other per-document information. * @@ -393,7 +393,9 @@ * doc ids, the (term freq, normalization factor) pairs that may trigger the * maximum score of the block. This information is recorded alongside skip data * in order to be able to skip blocks of doc ids if they may not produce high - * enough scores.
  • + * enough scores. + * Additionally doc values and norms has been extended with jump-tables to make access O(1) + * instead of O(n), where n is the number of elements to skip when advancing in the data. * * *

    Limitations

    diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat index 20463c5dc45..43ed529ec01 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat +org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java new file mode 100644 index 00000000000..6f102b38502 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestIndexedDISI.java @@ -0,0 +1,522 @@ +/* + * 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.codecs.lucene80; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; + +// Copied from the lucene70 package for separation of codec-code +public class TestIndexedDISI extends LuceneTestCase { + + public void testEmpty() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + // EMPTY blocks are special with regard to jumps as they have size 0 + public void testEmptyBlocks() throws IOException { + final int B = 65536; + int maxDoc = B*11; + FixedBitSet set = new FixedBitSet(maxDoc); + // block 0: EMPTY + set.set(B+5); // block 1: SPARSE + // block 2: EMPTY + // block 3: EMPTY + set.set(B*4+5); // block 4: SPARSE + + for (int i = 0 ; i < B ; i++) { + set.set(B*6+i); // block 6: ALL + } + for (int i = 0 ; i < B ; i+=3) { + set.set(B*7+i); // block 7: DENSE + } + for (int i = 0 ; i < B ; i++) { + if (i != 32768) { + set.set(B*8 + i); // block 8: DENSE (all-1) + } + } + // block 9-11: EMPTY + + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + + // Change the first block to DENSE to see if jump-tables sets to position 0 + set.set(0); + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + } + + // EMPTY blocks are special with regard to jumps as they have size 0 + public void testLastEmptyBlocks() throws IOException { + final int B = 65536; + int maxDoc = B*3; + FixedBitSet set = new FixedBitSet(maxDoc); + for (int docID = 0 ; docID < B*2 ; docID++) { // first 2 blocks are ALL + set.set(docID); + } + // Last block is EMPTY + + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + assertAdvanceBeyondEnd(set, dir); + } + } + + // Checks that advance after the end of the blocks has been reached has the correct behaviour + private void assertAdvanceBeyondEnd(FixedBitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = 9; // Not tested here so fixed to isolate factors + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("bar", IOContext.DEFAULT)) { + jumpTableentryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + } + + try (IndexInput in = dir.openInput("bar", IOContext.DEFAULT)) { + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + int doc = disi2.docID(); + int index = 0; + while (doc < cardinality) { + doc = disi2.nextDoc(); + index++; + } + + IndexedDISI disi = new IndexedDISI(in, 0L, in.length(), jumpTableentryCount, denseRankPower, cardinality); + // Advance 1 docID beyond end + assertFalse("There should be no set bit beyond the valid docID range", disi.advanceExact(set.length())); + disi.advance(doc); // Should be the special docID signifyin NO_MORE_DOCS from the BitSetIterator + assertEquals("The index when advancing beyond the last defined docID should be correct", + index, disi.index()+1); // disi.index()+1 as the while-loop also counts the NO_MORE_DOCS + } + } + + public void testRandomBlocks() throws IOException { + final int BLOCKS = 5; + FixedBitSet set = createSetWithRandomBlocks(BLOCKS); + try (Directory dir = newDirectory()) { + doTestAllSingleJump(set, dir); + } + } + + // When doing merges in Lucene80NormsProducer, IndexedDISI are created from slices where the offset is not 0 + public void testPositionNotZero() throws IOException { + final int BLOCKS = 10; + final byte denseRankPower = rarely() ? -1 : (byte) (random().nextInt(7)+7); // sane + chance of disable + + FixedBitSet set = createSetWithRandomBlocks(BLOCKS); + try (Directory dir = newDirectory()) { + final int cardinality = set.cardinality(); + int jumpTableEntryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableEntryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + } + try (IndexInput fullInput = dir.openInput("foo", IOContext.DEFAULT)) { + IndexInput blockData = + IndexedDISI.createBlockSlice(fullInput, "blocks", 0, fullInput.length(), jumpTableEntryCount); + blockData.seek(random().nextInt((int) blockData.length())); + + RandomAccessInput jumpTable = IndexedDISI.createJumpTable(fullInput, 0, fullInput.length(), jumpTableEntryCount); + IndexedDISI disi = new IndexedDISI(blockData, jumpTable, jumpTableEntryCount, denseRankPower, cardinality); + // This failed at some point during LUCENE-8585 development as it did not reset the slice position + disi.advanceExact(BLOCKS*65536-1); + } + } + } + + private FixedBitSet createSetWithRandomBlocks(int blockCount) { + final int B = 65536; + FixedBitSet set = new FixedBitSet(blockCount * B); + for (int block = 0; block < blockCount; block++) { + switch (random().nextInt(4)) { + case 0: { // EMPTY + break; + } + case 1: { // ALL + for (int docID = block* B; docID < (block+1)* B; docID++) { + set.set(docID); + } + break; + } + case 2: { // SPARSE ( < 4096 ) + for (int docID = block* B; docID < (block+1)* B; docID += 101) { + set.set(docID); + } + break; + } + case 3: { // DENSE ( >= 4096 ) + for (int docID = block* B; docID < (block+1)* B; docID += 3) { + set.set(docID); + } + break; + } + default: throw new IllegalStateException("Modulo logic error: there should only be 4 possibilities"); + } + } + return set; + } + + + private void doTestAllSingleJump(FixedBitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = rarely() ? -1 : (byte) (random().nextInt(7)+7); // sane + chance of disable + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + for (int i = 0; i < set.length(); i++) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + assertEquals("The bit at " + i + " should be correct with advanceExact", set.get(i), disi.advanceExact(i)); + + IndexedDISI disi2 = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + disi2.advance(i); + // Proper sanity check with jump tables as an error could make them seek backwards + assertTrue("The docID should at least be " + i + " after advance(" + i + ") but was " + disi2.docID(), + i <= disi2.docID()); + if (set.get(i)) { + assertEquals("The docID should be present with advance", i, disi2.docID()); + } else { + assertNotSame("The docID should not be present with advance", i, disi2.docID()); + } + } + } + } + + public void testOneDoc() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testTwoDocs() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(random().nextInt(maxDoc)); + set.set(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testAllDocs() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(1, maxDoc); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testHalfFull() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + for (int i = random().nextInt(2); i < maxDoc; i += TestUtil.nextInt(random(), 1, 3)) { + set.set(i); + } + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testDocRange() throws IOException { + try (Directory dir = newDirectory()) { + for (int iter = 0; iter < 10; ++iter) { + int maxDoc = TestUtil.nextInt(random(), 1, 1000000); + FixedBitSet set = new FixedBitSet(maxDoc); + final int start = random().nextInt(maxDoc); + final int end = TestUtil.nextInt(random(), start + 1, maxDoc); + set.set(start, end); + doTest(set, dir); + } + } + } + + public void testSparseDenseBoundary() throws IOException { + try (Directory dir = newDirectory()) { + FixedBitSet set = new FixedBitSet(200000); + int start = 65536 + random().nextInt(100); + final byte denseRankPower = rarely() ? -1 : (byte) (random().nextInt(7)+7); // sane + chance of disable + + // we set MAX_ARRAY_LENGTH bits so the encoding will be sparse + set.set(start, start + IndexedDISI.MAX_ARRAY_LENGTH); + long length; + int jumpTableEntryCount; + try (IndexOutput out = dir.createOutput("sparse", IOContext.DEFAULT)) { + jumpTableEntryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH), out, denseRankPower); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("sparse", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableEntryCount, denseRankPower, IndexedDISI.MAX_ARRAY_LENGTH); + assertEquals(start, disi.nextDoc()); + assertEquals(IndexedDISI.Method.SPARSE, disi.method); + } + doTest(set, dir); + + // now we set one more bit so the encoding will be dense + set.set(start + IndexedDISI.MAX_ARRAY_LENGTH + random().nextInt(100)); + try (IndexOutput out = dir.createOutput("bar", IOContext.DEFAULT)) { + IndexedDISI.writeBitSet(new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH + 1), out, denseRankPower); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("bar", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableEntryCount, denseRankPower, IndexedDISI.MAX_ARRAY_LENGTH + 1); + assertEquals(start, disi.nextDoc()); + assertEquals(IndexedDISI.Method.DENSE, disi.method); + } + doTest(set, dir); + } + } + + public void testOneDocMissing() throws IOException { + int maxDoc = TestUtil.nextInt(random(), 1, 1000000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + set.clear(random().nextInt(maxDoc)); + try (Directory dir = newDirectory()) { + doTest(set, dir); + } + } + + public void testFewMissingDocs() throws IOException { + try (Directory dir = newDirectory()) { + for (int iter = 0; iter < 100; ++iter) { + int maxDoc = TestUtil.nextInt(random(), 1, 100000); + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + final int numMissingDocs = TestUtil.nextInt(random(), 2, 1000); + for (int i = 0; i < numMissingDocs; ++i) { + set.clear(random().nextInt(maxDoc)); + } + doTest(set, dir); + } + } + } + public void testDenseMultiBlock() throws IOException { + try (Directory dir = newDirectory()) { + int maxDoc = 10 * 65536; // 10 blocks + FixedBitSet set = new FixedBitSet(maxDoc); + for (int i = 0; i < maxDoc; i += 2) { // Set every other to ensure dense + set.set(i); + } + doTest(set, dir); + } + } + + public void testIllegalDenseRankPower() throws IOException { + + // Legal values + for (byte denseRankPower: new byte[]{-1, 7, 8, 9, 10, 11, 12, 13, 14, 15}) { + createAndOpenDISI(denseRankPower, denseRankPower); + } + + // Illegal values + for (byte denseRankPower: new byte[]{-2, 0, 1, 6, 16}) { + try { + createAndOpenDISI(denseRankPower, (byte) 8); // Illegal write, legal read (should not reach read) + fail("Trying to create an IndexedDISI data stream with denseRankPower-read " + denseRankPower + + " and denseRankPower-write 8 should fail"); + } catch (IllegalArgumentException e) { + // Expected + } + try { + createAndOpenDISI((byte) 8, denseRankPower); // Legal write, illegal read (should reach read) + fail("Trying to create an IndexedDISI data stream with denseRankPower-write 8 and denseRankPower-read " + + denseRankPower + " should fail"); + } catch (IllegalArgumentException e) { + // Expected + } + } + } + + private void createAndOpenDISI(byte denseRankPowerWrite, byte denseRankPowerRead) throws IOException { + FixedBitSet set = new FixedBitSet(10); + set.set(set.length()-1); + try (Directory dir = newDirectory()) { + long length; + int jumpTableEntryCount = -1; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableEntryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, set.cardinality()), out, denseRankPowerWrite); + length = out.getFilePointer(); + } + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableEntryCount, denseRankPowerRead, set.cardinality()); + } + // This tests the legality of the denseRankPower only, so we don't do anything with the disi + } + } + + public void testOneDocMissingFixed() throws IOException { + int maxDoc = 9699; + final byte denseRankPower = rarely() ? -1 : (byte) (random().nextInt(7)+7); // sane + chance of disable + FixedBitSet set = new FixedBitSet(maxDoc); + set.set(0, maxDoc); + set.clear(1345); + try (Directory dir = newDirectory()) { + + final int cardinality = set.cardinality(); + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + int step = 16000; + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertAdvanceEquality(disi, disi2, step); + } + } + } + + public void testRandom() throws IOException { + try (Directory dir = newDirectory()) { + for (int i = 0; i < 10; ++i) { + doTestRandom(dir); + } + } + } + + private void doTestRandom(Directory dir) throws IOException { + List docs = new ArrayList<>(); + final int maxStep = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 2, 20)); + final int numDocs = TestUtil.nextInt(random(), 1, Math.min(100000, Integer.MAX_VALUE / maxStep)); + for (int doc = -1, i = 0; i < numDocs; ++i) { + doc += TestUtil.nextInt(random(), 1, maxStep); + docs.add(doc); + } + final int maxDoc = docs.get(docs.size() - 1) + TestUtil.nextInt(random(), 1, 100); + + FixedBitSet set = new FixedBitSet(maxDoc); + for (int doc : docs) { + set.set(doc); + } + + doTest(set, dir); + } + + private void doTest(FixedBitSet set, Directory dir) throws IOException { + final int cardinality = set.cardinality(); + final byte denseRankPower = rarely() ? -1 : (byte) (random().nextInt(7)+7); // sane + chance of disable + long length; + int jumpTableentryCount; + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + jumpTableentryCount = IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out, denseRankPower); + length = out.getFilePointer(); + } + + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertSingleStepEquality(disi, disi2); + } + + for (int step : new int[] {1, 10, 100, 1000, 10000, 100000}) { + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + assertAdvanceEquality(disi, disi2, step); + } + } + + for (int step : new int[] {10, 100, 1000, 10000, 100000}) { + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + IndexedDISI disi = new IndexedDISI(in, 0L, length, jumpTableentryCount, denseRankPower, cardinality); + BitSetIterator disi2 = new BitSetIterator(set, cardinality); + int disi2length = set.length(); + assertAdvanceExactRandomized(disi, disi2, disi2length, step); + } + } + + dir.deleteFile("foo"); + } + + private void assertAdvanceExactRandomized(IndexedDISI disi, BitSetIterator disi2, int disi2length, int step) + throws IOException { + int index = -1; + for (int target = 0; target < disi2length; ) { + target += TestUtil.nextInt(random(), 0, step); + int doc = disi2.docID(); + while (doc < target) { + doc = disi2.nextDoc(); + index++; + } + + boolean exists = disi.advanceExact(target); + assertEquals(doc == target, exists); + if (exists) { + assertEquals(index, disi.index()); + } else if (random().nextBoolean()) { + assertEquals(doc, disi.nextDoc()); + // This is a bit strange when doc == NO_MORE_DOCS as the index overcounts in the disi2 while-loop + assertEquals(index, disi.index()); + target = doc; + } + } + } + + private void assertSingleStepEquality(IndexedDISI disi, BitSetIterator disi2) throws IOException { + int i = 0; + for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) { + assertEquals(doc, disi.nextDoc()); + assertEquals(i++, disi.index()); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc()); + } + + private void assertAdvanceEquality(IndexedDISI disi, BitSetIterator disi2, int step) throws IOException { + int index = -1; + while (true) { + int target = disi2.docID() + step; + int doc; + do { + doc = disi2.nextDoc(); + index++; + } while (doc < target); + assertEquals(doc, disi.advance(target)); + if (doc == DocIdSetIterator.NO_MORE_DOCS) { + break; + } + assertEquals("Expected equality using step " + step + " at docID " + doc, index, disi.index()); + } + } + +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80DocValuesFormat.java new file mode 100644 index 00000000000..943f3cf9af3 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80DocValuesFormat.java @@ -0,0 +1,750 @@ +/* + * 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.codecs.lucene80; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.function.LongSupplier; +import java.util.function.Supplier; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.asserting.AssertingCodec; +import org.apache.lucene.document.BinaryDocValuesField; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.SerialMergeScheduler; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMFile; +import org.apache.lucene.store.RAMInputStream; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.TestUtil; + +/** + * Tests Lucene80DocValuesFormat + * Copied directly from the lucene70 package for separation of codec-code + */ +public class TestLucene80DocValuesFormat extends BaseCompressingDocValuesFormatTestCase { + private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene80DocValuesFormat()); + + @Override + protected Codec getCodec() { + return codec; + } + + // TODO: these big methods can easily blow up some of the other ram-hungry codecs... + // for now just keep them here, as we want to test this for this format. + + @Slow + public void testSortedSetVariableLengthBigVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100); + } + } + + @Nightly + public void testSortedSetVariableLengthManyVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100); + } + } + + @Slow + public void testSortedVariableLengthBigVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedVsStoredFields(atLeast(300), 1d, 1, 32766); + } + } + + @Nightly + public void testSortedVariableLengthManyVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1d, 1, 500); + } + } + + @Slow + public void testTermsEnumFixedWidth() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), () -> TestUtil.randomSimpleString(random(), 10, 10)); + } + } + + @Slow + public void testTermsEnumVariableWidth() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), () -> TestUtil.randomSimpleString(random(), 1, 500)); + } + } + + @Nightly + public void testTermsEnumRandomMany() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), () -> TestUtil.randomSimpleString(random(), 1, 500)); + } + } + + public void testTermsEnumLongSharedPrefixes() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), () -> { + char[] chars = new char[random().nextInt(500)]; + Arrays.fill(chars, 'a'); + if (chars.length > 0) { + chars[random().nextInt(chars.length)] = 'b'; + } + return new String(chars); + }); + } + } + + @Slow + public void testSparseDocValuesVsStoredFields() throws Exception { + int numIterations = atLeast(1); + for (int i = 0; i < numIterations; i++) { + doTestSparseDocValuesVsStoredFields(); + } + } + + private void doTestSparseDocValuesVsStoredFields() throws Exception { + final long[] values = new long[TestUtil.nextInt(random(), 1, 500)]; + for (int i = 0; i < values.length; ++i) { + values[i] = random().nextLong(); + } + + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMergeScheduler(new SerialMergeScheduler()); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); + + // sparse compression is only enabled if less than 1% of docs have a value + final int avgGap = 100; + + final int numDocs = atLeast(200); + for (int i = random().nextInt(avgGap * 2); i >= 0; --i) { + writer.addDocument(new Document()); + } + final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5); + for (int i = 0; i < numDocs; ++i) { + Document doc = new Document(); + + // single-valued + long docValue = values[random().nextInt(values.length)]; + doc.add(new NumericDocValuesField("numeric", docValue)); + doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue)))); + doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue)))); + doc.add(new StoredField("value", docValue)); + + // multi-valued + final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc); + for (int j = 0; j < numValues; ++j) { + docValue = values[random().nextInt(values.length)]; + doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue)); + doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue)))); + doc.add(new StoredField("values", docValue)); + } + + writer.addDocument(doc); + + // add a gap + for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) { + writer.addDocument(new Document()); + } + } + + if (random().nextBoolean()) { + writer.forceMerge(1); + } + + final IndexReader indexReader = writer.getReader(); + writer.close(); + + for (LeafReaderContext context : indexReader.leaves()) { + final LeafReader reader = context.reader(); + final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric"); + + final SortedDocValues sorted = DocValues.getSorted(reader, "sorted"); + + final BinaryDocValues binary = DocValues.getBinary(reader, "binary"); + + final SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(reader, "sorted_numeric"); + + final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set"); + + for (int i = 0; i < reader.maxDoc(); ++i) { + final Document doc = reader.document(i); + final IndexableField valueField = doc.getField("value"); + final Long value = valueField == null ? null : valueField.numericValue().longValue(); + + if (value == null) { + assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i); + } else { + assertEquals(i, numeric.nextDoc()); + assertEquals(i, binary.nextDoc()); + assertEquals(i, sorted.nextDoc()); + assertEquals(value.longValue(), numeric.longValue()); + assertTrue(sorted.ordValue() >= 0); + assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue())); + assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue()); + } + + final IndexableField[] valuesFields = doc.getFields("values"); + if (valuesFields.length == 0) { + assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i); + } else { + final Set valueSet = new HashSet<>(); + for (IndexableField sf : valuesFields) { + valueSet.add(sf.numericValue().longValue()); + } + + assertEquals(i, sortedNumeric.nextDoc()); + assertEquals(valuesFields.length, sortedNumeric.docValueCount()); + for (int j = 0; j < sortedNumeric.docValueCount(); ++j) { + assertTrue(valueSet.contains(sortedNumeric.nextValue())); + } + assertEquals(i, sortedSet.nextDoc()); + int sortedSetCount = 0; + while (true) { + long ord = sortedSet.nextOrd(); + if (ord == SortedSetDocValues.NO_MORE_ORDS) { + break; + } + assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString()))); + sortedSetCount++; + } + assertEquals(valueSet.size(), sortedSetCount); + } + } + } + + indexReader.close(); + dir.close(); + } + + // TODO: try to refactor this and some termsenum tests into the base class. + // to do this we need to fix the test class to get a DVF not a Codec so we can setup + // the postings format correctly. + private void doTestTermsEnumRandom(int numDocs, Supplier valuesProducer) throws Exception { + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMergeScheduler(new SerialMergeScheduler()); + // set to duel against a codec which has ordinals: + final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random()); + final DocValuesFormat dv = new Lucene80DocValuesFormat(); + conf.setCodec(new AssertingCodec() { + @Override + public PostingsFormat getPostingsFormatForField(String field) { + return pf; + } + + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return dv; + } + }); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); + + // index some docs + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + Field idField = new StringField("id", Integer.toString(i), Field.Store.NO); + doc.add(idField); + int numValues = random().nextInt(17); + // create a random list of strings + List values = new ArrayList<>(); + for (int v = 0; v < numValues; v++) { + values.add(valuesProducer.get()); + } + + // add in any order to the indexed field + ArrayList unordered = new ArrayList<>(values); + Collections.shuffle(unordered, random()); + for (String v : values) { + doc.add(newStringField("indexed", v, Field.Store.NO)); + } + + // add in any order to the dv field + ArrayList unordered2 = new ArrayList<>(values); + Collections.shuffle(unordered2, random()); + for (String v : unordered2) { + doc.add(new SortedSetDocValuesField("dv", new BytesRef(v))); + } + + writer.addDocument(doc); + if (random().nextInt(31) == 0) { + writer.commit(); + } + } + + // delete some docs + int numDeletions = random().nextInt(numDocs/10); + for (int i = 0; i < numDeletions; i++) { + int id = random().nextInt(numDocs); + writer.deleteDocuments(new Term("id", Integer.toString(id))); + } + + // compare per-segment + DirectoryReader ir = writer.getReader(); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + Terms terms = r.terms("indexed"); + if (terms != null) { + SortedSetDocValues ssdv = r.getSortedSetDocValues("dv"); + assertEquals(terms.size(), ssdv.getValueCount()); + TermsEnum expected = terms.iterator(); + TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum(); + assertEquals(terms.size(), expected, actual); + + doTestSortedSetEnumAdvanceIndependently(ssdv); + } + } + ir.close(); + + writer.forceMerge(1); + + // now compare again after the merge + ir = writer.getReader(); + LeafReader ar = getOnlyLeafReader(ir); + Terms terms = ar.terms("indexed"); + if (terms != null) { + assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount()); + TermsEnum expected = terms.iterator(); + TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum(); + assertEquals(terms.size(), expected, actual); + } + ir.close(); + + writer.close(); + dir.close(); + } + + private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception { + BytesRef ref; + + // sequential next() through all terms + while ((ref = expected.next()) != null) { + assertEquals(ref, actual.next()); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + assertNull(actual.next()); + + // sequential seekExact(ord) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + actual.seekExact(i); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // sequential seekExact(BytesRef) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + assertTrue(actual.seekExact(expected.term())); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // sequential seekCeil(BytesRef) through all terms + for (long i = 0; i < numOrds; i++) { + expected.seekExact(i); + assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term())); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekExact(ord) + for (long i = 0; i < numOrds; i++) { + long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1); + expected.seekExact(randomOrd); + actual.seekExact(randomOrd); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekExact(BytesRef) + for (long i = 0; i < numOrds; i++) { + long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1); + expected.seekExact(randomOrd); + actual.seekExact(expected.term()); + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + + // random seekCeil(BytesRef) + for (long i = 0; i < numOrds; i++) { + BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random())); + SeekStatus expectedStatus = expected.seekCeil(target); + assertEquals(expectedStatus, actual.seekCeil(target)); + if (expectedStatus != SeekStatus.END) { + assertEquals(expected.ord(), actual.ord()); + assertEquals(expected.term(), actual.term()); + } + } + } + + @Slow + public void testSortedSetAroundBlockSize() throws IOException { + final int frontier = 1 << Lucene80DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; + for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) { + final Directory dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy())); + RAMFile buffer = new RAMFile(); + RAMOutputStream out = new RAMOutputStream(buffer, false); + Document doc = new Document(); + SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef()); + doc.add(field1); + SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef()); + doc.add(field2); + for (int i = 0; i < maxDoc; ++i) { + BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2)); + BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2)); + field1.setBytesValue(s1); + field2.setBytesValue(s2); + w.addDocument(doc); + Set set = new TreeSet<>(Arrays.asList(s1, s2)); + out.writeVInt(set.size()); + for (BytesRef ref : set) { + out.writeVInt(ref.length); + out.writeBytes(ref.bytes, ref.offset, ref.length); + } + } + out.close(); + w.forceMerge(1); + DirectoryReader r = DirectoryReader.open(w); + w.close(); + LeafReader sr = getOnlyLeafReader(r); + assertEquals(maxDoc, sr.maxDoc()); + SortedSetDocValues values = sr.getSortedSetDocValues("sset"); + assertNotNull(values); + try (RAMInputStream in = new RAMInputStream("", buffer)) { + BytesRefBuilder b = new BytesRefBuilder(); + for (int i = 0; i < maxDoc; ++i) { + assertEquals(i, values.nextDoc()); + final int numValues = in.readVInt(); + + for (int j = 0; j < numValues; ++j) { + b.setLength(in.readVInt()); + b.grow(b.length()); + in.readBytes(b.bytes(), 0, b.length()); + assertEquals(b.get(), values.lookupOrd(values.nextOrd())); + } + + assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd()); + } + } + r.close(); + dir.close(); + } + } + + @Slow + public void testSortedNumericAroundBlockSize() throws IOException { + final int frontier = 1 << Lucene80DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; + for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) { + final Directory dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy())); + RAMFile buffer = new RAMFile(); + RAMOutputStream out = new RAMOutputStream(buffer, false); + Document doc = new Document(); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L); + doc.add(field1); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L); + doc.add(field2); + for (int i = 0; i < maxDoc; ++i) { + long s1 = random().nextInt(100); + long s2 = random().nextInt(100); + field1.setLongValue(s1); + field2.setLongValue(s2); + w.addDocument(doc); + out.writeVLong(Math.min(s1, s2)); + out.writeVLong(Math.max(s1, s2)); + } + out.close(); + w.forceMerge(1); + DirectoryReader r = DirectoryReader.open(w); + w.close(); + LeafReader sr = getOnlyLeafReader(r); + assertEquals(maxDoc, sr.maxDoc()); + SortedNumericDocValues values = sr.getSortedNumericDocValues("snum"); + assertNotNull(values); + try (RAMInputStream in = new RAMInputStream("", buffer)) { + for (int i = 0; i < maxDoc; ++i) { + assertEquals(i, values.nextDoc()); + assertEquals(2, values.docValueCount()); + assertEquals(in.readVLong(), values.nextValue()); + assertEquals(in.readVLong(), values.nextValue()); + } + } + r.close(); + dir.close(); + } + } + + @Slow + public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3)); + } + + @Slow + public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2)); + } + + @Slow + public void testNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSparseNumericBlocksOfVariousBitsPerValue(1); + } + + @Slow + public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception { + doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble()); + } + + // The LUCENE-8585 jump-tables enables O(1) skipping of IndexedDISI blocks, DENSE block lookup + // and numeric multi blocks. This test focuses on testing these jumps. + @Slow + public void testNumericFieldJumpTables() throws Exception { + // IndexedDISI block skipping only activated if target >= current+2, so we need at least 5 blocks to + // trigger consecutive block skips + final int maxDoc = atLeast(5*65536); + + Directory dir = newDirectory(); + IndexWriter iw = createFastIndexWriter(dir, maxDoc); + + Field idField = newStringField("id", "", Field.Store.NO); + Field storedField = newStringField("stored", "", Field.Store.YES); + Field dvField = new NumericDocValuesField("dv", 0); + + for (int i = 0 ; i < maxDoc ; i++) { + Document doc = new Document(); + idField.setStringValue(Integer.toBinaryString(i)); + doc.add(idField); + if (random().nextInt(100) > 10) { // Skip 10% to make DENSE blocks + int value = random().nextInt(100000); + storedField.setStringValue(Integer.toString(value)); + doc.add(storedField); + dvField.setLongValue(value); + doc.add(dvField); + } + iw.addDocument(doc); + } + iw.flush(); + iw.forceMerge(1, true); // Single segment to force large enough structures + iw.commit(); + iw.close(); + + assertDVIterate(dir); + assertDVAdvance(dir, rarely() ? 1 : 7); // 1 is heavy (~20 s), so we do it rarely. 7 is a lot faster (8 s) + + dir.close(); + } + + private IndexWriter createFastIndexWriter(Directory dir, int maxBufferedDocs) throws IOException { + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(maxBufferedDocs); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + return new IndexWriter(dir, conf); + } + + private static LongSupplier blocksOfVariousBPV() { + final long mul = TestUtil.nextInt(random(), 1, 100); + final long min = random().nextInt(); + return new LongSupplier() { + int i = Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE; + int maxDelta; + @Override + public long getAsLong() { + if (i == Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE) { + maxDelta = 1 << random().nextInt(5); + i = 0; + } + i++; + return min + mul * random().nextInt(maxDelta); + } + }; + } + + private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts) throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(atLeast(Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE)); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + IndexWriter writer = new IndexWriter(dir, conf); + + final int numDocs = atLeast(Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE*3); + final LongSupplier values = blocksOfVariousBPV(); + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + + int valueCount = (int) counts.getAsLong(); + long valueArray[] = new long[valueCount]; + for (int j = 0; j < valueCount; j++) { + long value = values.getAsLong(); + valueArray[j] = value; + doc.add(new SortedNumericDocValuesField("dv", value)); + } + Arrays.sort(valueArray); + for (int j = 0; j < valueCount; j++) { + doc.add(new StoredField("stored", Long.toString(valueArray[j]))); + } + writer.addDocument(doc); + if (random().nextInt(31) == 0) { + writer.commit(); + } + } + writer.forceMerge(1); + + writer.close(); + + // compare + DirectoryReader ir = DirectoryReader.open(dir); + TestUtil.checkReader(ir); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv"); + for (int i = 0; i < r.maxDoc(); i++) { + if (i > docValues.docID()) { + docValues.nextDoc(); + } + String expected[] = r.document(i).getValues("stored"); + if (i < docValues.docID()) { + assertEquals(0, expected.length); + } else { + String actual[] = new String[docValues.docValueCount()]; + for (int j = 0; j < actual.length; j++) { + actual[j] = Long.toString(docValues.nextValue()); + } + assertArrayEquals(expected, actual); + } + } + } + ir.close(); + dir.close(); + } + + private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); + conf.setMaxBufferedDocs(atLeast(Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE)); + conf.setRAMBufferSizeMB(-1); + conf.setMergePolicy(newLogMergePolicy(random().nextBoolean())); + IndexWriter writer = new IndexWriter(dir, conf); + Document doc = new Document(); + Field storedField = newStringField("stored", "", Field.Store.YES); + Field dvField = new NumericDocValuesField("dv", 0); + doc.add(storedField); + doc.add(dvField); + + final int numDocs = atLeast(Lucene80DocValuesFormat.NUMERIC_BLOCK_SIZE*3); + final LongSupplier longs = blocksOfVariousBPV(); + for (int i = 0; i < numDocs; i++) { + if (random().nextDouble() > density) { + writer.addDocument(new Document()); + continue; + } + long value = longs.getAsLong(); + storedField.setStringValue(Long.toString(value)); + dvField.setLongValue(value); + writer.addDocument(doc); + } + + writer.forceMerge(1); + + writer.close(); + + // compare + assertDVIterate(dir); + assertDVAdvance(dir, 1); // Tests all jump-lengths from 1 to maxDoc (quite slow ~= 1 minute for 200K docs) + + dir.close(); + } + + // Tests that advanceExact does not change the outcome + private void assertDVAdvance(Directory dir, int jumpStep) throws IOException { + DirectoryReader ir = DirectoryReader.open(dir); + TestUtil.checkReader(ir); + for (LeafReaderContext context : ir.leaves()) { + LeafReader r = context.reader(); + + + for (int jump = jumpStep; jump < r.maxDoc(); jump += jumpStep) { + // Create a new instance each time to ensure jumps from the beginning + NumericDocValues docValues = DocValues.getNumeric(r, "dv"); + for (int docID = 0; docID < r.maxDoc(); docID += jump) { + String base = "document #" + docID + "/" + r.maxDoc() + ", jumping " + jump + " from #" + (docID-jump); + String storedValue = r.document(docID).get("stored"); + if (storedValue == null) { + assertFalse("There should be no DocValue for " + base, + docValues.advanceExact(docID)); + } else { + assertTrue("There should be a DocValue for " + base, + docValues.advanceExact(docID)); + assertEquals("The doc value should be correct for " + base, + Long.parseLong(storedValue), docValues.longValue()); + } + } + } + } + ir.close(); + } + +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java new file mode 100644 index 00000000000..e6116a937d4 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java @@ -0,0 +1,33 @@ +/* + * 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.codecs.lucene80; + + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.BaseNormsFormatTestCase; + +/** + * Tests Lucene80NormsFormat + */ +public class TestLucene80NormsFormat extends BaseNormsFormatTestCase { + private final Codec codec = new Lucene80Codec(); + + @Override + protected Codec getCodec() { + return codec; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java index 0214e5447b4..442fe7d7f69 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java @@ -34,7 +34,7 @@ import org.apache.lucene.util.LuceneTestCase; /** Tests helper methods in DocValues */ public class TestDocValues extends LuceneTestCase { - + /** * If the field doesn't exist, we return empty instances: * it can easily happen that a segment just doesn't have any docs with the field. @@ -123,8 +123,8 @@ public class TestDocValues extends LuceneTestCase { iw.close(); dir.close(); } - - /** + + /** * field with binary docvalues */ public void testBinaryField() throws Exception { diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java index bd11b47d48e..fd724966708 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java @@ -1204,6 +1204,9 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes } private void doTestNumericsVsStoredFields(double density, LongSupplier longs) throws Exception { + doTestNumericsVsStoredFields(density, longs, 256); + } + private void doTestNumericsVsStoredFields(double density, LongSupplier longs, int minDocs) throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf); @@ -1216,7 +1219,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes doc.add(dvField); // index some docs - int numDocs = atLeast(300); + int numDocs = atLeast((int) (minDocs*1.172)); // numDocs should be always > 256 so that in case of a codec that optimizes // for numbers of values <= 256, all storage layouts are tested assert numDocs > 256; @@ -1243,12 +1246,17 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes } // merge some segments and ensure that at least one of them has more than - // 256 values - writer.forceMerge(numDocs / 256); + // max(256, minDocs) values + writer.forceMerge(numDocs / Math.max(256, minDocs)); writer.close(); - // compare + assertDVIterate(dir); + dir.close(); + } + + // Asserts equality of stored value vs. DocValue by iterating DocValues one at a time + protected void assertDVIterate(Directory dir) throws IOException { DirectoryReader ir = DirectoryReader.open(dir); TestUtil.checkReader(ir); for (LeafReaderContext context : ir.leaves()) { @@ -1268,9 +1276,8 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID()); } ir.close(); - dir.close(); } - + private void doTestSortedNumericsVsStoredFields(LongSupplier counts, LongSupplier values) throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java index 64287a0111b..5350890ad30 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java @@ -53,7 +53,7 @@ import org.apache.lucene.codecs.asserting.AssertingCodec; import org.apache.lucene.codecs.blockterms.LuceneFixedGap; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; -import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat; +import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; import org.apache.lucene.codecs.lucene80.Lucene80Codec; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -930,7 +930,7 @@ public final class TestUtil { * Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene. */ public static DocValuesFormat getDefaultDocValuesFormat() { - return new Lucene70DocValuesFormat(); + return new Lucene80DocValuesFormat(); } // TODO: generalize all 'test-checks-for-crazy-codecs' to diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 65604c2e6c4..5f770a60353 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -284,6 +284,8 @@ Bug Fixes * SOLR-13137: NPE when /admin/zookeeper/status endpoint hit in standalone mode (janhoy) +* SOLR-13091: REBALANCELEADERS is broken (Erick Erickson) + Improvements ---------------------- diff --git a/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java b/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java index 953023f9153..f5672ba31cb 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java +++ b/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java @@ -17,6 +17,7 @@ package org.apache.solr.cloud; +import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -39,6 +40,8 @@ import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkStateReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ONLY_ACTIVE_NODES; import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SHARD_UNIQUE; @@ -46,6 +49,7 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.BA // Class to encapsulate processing replica properties that have at most one replica hosting a property per slice. class ExclusiveSliceProperty { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private ClusterState clusterState; private final boolean onlyActiveNodes; private final String property; @@ -235,6 +239,15 @@ class ExclusiveSliceProperty { adjustLimits(nodesHostingProp.get(nodeName)); removeSliceAlreadyHostedFromPossibles(srToChange.slice.getName()); addProp(srToChange.slice, srToChange.replica.getName()); + // When you set the property, you must insure that it is _removed_ from any other replicas. + for (Replica rep : srToChange.slice.getReplicas()) { + if (rep.getName().equals(srToChange.replica.getName())) { + continue; + } + if (rep.getProperty(property) != null) { + removeProp(srToChange.slice, srToChange.replica.getName()); + } + } } } @@ -266,10 +279,12 @@ class ExclusiveSliceProperty { } private void removeProp(Slice origSlice, String replicaName) { + log.debug("Removing property {} from slice {}, replica {}", property, origSlice.getName(), replicaName); getReplicaFromChanged(origSlice, replicaName).getProperties().remove(property); } private void addProp(Slice origSlice, String replicaName) { + log.debug("Adding property {} to slice {}, replica {}", property, origSlice.getName(), replicaName); getReplicaFromChanged(origSlice, replicaName).getProperties().put(property, "true"); } @@ -342,5 +357,10 @@ class ExclusiveSliceProperty { this.slice = slice; this.replica = replica; } + public String toString() { + StringBuilder sb = new StringBuilder(System.lineSeparator()).append(System.lineSeparator()).append("******EOE20 starting toString of SliceReplica"); + sb.append(" :").append(System.lineSeparator()).append("slice: ").append(slice.toString()).append(System.lineSeparator()).append(" replica: ").append(replica.toString()).append(System.lineSeparator()); + return sb.toString(); + } } } diff --git a/solr/core/src/java/org/apache/solr/handler/admin/RebalanceLeaders.java b/solr/core/src/java/org/apache/solr/handler/admin/RebalanceLeaders.java index f0819bdded9..522a432fe05 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/RebalanceLeaders.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/RebalanceLeaders.java @@ -18,10 +18,13 @@ package org.apache.solr.handler.admin; import java.lang.invoke.MethodHandles; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang.StringUtils; import org.apache.solr.cloud.LeaderElector; @@ -55,13 +58,62 @@ import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP; import static org.apache.solr.common.params.CollectionParams.CollectionAction.REBALANCELEADERS; import static org.apache.solr.common.params.CommonAdminParams.ASYNC; + +/** + * The end point for the collections API REBALANCELEADERS call that actually does the work. + *

    + * Overview: + *

    + * The leader election process is that each replica of a shard watches one, and only one other replica via + * ephemeral nodes in ZooKeeper. When the node being watched goes down, the node watching it is sent a notification + * and, if the node being watched is the leader, the node getting the notification assumes leadership. + *

    + * ZooKeeper's ephemeral nodes get a monotonically increasing "sequence number" that defines it's position in the queue + *

    + * So to force a particular node to become a leader it must have a watch on the leader. This can lead to two nodes + * having the same sequence number. Say the process is this + * replica1 is the leader (seq 1) + * replica3 is on a Solr node that happens to be started next, it watches the leader (seq2) + * replica2 is on the next Solr node started. It will _also_ watch the leader, it's sequence number is 2 exactly + * like replica3s + *

    + * This is true on startup, but can also be a consequence of, say, a replica going into recovery. It's no longer + * eligible to become leader, so will be put at the end of the queue by default. So there's code to put it in the + * queue with the same sequence number as the current second replica. + *

    + * To compilcate matters further, when the nodes are sorted (see OverseerTaskProcessor.getSortedElectionNodes) + * the primary sort is on the sequence number, secondary sort on the session ID. So the preferredLeader may + * or may not be second in that list. + *

    + * what all this means is that when the REBALANCELEADER command is issued, this class examines the election queue and + * performs just three things for each shard in the collection: + *

    + * 1> insures that the preferredLeader is watching the leader (rejoins the election queue at the head) + *

    + * 2> if there are two ephemeral nodes with the same sequence number watching the leader, and if one of them is the + * preferredLeader it will send the _other_ node to the end of the queue (rejoins it) + *

    + * 3> rejoins the zeroth entry in the list at the end of the queue, which triggers the watch on the preferredLeader + * replica which then takes over leadership + *

    + * All this of course assuming the preferedLeader is alive and well and is assigned for any given shard. + */ + class RebalanceLeaders { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - + final SolrQueryRequest req; final SolrQueryResponse rsp; final CollectionsHandler collectionsHandler; final CoreContainer coreContainer; + private final Set asyncRequests = new HashSet<>(); + final static String INACTIVE_PREFERREDS = "inactivePreferreds"; + final static String ALREADY_LEADERS = "alreadyLeaders"; + final static String SUMMARY = "Summary"; + final NamedList results = new NamedList<>(); + final Map pendingOps = new HashMap<>(); + private String collectionName; + RebalanceLeaders(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler collectionsHandler) { this.req = req; @@ -71,38 +123,29 @@ class RebalanceLeaders { } void execute() throws KeeperException, InterruptedException { - req.getParams().required().check(COLLECTION_PROP); + DocCollection dc = checkParams(); + - String collectionName = req.getParams().get(COLLECTION_PROP); - if (StringUtils.isBlank(collectionName)) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, - String.format(Locale.ROOT, "The " + COLLECTION_PROP + " is required for the Rebalance Leaders command.")); - } - coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName); - ClusterState clusterState = coreContainer.getZkController().getClusterState(); - DocCollection dc = clusterState.getCollection(collectionName); - if (dc == null) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' does not exist, no action taken."); - } - Map currentRequests = new HashMap<>(); int max = req.getParams().getInt(MAX_AT_ONCE_PROP, Integer.MAX_VALUE); if (max <= 0) max = Integer.MAX_VALUE; int maxWaitSecs = req.getParams().getInt(MAX_WAIT_SECONDS_PROP, 60); - NamedList results = new NamedList<>(); + + // If there are a maximum number of simultaneous requests specified, we have to pause when we have that many + // outstanding requests and wait for at least one to finish before going on the the next rebalance. boolean keepGoing = true; for (Slice slice : dc.getSlices()) { - ensurePreferredIsLeader(results, slice, currentRequests); - if (currentRequests.size() == max) { + ensurePreferredIsLeader(slice); + if (asyncRequests.size() == max) { log.info("Queued " + max + " leader reassignments, waiting for some to complete."); - keepGoing = waitForLeaderChange(currentRequests, maxWaitSecs, false, results); + keepGoing = waitAsyncRequests(maxWaitSecs, false); if (keepGoing == false) { break; // If we've waited longer than specified, don't continue to wait! } } } if (keepGoing == true) { - keepGoing = waitForLeaderChange(currentRequests, maxWaitSecs, true, results); + keepGoing = waitAsyncRequests(maxWaitSecs, true); } if (keepGoing == true) { log.info("All leader reassignments completed."); @@ -110,15 +153,72 @@ class RebalanceLeaders { log.warn("Exceeded specified timeout of ." + maxWaitSecs + "' all leaders may not have been reassigned"); } + checkLeaderStatus(); + NamedList summary = new NamedList<>(); + if (pendingOps.size() == 0) { + summary.add("Success", "All active replicas with the preferredLeader property set are leaders"); + } else { + summary.add("Failure", "Not all active replicas with preferredLeader property are leaders"); + } + rsp.getValues().add(SUMMARY, summary); // we want this first. + rsp.getValues().addAll(results); } - private void ensurePreferredIsLeader(NamedList results, - Slice slice, Map currentRequests) throws KeeperException, InterruptedException { - final String inactivePreferreds = "inactivePreferreds"; - final String alreadyLeaders = "alreadyLeaders"; - String collectionName = req.getParams().get(COLLECTION_PROP); + // Insure that ll required parameters are there and the doc colection exists. + private DocCollection checkParams() throws KeeperException, InterruptedException { + req.getParams().required().check(COLLECTION_PROP); + collectionName = req.getParams().get(COLLECTION_PROP); + if (StringUtils.isBlank(collectionName)) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, + String.format(Locale.ROOT, "The " + COLLECTION_PROP + " is required for the Rebalance Leaders command.")); + } + coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName); + ClusterState clusterState = coreContainer.getZkController().getClusterState(); + + DocCollection dc = clusterState.getCollection(collectionName); + if (dc == null) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' does not exist, no action taken."); + } + return dc; + } + + // Once we've done all the fiddling with the queues, check on the way out to see if all the active preferred + // leaders that we intended to change are in fact the leaders. + private void checkLeaderStatus() throws InterruptedException, KeeperException { + for (int idx = 0; pendingOps.size() > 0 && idx < 600; ++idx) { + ClusterState clusterState = coreContainer.getZkController().getClusterState(); + Set liveNodes = clusterState.getLiveNodes(); + DocCollection dc = clusterState.getCollection(collectionName); + for (Slice slice : dc.getSlices()) { + for (Replica replica : slice.getReplicas()) { + if (replica.isActive(liveNodes) && replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false)) { + if (replica.getBool(LEADER_PROP, false)) { + if (pendingOps.containsKey(slice.getName())) { + // Record for return that the leader changed successfully + pendingOps.remove(slice.getName()); + addToSuccesses(slice, replica); + break; + } + } + } + } + } + TimeUnit.MILLISECONDS.sleep(100); + coreContainer.getZkController().getZkStateReader().forciblyRefreshAllClusterStateSlow(); + } + addAnyFailures(); + } + + // The process is: + // if the replica with preferredLeader is already the leader, do nothing + // Otherwise: + // > if two nodes have the same sequence number and both point to the current leader, we presume that we've just + // moved it, move the one that does _not_ have the preferredLeader to the end of the list. + // > move the current leader to the end of the list. This _should_ mean that the current ephemeral node in the + // leader election queue is removed and the only remaining node watching it is triggered to become leader. + private void ensurePreferredIsLeader(Slice slice) throws KeeperException, InterruptedException { for (Replica replica : slice.getReplicas()) { // Tell the replica to become the leader if we're the preferred leader AND active AND not the leader already if (replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false) == false) { @@ -127,87 +227,126 @@ class RebalanceLeaders { // OK, we are the preferred leader, are we the actual leader? if (replica.getBool(LEADER_PROP, false)) { //We're a preferred leader, but we're _also_ the leader, don't need to do anything. - NamedList noops = (NamedList) results.get(alreadyLeaders); - if (noops == null) { - noops = new NamedList<>(); - results.add(alreadyLeaders, noops); - } - NamedList res = new NamedList<>(); - res.add("status", "success"); - res.add("msg", "Already leader"); - res.add("shard", slice.getName()); - res.add("nodeName", replica.getNodeName()); - noops.add(replica.getName(), res); + addAlreadyLeaderToResults(slice, replica); return; // already the leader, do nothing. } - + ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader(); // We're the preferred leader, but someone else is leader. Only become leader if we're active. - if (replica.getState() != Replica.State.ACTIVE) { - NamedList inactives = (NamedList) results.get(inactivePreferreds); - if (inactives == null) { - inactives = new NamedList<>(); - results.add(inactivePreferreds, inactives); - } - NamedList res = new NamedList<>(); - res.add("status", "skipped"); - res.add("msg", "Node is a referredLeader, but it's inactive. Skipping"); - res.add("shard", slice.getName()); - res.add("nodeName", replica.getNodeName()); - inactives.add(replica.getName(), res); + if (replica.isActive(zkStateReader.getClusterState().getLiveNodes()) == false) { + addInactiveToResults(slice, replica); return; // Don't try to become the leader if we're not active! } + List electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zkStateReader.getZkClient(), + ZkStateReader.getShardLeadersElectPath(collectionName, slice.getName())); + + if (electionQueueInBadState(electionNodes, slice, replica)) { + return; + } + // Replica is the preferred leader but not the actual leader, do something about that. // "Something" is // 1> if the preferred leader isn't first in line, tell it to re-queue itself. // 2> tell the actual leader to re-queue itself. - ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader(); - - List electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zkStateReader.getZkClient(), - ZkStateReader.getShardLeadersElectPath(collectionName, slice.getName())); - - if (electionNodes.size() < 2) { // if there's only one node in the queue, should already be leader and we shouldn't be here anyway. - log.info("Rebalancing leaders and slice " + slice.getName() + " has less than two elements in the leader " + - "election queue, but replica " + replica.getName() + " doesn't think it's the leader."); - return; - } - // Ok, the sorting for election nodes is a bit strange. If the sequence numbers are the same, then the whole // string is used, but that sorts nodes with the same sequence number by their session IDs from ZK. // While this is determinate, it's not quite what we need, so re-queue nodes that aren't us and are // watching the leader node.. + String firstWatcher = electionNodes.get(1); if (LeaderElector.getNodeName(firstWatcher).equals(replica.getName()) == false) { - makeReplicaFirstWatcher(collectionName, slice, replica); + makeReplicaFirstWatcher(slice, replica); } - String coreName = slice.getReplica(LeaderElector.getNodeName(electionNodes.get(0))).getStr(CORE_NAME_PROP); - rejoinElection(collectionName, slice, electionNodes.get(0), coreName, false); - waitForNodeChange(collectionName, slice, electionNodes.get(0)); - + // This replica should be the leader at the end of the day, so let's record that information to check at the end + pendingOps.put(slice.getName(), replica.getName()); + String leaderElectionNode = electionNodes.get(0); + String coreName = slice.getReplica(LeaderElector.getNodeName(leaderElectionNode)).getStr(CORE_NAME_PROP); + rejoinElectionQueue(slice, leaderElectionNode, coreName, false); + waitForNodeChange(slice, leaderElectionNode); return; // Done with this slice, skip the rest of the replicas. } } + + // Check that the election queue has some members! There really should be two or more for this to make any sense, + // if there's only one we can't change anything. + private boolean electionQueueInBadState(List electionNodes, Slice slice, Replica replica) { + if (electionNodes.size() < 2) { // if there's only one node in the queue, should already be leader and we shouldn't be here anyway. + log.warn("Rebalancing leaders and slice {} has less than two elements in the leader " + + "election queue, but replica {} doesn't think it's the leader.", slice.getName(), replica.getName()); + return true; + } + + return false; + } + + // Provide some feedback to the user about what actually happened, or in this case where no action was + // possible + private void addInactiveToResults(Slice slice, Replica replica) { + NamedList inactives = (NamedList) results.get(INACTIVE_PREFERREDS); + if (inactives == null) { + inactives = new NamedList<>(); + results.add(INACTIVE_PREFERREDS, inactives); + } + NamedList res = new NamedList<>(); + res.add("status", "skipped"); + res.add("msg", "Replica " + replica.getName() + " is a referredLeader for shard " + slice.getName() + ", but is inactive. No change necessary"); + inactives.add(replica.getName(), res); + } + + // Provide some feedback to the user about what actually happened, or in this case where no action was + // necesary since this preferred replica was already the leader + private void addAlreadyLeaderToResults(Slice slice, Replica replica) { + NamedList alreadyLeaders = (NamedList) results.get(ALREADY_LEADERS); + if (alreadyLeaders == null) { + alreadyLeaders = new NamedList<>(); + results.add(ALREADY_LEADERS, alreadyLeaders); + } + NamedList res = new NamedList<>(); + res.add("status", "skipped"); + res.add("msg", "Replica " + replica.getName() + " is already the leader for shard " + slice.getName() + ". No change necessary"); + alreadyLeaders.add(replica.getName(), res); + } + // Put the replica in at the head of the queue and send all nodes with the same sequence number to the back of the list - void makeReplicaFirstWatcher(String collectionName, Slice slice, Replica replica) + // There can be "ties", i.e. replicas in the queue with the same sequence number. Sorting doesn't necessarily sort + // the one we most care about first. So put the node we _don't care about at the end of the election queuel + + void makeReplicaFirstWatcher(Slice slice, Replica replica) throws KeeperException, InterruptedException { ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader(); List electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zkStateReader.getZkClient(), ZkStateReader.getShardLeadersElectPath(collectionName, slice.getName())); - // First, queue up the preferred leader at the head of the queue. + // First, queue up the preferred leader watching the leader if it isn't already + int secondSeq = Integer.MAX_VALUE; + + int candidateSeq = -1; + for (int idx = 1; idx < electionNodes.size(); ++idx) { + String candidate = electionNodes.get(idx); + secondSeq = Math.min(secondSeq, LeaderElector.getSeq(candidate)); + if (LeaderElector.getNodeName(candidate).equals(replica.getName())) { + candidateSeq = LeaderElector.getSeq(candidate); + } + } int newSeq = -1; - for (String electionNode : electionNodes) { - if (LeaderElector.getNodeName(electionNode).equals(replica.getName())) { - String coreName = slice.getReplica(LeaderElector.getNodeName(electionNode)).getStr(CORE_NAME_PROP); - rejoinElection(collectionName, slice, electionNode, coreName, true); - newSeq = waitForNodeChange(collectionName, slice, electionNode); - break; + if (candidateSeq == secondSeq) { + // the preferredLeader is already watching the leader, no need to move it around. + newSeq = secondSeq; + } else { + for (String electionNode : electionNodes) { + if (LeaderElector.getNodeName(electionNode).equals(replica.getName())) { + // Make the preferred leader watch the leader. + String coreName = slice.getReplica(LeaderElector.getNodeName(electionNode)).getStr(CORE_NAME_PROP); + rejoinElectionQueue(slice, electionNode, coreName, true); + newSeq = waitForNodeChange(slice, electionNode); + break; + } } } if (newSeq == -1) { @@ -225,18 +364,22 @@ class RebalanceLeaders { if (LeaderElector.getNodeName(thisNode).equals(replica.getName())) { continue; } + // We won't get here for the preferredLeader node if (LeaderElector.getSeq(thisNode) == newSeq) { String coreName = slice.getReplica(LeaderElector.getNodeName(thisNode)).getStr(CORE_NAME_PROP); - rejoinElection(collectionName, slice, thisNode, coreName, false); - waitForNodeChange(collectionName, slice, thisNode); + rejoinElectionQueue(slice, thisNode, coreName, false); + waitForNodeChange(slice, thisNode); } } } - int waitForNodeChange(String collectionName, Slice slice, String electionNode) throws InterruptedException, KeeperException { + // We're just waiting for the electionNode to rejoin the queue with a _different_ node, indicating that any + // requeueing we've done has happened. + int waitForNodeChange(Slice slice, String electionNode) throws InterruptedException, KeeperException { String nodeName = LeaderElector.getNodeName(electionNode); int oldSeq = LeaderElector.getSeq(electionNode); for (int idx = 0; idx < 600; ++idx) { + ZkStateReader zkStateReader = coreContainer.getZkController().getZkStateReader(); List electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zkStateReader.getZkClient(), ZkStateReader.getShardLeadersElectPath(collectionName, slice.getName())); @@ -245,14 +388,16 @@ class RebalanceLeaders { return LeaderElector.getSeq(testNode); } } - - Thread.sleep(100); + TimeUnit.MILLISECONDS.sleep(100); + zkStateReader.forciblyRefreshAllClusterStateSlow(); } return -1; } - - private void rejoinElection(String collectionName, Slice slice, String electionNode, String core, - boolean rejoinAtHead) throws KeeperException, InterruptedException { + + // Move an election node to some other place in the queue. If rejoinAtHead==false, then at the end, otherwise + // the new node should point at the leader. + private void rejoinElectionQueue(Slice slice, String electionNode, String core, boolean rejoinAtHead) + throws KeeperException, InterruptedException { Replica replica = slice.getReplica(LeaderElector.getNodeName(electionNode)); Map propMap = new HashMap<>(); propMap.put(COLLECTION_PROP, collectionName); @@ -265,64 +410,84 @@ class RebalanceLeaders { propMap.put(ELECTION_NODE_PROP, electionNode); String asyncId = REBALANCELEADERS.toLower() + "_" + core + "_" + Math.abs(System.nanoTime()); propMap.put(ASYNC, asyncId); + asyncRequests.add(asyncId); + collectionsHandler.sendToOCPQueue(new ZkNodeProps(propMap)); // ignore response; we construct our own } - // currentAsyncIds - map of request IDs and reporting data (value) // maxWaitSecs - How long are we going to wait? Defaults to 30 seconds. - // waitForAll - if true, do not return until all assignments have been made. - // results - a place to stash results for reporting back to the user. + // waitForAll - if true, do not return until all requests have been processed. "Processed" could mean failure! // - private boolean waitForLeaderChange(Map currentAsyncIds, final int maxWaitSecs, - Boolean waitForAll, NamedList results) + + private boolean waitAsyncRequests(final int maxWaitSecs, Boolean waitForAll) throws KeeperException, InterruptedException { - if (currentAsyncIds.size() == 0) return true; + if (asyncRequests.size() == 0) { + return true; + } for (int idx = 0; idx < maxWaitSecs * 10; ++idx) { - Iterator> iter = currentAsyncIds.entrySet().iterator(); + Iterator iter = asyncRequests.iterator(); boolean foundChange = false; while (iter.hasNext()) { - Map.Entry pair = iter.next(); - String asyncId = pair.getKey(); + String asyncId = iter.next(); if (coreContainer.getZkController().getOverseerFailureMap().contains(asyncId)) { coreContainer.getZkController().getOverseerFailureMap().remove(asyncId); coreContainer.getZkController().clearAsyncId(asyncId); - NamedList fails = (NamedList) results.get("failures"); - if (fails == null) { - fails = new NamedList<>(); - results.add("failures", fails); - } - NamedList res = new NamedList<>(); - res.add("status", "failed"); - res.add("msg", "Failed to assign '" + pair.getValue() + "' to be leader"); - fails.add(asyncId.substring(REBALANCELEADERS.toLower().length()), res); iter.remove(); foundChange = true; } else if (coreContainer.getZkController().getOverseerCompletedMap().contains(asyncId)) { coreContainer.getZkController().getOverseerCompletedMap().remove(asyncId); coreContainer.getZkController().clearAsyncId(asyncId); - NamedList successes = (NamedList) results.get("successes"); - if (successes == null) { - successes = new NamedList<>(); - results.add("successes", successes); - } - NamedList res = new NamedList<>(); - res.add("status", "success"); - res.add("msg", "Assigned '" + pair.getValue() + "' to be leader"); - successes.add(asyncId.substring(REBALANCELEADERS.toLower().length()), res); iter.remove(); foundChange = true; } } // We're done if we're processing a few at a time or all requests are processed. - if ((foundChange && waitForAll == false) || currentAsyncIds.size() == 0) { + // We don't want to change, say, 100s of leaders simultaneously. So if the request specifies some limit, + // and we're at that limit, we want to return to the caller so it can immediately add another request. + // That's the purpose of the first clause here. Otherwise, of course, just return if all requests are + // processed. + if ((foundChange && waitForAll == false) || asyncRequests.size() == 0) { return true; } - Thread.sleep(100); //TODO: Is there a better thing to do than sleep here? + TimeUnit.MILLISECONDS.sleep(100); } + // If we get here, we've timed out waiting. return false; } + // If we actually changed the leader, we should send that fact back in the response. + private void addToSuccesses(Slice slice, Replica replica) { + NamedList successes = (NamedList) results.get("successes"); + if (successes == null) { + successes = new NamedList<>(); + results.add("successes", successes); + } + log.info("Successfully changed leader of shard {} to replica {}", slice.getName(), replica.getName()); + NamedList res = new NamedList<>(); + res.add("status", "success"); + res.add("msg", "Successfully changed leader of slice " + slice.getName() + " to " + replica.getName()); + successes.add(slice.getName(), res); + } + // If for any reason we were supposed to change leadership, that should be recorded in changingLeaders. Any + // time we verified that the change actually occurred, that entry should have been removed. So report anything + // left over as a failure. + private void addAnyFailures() { + if (pendingOps.size() == 0) { + return; + } + NamedList fails = (NamedList) new NamedList<>(); + results.add("failures", fails); + + for (Map.Entry ent : pendingOps.entrySet()) { + log.info("Failed to change leader of shard {} to replica {}", ent.getKey(), ent.getValue()); + NamedList res = new NamedList<>(); + res.add("status", "failed"); + res.add("msg", String.format(Locale.ROOT, "Could not change leder for slice %s to %s", ent.getKey(), ent.getValue())); + fails.add(ent.getKey(), res); + + } + } } diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml index e2597401a19..5c88fcd14a6 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml @@ -19,7 +19,7 @@ - + diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java index b47424fe956..b207fa3c91c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java @@ -15,335 +15,629 @@ * limitations under the License. */ package org.apache.solr.cloud; + import java.io.IOException; import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.TimeUnit; +import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.ModifiableSolrParams; -import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.TimeSource; -import org.apache.solr.common.util.Utils; import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; +import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase { +@LuceneTestCase.Slow +public class TestRebalanceLeaders extends SolrCloudTestCase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static final String COLLECTION_NAME = "testcollection"; + private static final String COLLECTION_NAME = "TestColl"; + + private static int numNodes; + private static int numShards; + private static int numReplicas; + + private static boolean useAdminToSetProps = false; + + @BeforeClass + public static void setupCluster() throws Exception { + + numNodes = random().nextInt(4) + 4; + numShards = random().nextInt(3) + 3; + numReplicas = random().nextInt(2) + 2; + useAdminToSetProps = random().nextBoolean(); + + configureCluster(numNodes) + .addConfig(COLLECTION_NAME, configset("cloud-minimal")) + .configure(); + + CollectionAdminResponse resp = CollectionAdminRequest.createCollection(COLLECTION_NAME, COLLECTION_NAME, + numShards, numReplicas, 0, 0) + .setMaxShardsPerNode((numShards * numReplicas) / numNodes + 1) + .process(cluster.getSolrClient()); + assertEquals("Admin request failed; ", 0, resp.getStatus()); + cluster.waitForActiveCollection(COLLECTION_NAME, numShards, numShards * numReplicas); - public TestRebalanceLeaders() { - schemaString = "schema15.xml"; // we need a string id - sliceCount = 4; } - int reps = 10; + @Before + public void removeAllProperties() throws KeeperException, InterruptedException { + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + for (Slice slice : docCollection.getSlices()) { + for (Replica rep : slice.getReplicas()) { + rep.getProperties().forEach((key, value) -> { + if (key.startsWith("property.")) { + try { + delProp(slice, rep, key); + } catch (IOException | SolrServerException e) { + fail("Caught unexpected exception in @Before " + e.getMessage()); + } + } + }); + } + } + } + int timeoutMs = 60000; - Map> initial = new HashMap<>(); - Map expected = new HashMap<>(); + // test that setting an arbitrary "slice unique" property un-sets the property if it's on another replica in the + // slice. This is testing when the property is set on an _individual_ replica whereas testBalancePropertySliceUnique + // tests whether changing an individual _replica_ un-sets the property on other replicas _in that slice_. + // + // NOTE: There were significant problems because at one point the code implicitly defined + // shardUnique=true for the special property preferredLeader. That was removed at one point so we're explicitly + // testing that as well. @Test - @ShardsFixed(num = 4) - public void test() throws Exception { - reps = random().nextInt(9) + 1; // make sure and do at least one. - try (CloudSolrClient client = createCloudClient(null)) { - // Mix up a bunch of different combinations of shards and replicas in order to exercise boundary cases. - // shards, replicationfactor, maxreplicaspernode - int shards = random().nextInt(7); - if (shards < 2) shards = 2; - int rFactor = random().nextInt(4); - if (rFactor < 2) rFactor = 2; - createCollection(null, COLLECTION_NAME, shards, rFactor, shards * rFactor + 1, client, null, "conf1"); - } - - waitForCollection(cloudClient.getZkStateReader(), COLLECTION_NAME, 2); - waitForRecoveriesToFinish(COLLECTION_NAME, false); - - listCollection(); - rebalanceLeaderTest(); + public void testSetArbitraryPropertySliceUnique() throws IOException, SolrServerException, InterruptedException, KeeperException { + // Check both special (preferredLeader) and something arbitrary. + doTestSetArbitraryPropertySliceUnique("foo" + random().nextInt(1_000_000)); + removeAllProperties(); + doTestSetArbitraryPropertySliceUnique("preferredleader"); } - private void listCollection() throws IOException, SolrServerException { - //CloudSolrServer client = createCloudClient(null); - try { - ModifiableSolrParams params = new ModifiableSolrParams(); - params.set("action", CollectionParams.CollectionAction.LIST.toString()); - SolrRequest request = new QueryRequest(params); - request.setPath("/admin/collections"); - NamedList rsp = cloudClient.request(request); - List collections = (List) rsp.get("collections"); - assertTrue("control_collection was not found in list", collections.contains("control_collection")); - assertTrue(DEFAULT_COLLECTION + " was not found in list", collections.contains(DEFAULT_COLLECTION)); - assertTrue(COLLECTION_NAME + " was not found in list", collections.contains(COLLECTION_NAME)); - } finally { - //remove collections - //client.shutdown(); + // Test that automatically distributing a slice unique property un-sets that property if it's in any other replica + // on that slice. + // This is different than the test above. The test above sets individual properties on individual nodes. This one + // relies on Solr to pick which replicas to set the property on + @Test + public void testBalancePropertySliceUnique() throws KeeperException, InterruptedException, IOException, SolrServerException { + // Check both cases of "special" property preferred(Ll)eader + doTestBalancePropertySliceUnique("foo" + random().nextInt(1_000_000)); + removeAllProperties(); + doTestBalancePropertySliceUnique("preferredleader"); + } + + // We've moved on from a property being tested, we need to check if rebalancing the leaders actually chantges the + // leader appropriately. + @Test + public void testRebalanceLeaders() throws Exception { + + // First let's unbalance the preferredLeader property, do all the leaders get reassigned properly? + concentrateProp("preferredLeader"); + sendRebalanceCommand(); + checkPreferredsAreLeaders(); + + // Now follow up by evenly distributing the property as well as possible. + doTestBalancePropertySliceUnique("preferredLeader"); + sendRebalanceCommand(); + checkPreferredsAreLeaders(); + + // Now check the condition we saw "in the wild" where you could not rebalance properly when Jetty was restarted. + concentratePropByRestartingJettys(); + sendRebalanceCommand(); + checkPreferredsAreLeaders(); + } + + // Insure that the property is set on only one replica per slice when changing a unique property on an individual + // replica. + private void doTestSetArbitraryPropertySliceUnique(String propIn) throws InterruptedException, KeeperException, IOException, SolrServerException { + final String prop = (random().nextBoolean()) ? propIn : propIn.toUpperCase(Locale.ROOT); + // First set the property in some replica in some slice + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + + Slice[] slices = docCollection.getSlices().toArray(new Slice[0]); + Slice slice = slices[random().nextInt(slices.length)]; + + // Bounce around a bit setting this property and insure it's only set in one replica. + Replica[] reps = slice.getReplicas().toArray(new Replica[0]); + for (int idx = 0; idx < 4; ++idx) { + Replica rep = reps[random().nextInt(reps.length)]; + // Set the property on a particular replica + setProp(slice, rep, prop); + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + + long count = 0; + boolean rightRep = false; + Slice modSlice; + DocCollection modColl = null; // keeps IDE happy + + // insure that no other replica in that slice has the property when we return. + while (timeout.hasTimedOut() == false) { + forceUpdateCollectionStatus(); + modColl = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + modSlice = modColl.getSlice(slice.getName()); + rightRep = modSlice.getReplica(rep.getName()).getBool("property." + prop.toLowerCase(Locale.ROOT), false); + count = modSlice.getReplicas().stream().filter(thisRep -> thisRep.getBool("property." + prop.toLowerCase(Locale.ROOT), false)).count(); + + if (count == 1 && rightRep) { + break; + } + + TimeUnit.MILLISECONDS.sleep(100); + } + if (count != 1 || rightRep == false) { + fail("The property " + prop + " was not uniquely distributed in slice " + slice.getName() + + " " + modColl.toString()); + } } } - void recordInitialState() throws InterruptedException { - Map slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap(); - // Assemble a list of all the replicas for all the shards in a convenient way to look at them. - for (Map.Entry ent : slices.entrySet()) { - initial.put(ent.getKey(), new ArrayList<>(ent.getValue().getReplicas())); - } - } - - void rebalanceLeaderTest() throws InterruptedException, IOException, SolrServerException, KeeperException { - recordInitialState(); - for (int idx = 0; idx < reps; ++idx) { - issueCommands(); - checkConsistency(); - } - } - - // After we've called the rebalance command, we want to insure that: - // 1> all replicas appear once and only once in the respective leader election queue - // 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue. - // 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader. - void checkConsistency() throws InterruptedException, KeeperException { + // Fail if we the replicas with the preferredLeader property are _not_ also the leaders. + private void checkPreferredsAreLeaders() throws InterruptedException, KeeperException { + // Make sure that the shard unique are where you expect. TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); - boolean checkAppearOnce = false; - boolean checkElectionZero = false; - boolean checkZkLeadersAgree = false; - while (!timeout.hasTimedOut()) { - checkAppearOnce = checkAppearOnce(); - checkElectionZero = checkElectionZero(); - checkZkLeadersAgree = checkZkLeadersAgree(); - if (checkAppearOnce && checkElectionZero && checkZkLeadersAgree) { + + while (timeout.hasTimedOut() == false) { + if (checkPreferredsAreLeaders(false)) { + // Ok, all preferreds are leaders. Just for Let's also get the election queue and guarantee that every + // live replica is in the queue and none are repeated. + checkElectionQueues(); return; } - Thread.sleep(1000); + TimeUnit.MILLISECONDS.sleep(100); } - fail("Checking the rebalance leader command failed, checkAppearOnce=" + checkAppearOnce + " checkElectionZero=" - + checkElectionZero + " checkZkLeadersAgree=" + checkZkLeadersAgree); + log.error("Leaders are not all preferres {}", cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME)); + // Show the errors + checkPreferredsAreLeaders(true); } + // Do all active nodes in each slice appear exactly once in the slice's leader election queue? + // Since we assert that the number of live replicas is the same size as the leader election queue, we only + // have to compare one way. + private void checkElectionQueues() throws KeeperException, InterruptedException { - // Do all the nodes appear exactly once in the leader election queue and vice-versa? - Boolean checkAppearOnce() throws KeeperException, InterruptedException { + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + Set liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes(); - for (Map.Entry> ent : initial.entrySet()) { - List leaderQueue = cloudClient.getZkStateReader().getZkClient().getChildren("/collections/" + COLLECTION_NAME + - "/leader_elect/" + ent.getKey() + "/election", null, true); - - if (leaderQueue.size() != ent.getValue().size()) { - return false; - } - // Check that each election node has a corresponding replica. - for (String electionNode : leaderQueue) { - if (checkReplicaName(LeaderElector.getNodeName(electionNode), ent.getValue())) { - continue; + for (Slice slice : docCollection.getSlices()) { + Set liveReplicas = new HashSet<>(); + slice.getReplicas().forEach(replica -> { + if (replica.isActive(liveNodes)) { + liveReplicas.add(replica); } - return false; - } - // Check that each replica has an election node. - for (Replica rep : ent.getValue()) { - if (checkElectionNode(rep.getName(), leaderQueue)) { - continue; - } - return false; - } + }); + checkOneQueue(docCollection, slice, liveReplicas); } - return true; } - // Check that the given name is in the leader election queue - Boolean checkElectionNode(String repName, List leaderQueue) { + // Helper method to check one leader election queue's consistency. + private void checkOneQueue(DocCollection coll, Slice slice, Set liveReplicas) throws KeeperException, InterruptedException { + + List leaderQueue = cluster.getSolrClient().getZkStateReader().getZkClient().getChildren("/collections/" + COLLECTION_NAME + + "/leader_elect/" + slice.getName() + "/election", null, true); + + if (leaderQueue.size() != liveReplicas.size()) { + + log.error("One or more replicas is missing from the leader election queue! Slice {}, election queue: {}, collection: {}" + , slice.getName(), leaderQueue, coll); + fail("One or more replicas is missing from the leader election queue"); + } + // Check that each election node has a corresponding live replica. for (String electionNode : leaderQueue) { - if (repName.equals(LeaderElector.getNodeName(electionNode))) { - return true; + String replica = LeaderElector.getNodeName(electionNode); + if (slice.getReplica(replica) == null) { + log.error("Replica {} is not in the election queue: {}", replica, leaderQueue); + fail("Replica is not in the election queue!"); } } - return false; } - // Check that the name passed in corresponds to a replica. - Boolean checkReplicaName(String toCheck, List replicas) { - for (Replica rep : replicas) { - if (toCheck.equals(rep.getName())) { - return true; - } - } - return false; - } - - // Get the shard leader election from ZK and sort it. The node may not actually be there, so retry - List getOverseerSort(String key) { - List ret = null; - try { - ret = OverseerCollectionConfigSetProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(), - "/collections/" + COLLECTION_NAME + "/leader_elect/" + key + "/election"); - return ret; - } catch (KeeperException e) { - cloudClient.connect(); - } catch (InterruptedException e) { - return null; - } - return null; - } - - // Is every node we think is the leader in the zeroth position in the leader election queue? - Boolean checkElectionZero() { - for (Map.Entry ent : expected.entrySet()) { - - List leaderQueue = getOverseerSort(ent.getKey()); - if (leaderQueue == null) return false; - - String electName = LeaderElector.getNodeName(leaderQueue.get(0)); - String coreName = ent.getValue().getName(); - if (electName.equals(coreName) == false) { - return false; + // Just an encapsulation for checkPreferredsAreLeaders to make returning easier. + // the doAsserts var is to actually print the problem and fail the test if the condition is not met. + private boolean checkPreferredsAreLeaders(boolean doAsserts) throws KeeperException, InterruptedException { + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + for (Slice slice : docCollection.getSlices()) { + for (Replica rep : slice.getReplicas()) { + if (rep.getBool("property.preferredleader", false)) { + boolean isLeader = rep.getBool("leader", false); + if (doAsserts) { + assertTrue("PreferredLeader should be the leader: ", isLeader); + } else if (isLeader == false) { + return false; + } + } } } return true; } - // Do who we _think_ should be the leader agree with the leader nodes? - Boolean checkZkLeadersAgree() throws KeeperException, InterruptedException { - for (Map.Entry ent : expected.entrySet()) { - - String path = "/collections/" + COLLECTION_NAME + "/leaders/" + ent.getKey() + "/leader"; - byte[] data = getZkData(cloudClient, path); - if (data == null) { - log.warn("path to check not found {}", path); - return false; - } - - String repCore = null; - String zkCore = null; - - Map m = (Map) Utils.fromJSON(data); - zkCore = (String) m.get("core"); - repCore = ent.getValue().getStr("core"); - if (zkCore.equals(repCore) == false) { - log.warn("leader in zk does not match what we expect: {} != {}", zkCore, repCore); - return false; - } - - } - return true; - } - - byte[] getZkData(CloudSolrClient client, String path) { - org.apache.zookeeper.data.Stat stat = new org.apache.zookeeper.data.Stat(); - try { - byte[] data = client.getZkStateReader().getZkClient().getData(path, null, stat, true); - if (data != null) { - return data; - } - } catch (KeeperException.NoNodeException e) { - try { - Thread.sleep(1000); - } catch (InterruptedException e1) { - return null; - } - } catch (InterruptedException | KeeperException e) { - return null; - } - return null; - } - - // It's OK not to check the return here since the subsequent tests will fail. - void issueCommands() throws IOException, SolrServerException, KeeperException, InterruptedException { - - // Find a replica to make the preferredLeader. NOTE: may be one that's _already_ leader! - expected.clear(); - for (Map.Entry> ent : initial.entrySet()) { - List replicas = ent.getValue(); - Replica rep = replicas.get(Math.abs(random().nextInt()) % replicas.size()); - expected.put(ent.getKey(), rep); - issuePreferred(ent.getKey(), rep); - } - - if (waitForAllPreferreds() == false) { - fail("Waited for timeout for preferredLeader assignments to be made and they werent."); - } - //fillExpectedWithCurrent(); - // Now rebalance the leaders randomly using SolrJ or direct call - if(random().nextBoolean()) + // Arbitrarily send the rebalance command either with the SolrJ interface or with an HTTP request. + private void sendRebalanceCommand() throws SolrServerException, InterruptedException, IOException { + if (random().nextBoolean()) { rebalanceLeaderUsingSolrJAPI(); - else - rebalanceLeaderUsingDirectCall(); + } else { + rebalanceLeaderUsingStandardRequest(); + } + } + + // Helper method to make sure the property is _unbalanced_ first, then it gets properly re-assigned with the + // BALANCESHARDUNIQUE command. + private void doTestBalancePropertySliceUnique(String propIn) throws InterruptedException, IOException, KeeperException, SolrServerException { + final String prop = (random().nextBoolean()) ? propIn : propIn.toUpperCase(Locale.ROOT); + + // Concentrate the properties on as few replicas a possible + concentrateProp(prop); + + // issue the BALANCESHARDUNIQUE command + rebalancePropAndCheck(prop); + + // Verify that there are no more than one replica with the property per shard. + verifyPropUniquePerShard(prop); + + // Verify that the property is reasonably evenly distributed + verifyPropCorrectlyDistributed(prop); } - private void rebalanceLeaderUsingSolrJAPI() throws IOException, SolrServerException { - CollectionAdminRequest.RebalanceLeaders rebalanceLeaders = CollectionAdminRequest.rebalanceLeaders(COLLECTION_NAME); - rebalanceLeaders.setMaxAtOnce(10) - .process(cloudClient); + private void verifyPropCorrectlyDistributed(String prop) throws KeeperException, InterruptedException { + + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + + String propLC = prop.toLowerCase(Locale.ROOT); + DocCollection docCollection = null; + while (timeout.hasTimedOut() == false) { + forceUpdateCollectionStatus(); + docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + int maxPropCount = Integer.MAX_VALUE; + int minPropCount = Integer.MIN_VALUE; + for (Slice slice : docCollection.getSlices()) { + int repCount = 0; + for (Replica rep : slice.getReplicas()) { + if (rep.getBool("property." + propLC, false)) { + repCount++; + } + } + maxPropCount = Math.max(maxPropCount, repCount); + minPropCount = Math.min(minPropCount, repCount); + } + if (Math.abs(maxPropCount - minPropCount) < 2) return; + } + log.error("Property {} is not distributed evenly. {}", prop, docCollection); + fail("Property is not distributed evenly " + prop); } - private void rebalanceLeaderUsingDirectCall() throws IOException, SolrServerException { + // Used when we concentrate the leader on a few nodes. + private void verifyPropDistributedAsExpected(Map expectedShardReplicaMap, String prop) throws InterruptedException, KeeperException { + // Make sure that the shard unique are where you expect. + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + + String propLC = prop.toLowerCase(Locale.ROOT); + boolean failure = false; + DocCollection docCollection = null; + while (timeout.hasTimedOut() == false) { + forceUpdateCollectionStatus(); + docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + failure = false; + for (Map.Entry ent : expectedShardReplicaMap.entrySet()) { + Replica rep = docCollection.getSlice(ent.getKey()).getReplica(ent.getValue()); + if (rep.getBool("property." + propLC, false) == false) { + failure = true; + } + } + if (failure == false) { + return; + } + TimeUnit.MILLISECONDS.sleep(100); + } + + fail(prop + " properties are not on the expected replicas: " + docCollection.toString() + + System.lineSeparator() + "Expected " + expectedShardReplicaMap.toString()); + } + + // Just check that the property is distributed as expectecd. This does _not_ rebalance the leaders + private void rebalancePropAndCheck(String prop) throws IOException, SolrServerException, InterruptedException, KeeperException { + + if (random().nextBoolean()) { + rebalancePropUsingSolrJAPI(prop); + } else { + rebalancePropUsingStandardRequest(prop); + } + } + + + private void rebalanceLeaderUsingSolrJAPI() throws IOException, SolrServerException, InterruptedException { + CollectionAdminResponse resp = CollectionAdminRequest + .rebalanceLeaders(COLLECTION_NAME) + .process(cluster.getSolrClient()); + assertTrue("All leaders should have been verified", resp.getResponse().get("Summary").toString().contains("Success")); + assertEquals("Admin request failed; ", 0, resp.getStatus()); + } + + private void rebalanceLeaderUsingStandardRequest() throws IOException, SolrServerException { ModifiableSolrParams params = new ModifiableSolrParams(); params.set("action", CollectionParams.CollectionAction.REBALANCELEADERS.toString()); - - // Insure we get error returns when omitting required parameters params.set("collection", COLLECTION_NAME); - params.set("maxAtOnce", "10"); - SolrRequest request = new QueryRequest(params); + QueryRequest request = new QueryRequest(params); request.setPath("/admin/collections"); - cloudClient.request(request); - + QueryResponse resp = request.process(cluster.getSolrClient()); + assertTrue("All leaders should have been verified", resp.getResponse().get("Summary").toString().contains("Success")); + assertEquals("Call to rebalanceLeaders failed ", 0, resp.getStatus()); } - void issuePreferred(String slice, Replica rep) throws IOException, SolrServerException, InterruptedException { + + private void rebalancePropUsingSolrJAPI(String prop) throws IOException, SolrServerException, InterruptedException { + // Don't set the value, that should be done automatically. + CollectionAdminResponse resp; + + if (prop.toLowerCase(Locale.ROOT).contains("preferredleader")) { + resp = CollectionAdminRequest + .balanceReplicaProperty(COLLECTION_NAME, prop) + .process(cluster.getSolrClient()); + + } else { + resp = CollectionAdminRequest + .balanceReplicaProperty(COLLECTION_NAME, prop) + .setShardUnique(true) + .process(cluster.getSolrClient()); + + } + assertEquals("Admin request failed; ", 0, resp.getStatus()); + } + + private void rebalancePropUsingStandardRequest(String prop) throws IOException, SolrServerException { + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set("action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString()); + params.set("property", prop); + + params.set("collection", COLLECTION_NAME); + if (prop.toLowerCase(Locale.ROOT).contains("preferredleader") == false) { + params.set("shardUnique", true); + } + QueryRequest request = new QueryRequest(params); + request.setPath("/admin/collections"); + QueryResponse resp = request.process(cluster.getSolrClient()); + assertEquals("Call to rebalanceLeaders failed ", 0, resp.getStatus()); + } + + // This important. I've (Erick Erickson) run across a situation where the "standard request" causes failures, but + // never the Admin request. So let's test both all the time for a given test. + // + // This sets an _individual_ replica to have the property, not collection-wide + private void setProp(Slice slice, Replica rep, String prop) throws IOException, SolrServerException { + if (useAdminToSetProps) { + setPropWithAdminRequest(slice, rep, prop); + } else { + setPropWithStandardRequest(slice, rep, prop); + } + } + + void setPropWithStandardRequest(Slice slice, Replica rep, String prop) throws IOException, SolrServerException { ModifiableSolrParams params = new ModifiableSolrParams(); params.set("action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString()); - // Insure we get error returns when omitting required parameters - params.set("collection", COLLECTION_NAME); - params.set("shard", slice); + params.set("shard", slice.getName()); params.set("replica", rep.getName()); - params.set("property", "preferredLeader"); + params.set("property", prop); params.set("property.value", "true"); + // Test to insure that implicit shardUnique is added for preferredLeader. + if (prop.toLowerCase(Locale.ROOT).equals("preferredleader") == false) { + params.set("shardUnique", "true"); + } SolrRequest request = new QueryRequest(params); request.setPath("/admin/collections"); - cloudClient.request(request); + cluster.getSolrClient().request(request); + String propLC = prop.toLowerCase(Locale.ROOT); + waitForState("Expecting property '" + prop + "'to appear on replica " + rep.getName(), COLLECTION_NAME, + (n, c) -> "true".equals(c.getReplica(rep.getName()).getProperty(propLC))); + } - boolean waitForAllPreferreds() throws KeeperException, InterruptedException { - boolean goAgain = true; - TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); - while (! timeout.hasTimedOut()) { - goAgain = false; - Map slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap(); + void setPropWithAdminRequest(Slice slice, Replica rep, String prop) throws IOException, SolrServerException { + boolean setUnique = (prop.toLowerCase(Locale.ROOT).equals("preferredleader") == false); + CollectionAdminRequest.AddReplicaProp addProp = + CollectionAdminRequest.addReplicaProperty(COLLECTION_NAME, slice.getName(), rep.getName(), prop, "true"); + if (setUnique) { + addProp.setShardUnique(true); + } + CollectionAdminResponse resp = addProp.process(cluster.getSolrClient()); + assertEquals(0, resp.getStatus()); + String propLC = prop.toLowerCase(Locale.ROOT); + waitForState("Expecting property '" + prop + "'to appear on replica " + rep.getName(), COLLECTION_NAME, + (n, c) -> "true".equals(c.getReplica(rep.getName()).getProperty(propLC))); - for (Map.Entry ent : expected.entrySet()) { - Replica me = slices.get(ent.getKey()).getReplica(ent.getValue().getName()); - if (me.getBool("property.preferredleader", false) == false) { - goAgain = true; - break; + } + + private void delProp(Slice slice, Replica rep, String prop) throws IOException, SolrServerException { + String propLC = prop.toLowerCase(Locale.ROOT); + CollectionAdminResponse resp = CollectionAdminRequest.deleteReplicaProperty(COLLECTION_NAME, slice.getName(), rep.getName(), propLC) + .process(cluster.getSolrClient()); + assertEquals("Admin request failed; ", 0, resp.getStatus()); + waitForState("Expecting property '" + prop + "' to be removed from replica " + rep.getName(), COLLECTION_NAME, + (n, c) -> c.getReplica(rep.getName()).getProperty(prop) == null); + } + + // Intentionally un-balance the property to insure that BALANCESHARDUNIQUE does its job. There was an odd case + // where rebalancing didn't work very well if the Solr nodes were stopped and restarted that worked perfectly + // when if the nodes were _not_ restarted in the test. So we have to test that too. + private void concentratePropByRestartingJettys() throws Exception { + + List jettys = new ArrayList<>(cluster.getJettySolrRunners()); + Collections.shuffle(jettys, random()); + jettys.remove(random().nextInt(jettys.size())); + // Now we have a list of jettys, and there is one missing. Stop all of the remaining jettys, then start them again + // to concentrate the leaders. It's not necessary that all shards have a leader. + + for (JettySolrRunner jetty : jettys) { + cluster.stopJettySolrRunner(jetty); + cluster.waitForJettyToStop(jetty); + } + checkReplicasInactive(jettys); + + for (int idx = 0; idx < jettys.size(); ++idx) { + cluster.startJettySolrRunner(jettys.get(idx)); + } + cluster.waitForAllNodes(60); + // the nodes are present, but are all replica active? + checkAllReplicasActive(); + } + + // while banging my nead against a wall, I put a lot of force refresh statements in. Want to leave them in + // but have this be a no-op so if we start to get failures, we can re-enable with minimal effort. + private void forceUpdateCollectionStatus() throws KeeperException, InterruptedException { + // cluster.getSolrClient().getZkStateReader().forceUpdateCollection(COLLECTION_NAME); + } + + // Since we have to restart jettys, we don't want to try rebalancing etc. until we're sure all jettys that should + // be up are up and all replicas are active. + private void checkReplicasInactive(List downJettys) throws KeeperException, InterruptedException { + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + DocCollection docCollection = null; + Set liveNodes = null; + + Set downJettyNodes = new TreeSet<>(); + for (JettySolrRunner jetty : downJettys) { + downJettyNodes.add(jetty.getBaseUrl().getHost() + ":" + jetty.getBaseUrl().getPort() + "_solr"); + } + while (timeout.hasTimedOut() == false) { + forceUpdateCollectionStatus(); + docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes(); + boolean expectedInactive = true; + + for (Slice slice : docCollection.getSlices()) { + for (Replica rep : slice.getReplicas()) { + if (downJettyNodes.contains(rep.getNodeName()) == false) { + continue; // We are on a live node + } + // A replica on an allegedly down node is reported as active. + if (rep.isActive(liveNodes)) { + expectedInactive = false; + } } } - if (goAgain) { - Thread.sleep(250); - } else { - return true; + if (expectedInactive) { + return; } + TimeUnit.MILLISECONDS.sleep(100); } - return false; + fail("timed out waiting for all replicas to become inactive: livenodes: " + liveNodes + + " Collection state: " + docCollection.toString()); } -} + // We need to wait around until all replicas are active before expecting rebalancing or distributing shard-unique + // properties to work. + private void checkAllReplicasActive() throws KeeperException, InterruptedException { + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + while (timeout.hasTimedOut() == false) { + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + Set liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes(); + boolean allActive = true; + for (Slice slice : docCollection.getSlices()) { + for (Replica rep : slice.getReplicas()) { + if (rep.isActive(liveNodes) == false) { + allActive = false; + } + } + } + if (allActive) { + return; + } + TimeUnit.MILLISECONDS.sleep(100); + } + fail("timed out waiting for all replicas to become active"); + } + // use a simple heuristic to put as many replicas with the property on as few nodes as possible. The point is that + // then we can execute BALANCESHARDUNIQUE and be sure it worked correctly + private void concentrateProp(String prop) throws KeeperException, InterruptedException, IOException, SolrServerException { + // find all the live nodes + // for each slice, assign the leader to the first replica that is in the lowest position on live_nodes + List liveNodes = new ArrayList<>(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes()); + Collections.shuffle(liveNodes, random()); + + Map uniquePropMap = new TreeMap<>(); + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + for (Slice slice : docCollection.getSlices()) { + Replica changedRep = null; + int livePos = Integer.MAX_VALUE; + for (Replica rep : slice.getReplicas()) { + int pos = liveNodes.indexOf(rep.getNodeName()); + if (pos >= 0 && pos < livePos) { + livePos = pos; + changedRep = rep; + } + } + if (livePos == Integer.MAX_VALUE) { + fail("Invalid state! We should have a replica to add the property to! " + docCollection.toString()); + } + + uniquePropMap.put(slice.getName(), changedRep.getName()); + // Now set the property on the "lowest" node in live_nodes. + setProp(slice, changedRep, prop); + } + verifyPropDistributedAsExpected(uniquePropMap, prop); + } + + // make sure that the property in question is unique per shard. + private Map verifyPropUniquePerShard(String prop) throws InterruptedException, KeeperException { + Map uniquePropMaps = new TreeMap<>(); + + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME); + while (timeout.hasTimedOut() == false) { + uniquePropMaps.clear(); + if (checkdUniquePropPerShard(uniquePropMaps, prop)) { + return uniquePropMaps; + } + TimeUnit.MILLISECONDS.sleep(100); + } + fail("There should be exactly one replica with value " + prop + " set to true per shard: " + + cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).toString()); + return null; // keeps IDE happy. + } + + // return true if every shard has exactly one replica with the unique property set to "true" + private boolean checkdUniquePropPerShard(Map uniques, String prop) throws KeeperException, InterruptedException { + forceUpdateCollectionStatus(); + DocCollection docCollection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME); + + for (Slice slice : docCollection.getSlices()) { + int propfCount = 0; + for (Replica rep : slice.getReplicas()) { + if (rep.getBool("property." + prop.toLowerCase(Locale.ROOT), false)) { + propfCount++; + uniques.put(slice.getName(), rep.getName()); + } + } + if (1 != propfCount) { + return false; + } + } + return true; + } +} \ No newline at end of file diff --git a/solr/solr-ref-guide/build.xml b/solr/solr-ref-guide/build.xml index fb6a36af950..1c24d887105 100644 --- a/solr/solr-ref-guide/build.xml +++ b/solr/solr-ref-guide/build.xml @@ -296,7 +296,7 @@ Running Jekyll... - + diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc index 73c3e1bd061..ddaaeb8257f 100644 --- a/solr/solr-ref-guide/src/collections-api.adoc +++ b/solr/solr-ref-guide/src/collections-api.adoc @@ -1291,7 +1291,7 @@ The response will include the status of the request and the properties that were [source,text] ---- -http://localhost:8983/solr/admin/collections?action=COLLECTIONPROP&name=coll&propertyName=foo&val=bar&wt=xml +http://localhost:8983/solr/admin/collections?action=COLLECTIONPROP&name=coll&propertyName=foo&propertyValue=bar&wt=xml ---- *Output* @@ -2163,6 +2163,8 @@ http://localhost:8983/solr/admin/collections?action=REBALANCELEADERS&collection= In this example, two replicas in the "alreadyLeaders" section already had the leader assigned to the same node as the `preferredLeader` property so no action was taken. +The "Success" tag indicates that the command rebalanced all leaders. If, for any reason some replicas with preferredLeader=true are not leaders, this will be "Failure" rather than "Success". If a replica cannot be made leader due to not being "Active", it's also considered a failure. + The replica in the "inactivePreferreds" section had the `preferredLeader` property set but the node was down and no action was taken. The three nodes in the "successes" section were made leaders because they had the `preferredLeader` property set but were not leaders and they were active. [source,xml] @@ -2172,6 +2174,9 @@ The replica in the "inactivePreferreds" section had the `preferredLeader` proper 0 123 + + All replicas with the preferredLeader property set are leaders + success @@ -2219,6 +2224,10 @@ The replica in the "inactivePreferreds" section had the `preferredLeader` proper Examining the clusterstate after issuing this call should show that every live node that has the `preferredLeader` property should also have the "leader" property set to _true_. +NOTE: The added work done by an NRT leader during indexing is quite small. The primary use-case is to redistribute the leader role if there are a large number of leaders concentrated on a small number of nodes. Rebalancing will likely not improve performance unless the imbalance of leadership roles is measured in multiples of 10. + +NOTE: The BALANCESHARDUNIQUE command that distributes the preferredLeader property does not guarantee perfect distribution and in some collection topoligies it is impossible to make that guarantee. + [[forceleader]] == FORCELEADER: Force Shard Leader diff --git a/solr/solr-ref-guide/src/indexupgrader-tool.adoc b/solr/solr-ref-guide/src/indexupgrader-tool.adoc index bb01eeafa1d..e74ac032b47 100644 --- a/solr/solr-ref-guide/src/indexupgrader-tool.adoc +++ b/solr/solr-ref-guide/src/indexupgrader-tool.adoc @@ -1,4 +1,4 @@ -= IndexUpgrader Tool += IndexUpgraderTool // 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 @@ -16,7 +16,7 @@ // specific language governing permissions and limitations // under the License. -The Lucene distribution includes {lucene-javadocs}/core/org/apache/lucene/index/IndexUpgrader.html[a tool that upgrades] an index from previous Lucene versions to the current file format. +The Lucene distribution includes {lucene-javadocs}/core/org/apache/lucene/index/IndexUpgrader.html[a tool that upgrades] an index from the previous Lucene version to the current file format. The tool can be used from command line, or it can be instantiated and executed in Java. @@ -24,9 +24,11 @@ The tool can be used from command line, or it can be instantiated and executed i ==== Indexes can *only* be upgraded from the previous major release version to the current major release version. -This means that the IndexUpgrader Tool in any Solr 7.x release, for example, can only work with indexes from 6.x releases, but cannot work with indexes from Solr 5.x or earlier. +This means that the IndexUpgraderTool in any Solr 8.x release, for example, can only work with indexes from 7.x releases, but cannot work with indexes from Solr 6.x or earlier. -If you are currently using an earlier release such as 5.x and want to move more than one major version ahead, you need to first upgrade your indexes to the next major version (6.x), then again to the major version after that (7.x), etc. +If you are currently using a release two or more major versions older, such as moving from Solr 6x to Solr 8x, you will need to reindex your content. + +The IndexUpgraderTool performs a forceMerge (optimize) down to one segment, which may be undesirable. ==== In a Solr distribution, the Lucene files are located in `./server/solr-webapp/webapp/WEB-INF/lib`. You will need to include the `lucene-core-.jar` and `lucene-backwards-codecs-.jar` on the classpath when running the tool. diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc index 279a396fc78..b7c438c42ab 100644 --- a/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc +++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc @@ -49,13 +49,13 @@ The selection of the node that will host the new replica is made according to th == Cluster Preferences -Cluster preferences allow you to tell Solr how to assess system load on each node. This information is used to guide selection of the node(s) on which cluster management operations will be performed. +Cluster preferences allow you to tell Solr how to assess system load on each node. This information is used to guide selection of the node(s) on which cluster management operations will be performed. -In general, when an operation increases replica counts, the *least loaded* <> will be chosen, and when the operation reduces replica counts, the *most loaded* qualified node will be chosen. +In general, when an operation increases replica counts, the *least loaded* <> will be chosen, and when the operation reduces replica counts, the *most loaded* qualified node will be chosen. The default cluster preferences are `[{minimize:cores},{maximize:freedisk}]`, which tells Solr to minimize the number of cores on all nodes and if number of cores are equal, maximize the free disk space available. In this case, the least loaded node is the one with the fewest cores or if two nodes have an equal number of cores, the node with the most free disk space. -You can learn more about preferences in the <> section. +You can learn more about preferences in the section on <>. == Cluster Policy @@ -63,7 +63,7 @@ A cluster policy is a set of rules that a node, shard, or collection must satisf There are many metrics on which the rule can be based, e.g., system load average, heap usage, free disk space, etc. The full list of supported metrics can be found in the section describing <>. -When a node, shard, or collection does not satisfy a policy rule, we call it a *violation*. By default, cluster management operations will fail if there is even one violation. You can allow operations to succeed in the face of a violation by marking the corresponding rule with <>. When you do this, Solr ensures that cluster management operations minimize the number of violations. +When a node, shard, or collection does not satisfy a policy rule, we call it a *violation*. By default, cluster management operations will fail if there is even one violation. You can allow operations to succeed in the face of a violation by marking the corresponding rule with <>. When you do this, Solr ensures that cluster management operations minimize the number of violations. Solr also supports <>, which operate in tandem with the cluster policy. diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc index a20ea7c82f6..c4217030b8f 100644 --- a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc +++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc @@ -26,7 +26,11 @@ See the section <> for an exa == Cluster Preferences Specification -A preference is a hint to Solr on how to sort nodes based on their utilization. The default cluster preference is to sort by the total number of Solr cores (or replicas) hosted by a node. Therefore, by default, when selecting a node to which to add a replica, Solr can apply the preferences and choose the node with the fewest cores. +A preference is a hint to Solr on how to sort nodes based on their utilization. + +The default cluster preference is to sort by the total number of Solr cores (or replicas) hosted by a node, with a precision of 1. +Therefore, by default, when selecting a node to which to add a replica, Solr can apply the preferences and choose the node with the fewest cores. +In the case of a tie in the number of cores, available freedisk will be used to further sort nodes. More than one preference can be added to break ties. For example, we may choose to use free disk space to break ties if the number of cores on two nodes is the same. The node with the higher free disk space can be chosen as the target of the cluster operation. diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc index 09d291c727d..6d386c891aa 100644 --- a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc +++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc @@ -318,7 +318,6 @@ Non-zero values are useful for large indexes with aggressively growing size, as avalanches of split shard requests when the total size of the index reaches even multiples of the maximum shard size thresholds. - Events generated by this trigger contain additional details about the shards that exceeded thresholds and the types of violations (upper / lower bounds, bytes / docs metrics). @@ -546,4 +545,4 @@ ever executing if a new scheduled event is ready as soon as the cooldown period Solr randomizes the order in which the triggers are resumed after the cooldown period to mitigate this problem. However, it is recommended that scheduled triggers are not used with low `every` values and an external scheduling process such as cron be used for such cases instead. -==== \ No newline at end of file +==== diff --git a/solr/solr-ref-guide/src/working-with-dates.adoc b/solr/solr-ref-guide/src/working-with-dates.adoc index 7a42909b806..c86eaa12207 100644 --- a/solr/solr-ref-guide/src/working-with-dates.adoc +++ b/solr/solr-ref-guide/src/working-with-dates.adoc @@ -60,7 +60,8 @@ These are valid queries: + Solr's `DateRangeField` supports the same point in time date syntax described above (with _date math_ described below) and more to express date ranges. One class of examples is truncated dates, which represent the entire date span to the precision indicated. The other class uses the range syntax (`[ TO ]`). Here are some examples: * `2000-11` – The entire month of November, 2000. -* `2000-11T13` – Likewise but for an hour of the day (1300 to before 1400, i.e., 1pm to 2pm). +* `1605-11-05` – The Fifth of November. +* `2000-11-05T13` – Likewise but for an hour of the day (1300 to before 1400, i.e., 1pm to 2pm). * `-0009` – The year 10 BC. A 0 in the year position is 0 AD, and is also considered 1 BC. * `[2000-11-01 TO 2014-12-01]` – The specified date range at a day resolution. * `[2014 TO 2014-12-01]` – From the start of 2014 till the end of the first day of December.