From 74880a0df58a4d93fcd97ca59f8e23c053bde8b1 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 23 Oct 2025 12:58:20 -0500 Subject: [PATCH 01/64] Copy binary compression from LUCENE-9211 Reintroduce the LZ4 binary doc values compression originally added to Lucene in LUCENE-9211. Modify so that works in ES819TSDBDocValuesFormat --- .../codec/tsdb/BinaryDVCompressionMode.java | 30 +++ .../es819/ES819TSDBDocValuesConsumer.java | 225 ++++++++++++++++++ .../tsdb/es819/ES819TSDBDocValuesFormat.java | 11 +- .../es819/ES819TSDBDocValuesProducer.java | 194 +++++++++++++-- .../codec/tsdb/DocValuesCodecDuelTests.java | 4 +- .../codec/tsdb/TsdbDocValueBwcTests.java | 4 +- .../es819/ES819TSDBDocValuesFormatTests.java | 9 +- ...ValuesFormatVariableSkipIntervalTests.java | 4 +- 8 files changed, 459 insertions(+), 22 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java new file mode 100644 index 0000000000000..534c8117eaae0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -0,0 +1,30 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.tsdb; + +public enum BinaryDVCompressionMode { + + NO_COMPRESS((byte) 0), + COMPRESSED_WITH_LZ4((byte) 1); + + public final byte code; + + BinaryDVCompressionMode(byte code) { + this.code = code; + } + + public static BinaryDVCompressionMode fromMode(byte mode) { + return switch (mode) { + case 0 -> NO_COMPRESS; + case 1 -> COMPRESSED_WITH_LZ4; + default -> throw new IllegalStateException("unknown compression mode [" + mode + "]"); + }; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 968e50eaf32be..94177f655ded3 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -9,10 +9,12 @@ package org.elasticsearch.index.codec.tsdb.es819; +import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene90.IndexedDISI; import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValuesSkipIndexType; import org.apache.lucene.index.FieldInfo; @@ -29,6 +31,7 @@ import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; @@ -41,8 +44,10 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter; import org.apache.lucene.util.packed.PackedInts; import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -65,8 +70,11 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { private final int minDocsPerOrdinalForOrdinalRangeEncoding; final boolean enableOptimizedMerge; private final int primarySortFieldNumber; + private final SegmentWriteState state; + private final BinaryDVCompressionMode binaryDVCompressionMode; ES819TSDBDocValuesConsumer( + BinaryDVCompressionMode binaryDVCompressionMode, SegmentWriteState state, int skipIndexIntervalSize, int minDocsPerOrdinalForOrdinalRangeEncoding, @@ -76,6 +84,8 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { String metaCodec, String metaExtension ) throws IOException { + this.binaryDVCompressionMode = binaryDVCompressionMode; + this.state = state; this.termsDictBuffer = new byte[1 << 14]; this.dir = state.directory; this.minDocsPerOrdinalForOrdinalRangeEncoding = minDocsPerOrdinalForOrdinalRangeEncoding; @@ -315,7 +325,14 @@ public void mergeBinaryField(FieldInfo mergeFieldInfo, MergeState mergeState) th public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { meta.writeInt(field.number); meta.writeByte(ES819TSDBDocValuesFormat.BINARY); + meta.writeByte(binaryDVCompressionMode.code); + switch (binaryDVCompressionMode) { + case NO_COMPRESS -> doAddUncompressedBinary(field, valuesProducer); + case COMPRESSED_WITH_LZ4 -> doAddCompressedBinary(field, valuesProducer); + } + } + private void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { if (valuesProducer instanceof TsdbDocValuesProducer tsdbValuesProducer && tsdbValuesProducer.mergeStats.supported()) { final int numDocsWithField = tsdbValuesProducer.mergeStats.sumNumDocsWithField(); final int minLength = tsdbValuesProducer.mergeStats.minLength(); @@ -444,6 +461,214 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th } } + // BEGIN: Copied fom LUCENE-9211 + private void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) { + 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(); + blockWriter.addDoc(doc, v); + int length = v.length; + minLength = Math.min(length, minLength); + maxLength = Math.max(length, maxLength); + } + blockWriter.flushData(); + + 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); + + blockWriter.writeMetaData(); + } + } + + static final int BINARY_BLOCK_SHIFT = 5; + static final int BINARY_DOCS_PER_COMPRESSED_BLOCK = 1 << BINARY_BLOCK_SHIFT; + + private class CompressedBinaryBlockWriter implements Closeable { + final LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); + int uncompressedBlockLength = 0; + int maxUncompressedBlockLength = 0; + int numDocsInCurrentBlock = 0; + final int[] docLengths = new int[BINARY_DOCS_PER_COMPRESSED_BLOCK]; + byte[] block = BytesRef.EMPTY_BYTES; + int totalChunks = 0; + long maxPointer = 0; + final long blockAddressesStart; + + final IndexOutput tempBinaryOffsets; + + CompressedBinaryBlockWriter() throws IOException { + tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( + state.directory, + state.segmentInfo.name, + "binary_pointers", + state.context + ); + boolean success = false; + try { + CodecUtil.writeHeader( + tempBinaryOffsets, + ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + blockAddressesStart = data.getFilePointer(); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } + } + + void addDoc(int doc, BytesRef v) throws IOException { + docLengths[numDocsInCurrentBlock] = v.length; + block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); + System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); + uncompressedBlockLength += v.length; + numDocsInCurrentBlock++; + if (numDocsInCurrentBlock == BINARY_DOCS_PER_COMPRESSED_BLOCK) { + flushData(); + } + } + + private void flushData() throws IOException { + if (numDocsInCurrentBlock > 0) { + // Write offset to this block to temporary offsets file + totalChunks++; + long thisBlockStartPointer = data.getFilePointer(); + + // Optimisation - check if all lengths are same + boolean allLengthsSame = true; + for (int i = 1; i < BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { + if (docLengths[i] != docLengths[i - 1]) { + allLengthsSame = false; + break; + } + } + if (allLengthsSame) { + // Only write one value shifted. Steal a bit to indicate all other lengths are the same + int onlyOneLength = (docLengths[0] << 1) | 1; + data.writeVInt(onlyOneLength); + } else { + for (int i = 0; i < BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { + if (i == 0) { + // Write first value shifted and steal a bit to indicate other lengths are to follow + int multipleLengths = (docLengths[0] << 1); + data.writeVInt(multipleLengths); + } else { + data.writeVInt(docLengths[i]); + } + } + } + maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); + LZ4.compress(block, 0, uncompressedBlockLength, EndiannessReverserUtil.wrapDataOutput(data), ht); + numDocsInCurrentBlock = 0; + // Ensure initialized with zeroes because full array is always written + Arrays.fill(docLengths, 0); + uncompressedBlockLength = 0; + maxPointer = data.getFilePointer(); + tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer); + } + } + + void writeMetaData() throws IOException { + if (totalChunks == 0) { + return; + } + + long startDMW = data.getFilePointer(); + meta.writeLong(startDMW); + + meta.writeVInt(totalChunks); + meta.writeVInt(BINARY_BLOCK_SHIFT); + meta.writeVInt(maxUncompressedBlockLength); + meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); + + CodecUtil.writeFooter(tempBinaryOffsets); + IOUtils.close(tempBinaryOffsets); + // write the compressed block offsets info to the meta file by reading from temp file + try ( + ChecksumIndexInput filePointersIn = EndiannessReverserUtil.openChecksumInput( + state.directory, + tempBinaryOffsets.getName(), + IOContext.READONCE + ) + ) { + CodecUtil.checkHeader( + filePointersIn, + ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", + ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + Throwable priorE = null; + try { + final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance( + meta, + data, + totalChunks, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + ); + long fp = blockAddressesStart; + for (int i = 0; i < totalChunks; ++i) { + filePointers.add(fp); + fp += filePointersIn.readVLong(); + } + if (maxPointer < fp) { + throw new CorruptIndexException( + "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")", + filePointersIn + ); + } + filePointers.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(filePointersIn, priorE); + } + } + // Write the length of the DMW block in the data + meta.writeLong(data.getFilePointer() - startDMW); + } + + @Override + public void close() throws IOException { + if (tempBinaryOffsets != null) { + IOUtils.close(tempBinaryOffsets, () -> state.directory.deleteFile(tempBinaryOffsets.getName())); + } + } + } + // END: Copied fom LUCENE-9211 + @Override public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { meta.writeInt(field.number); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index fbdef488b8318..bd2865df2f1e2 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -14,6 +14,7 @@ import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.elasticsearch.core.SuppressForbidden; +import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import java.io.IOException; @@ -47,7 +48,8 @@ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValues static final byte SORTED_NUMERIC = 4; static final int VERSION_START = 0; - static final int VERSION_CURRENT = VERSION_START; + static final int VERSION_BINARY_DV_COMPRESSION = VERSION_START; + static final int VERSION_CURRENT = VERSION_BINARY_DV_COMPRESSION; static final int TERMS_DICT_BLOCK_LZ4_SHIFT = 6; static final int TERMS_DICT_BLOCK_LZ4_SIZE = 1 << TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -119,14 +121,15 @@ private static boolean getOptimizedMergeEnabledDefault() { final int skipIndexIntervalSize; final int minDocsPerOrdinalForRangeEncoding; private final boolean enableOptimizedMerge; + final BinaryDVCompressionMode binaryDVCompressionMode; /** Default constructor. */ public ES819TSDBDocValuesFormat() { - this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT); + this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.NO_COMPRESS); } /** Doc values fields format with specified skipIndexIntervalSize. */ - public ES819TSDBDocValuesFormat(int skipIndexIntervalSize, int minDocsPerOrdinalForRangeEncoding, boolean enableOptimizedMerge) { + public ES819TSDBDocValuesFormat(int skipIndexIntervalSize, int minDocsPerOrdinalForRangeEncoding, boolean enableOptimizedMerge, BinaryDVCompressionMode binaryDVCompressionMode) { super(CODEC_NAME); if (skipIndexIntervalSize < 2) { throw new IllegalArgumentException("skipIndexIntervalSize must be > 1, got [" + skipIndexIntervalSize + "]"); @@ -134,11 +137,13 @@ public ES819TSDBDocValuesFormat(int skipIndexIntervalSize, int minDocsPerOrdinal this.skipIndexIntervalSize = skipIndexIntervalSize; this.minDocsPerOrdinalForRangeEncoding = minDocsPerOrdinalForRangeEncoding; this.enableOptimizedMerge = enableOptimizedMerge; + this.binaryDVCompressionMode = binaryDVCompressionMode; } @Override public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { return new ES819TSDBDocValuesConsumer( + binaryDVCompressionMode, state, skipIndexIntervalSize, minDocsPerOrdinalForRangeEncoding, diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 5d90f2814853d..cf4cce7bceb6f 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -9,6 +9,7 @@ package org.elasticsearch.index.codec.tsdb.es819; +import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene90.IndexedDISI; @@ -45,12 +46,15 @@ import org.apache.lucene.util.packed.PackedInts; import org.elasticsearch.core.Assertions; import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; import org.elasticsearch.index.mapper.BlockDocValuesReader; import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; +import static org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode.COMPRESSED_WITH_LZ4; +import static org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode.NO_COMPRESS; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -97,7 +101,7 @@ final class ES819TSDBDocValuesProducer extends DocValuesProducer { state.segmentSuffix ); - readFields(in, state.fieldInfos); + readFields(in, state.fieldInfos, version); } catch (Throwable exception) { priorE = exception; @@ -193,6 +197,14 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return DocValues.emptyBinary(); } + return switch (entry.compression) { + case NO_COMPRESS -> getUncompressedBinary(entry); + case COMPRESSED_WITH_LZ4 -> getCompressedBinary(entry); + }; + } + + private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException { + final RandomAccessInput bytesSlice = data.randomAccessSlice(entry.dataOffset, entry.dataLength); if (entry.docsWithFieldOffset == -1) { @@ -267,6 +279,133 @@ public BytesRef binaryValue() throws IOException { } } + // START: Copied fom LUCENE-9211 + private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException { + if (entry.docsWithFieldOffset == -1) { + // dense + final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new DenseBinaryDocValues(maxDoc) { + final BinaryDecoder decoder = new BinaryDecoder( + addresses, + data.clone(), + entry.maxUncompressedChunkSize, + entry.docsPerChunkShift + ); + + @Override + public BytesRef binaryValue() throws IOException { + return decoder.decode(doc); + } + }; + } else { + // sparse + final IndexedDISI disi = new IndexedDISI( + data, + entry.docsWithFieldOffset, + entry.docsWithFieldLength, + entry.jumpTableEntryCount, + entry.denseRankPower, + entry.numDocsWithField + ); + final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); + final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + return new SparseBinaryDocValues(disi) { + final BinaryDecoder decoder = new BinaryDecoder( + addresses, + data.clone(), + entry.maxUncompressedChunkSize, + entry.docsPerChunkShift + ); + + @Override + public BytesRef binaryValue() throws IOException { + return decoder.decode(disi.index()); + } + }; + } + + } + + // Decompresses blocks of binary values to retrieve content + static final class BinaryDecoder { + + private final LongValues addresses; + private final IndexInput compressedData; + // Cache of last uncompressed block + private long lastBlockId = -1; + private final int[] uncompressedDocStarts; + private final byte[] uncompressedBlock; + private final BytesRef uncompressedBytesRef; + private final int docsPerChunk; + private final int docsPerChunkShift; + + BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize, int docsPerChunkShift) { + super(); + this.addresses = addresses; + this.compressedData = compressedData; + // pre-allocate a byte array large enough for the biggest uncompressed block needed. + this.uncompressedBlock = new byte[biggestUncompressedBlockSize]; + uncompressedBytesRef = new BytesRef(uncompressedBlock); + this.docsPerChunk = 1 << docsPerChunkShift; + this.docsPerChunkShift = docsPerChunkShift; + uncompressedDocStarts = new int[docsPerChunk + 1]; + } + + BytesRef decode(int docNumber) throws IOException { + int blockId = docNumber >> docsPerChunkShift; + int docInBlockId = docNumber % docsPerChunk; + assert docInBlockId < docsPerChunk; + + // already read and uncompressed? + if (blockId != lastBlockId) { + lastBlockId = blockId; + long blockStartOffset = addresses.get(blockId); + compressedData.seek(blockStartOffset); + + int uncompressedBlockLength = 0; + + int onlyLength = -1; + for (int i = 0; i < docsPerChunk; i++) { + if (i == 0) { + // The first length value is special. It is shifted and has a bit to denote if + // all other values are the same length + int lengthPlusSameInd = compressedData.readVInt(); + int sameIndicator = lengthPlusSameInd & 1; + int firstValLength = lengthPlusSameInd >>> 1; + if (sameIndicator == 1) { + onlyLength = firstValLength; + } + uncompressedBlockLength += firstValLength; + } else { + if (onlyLength == -1) { + // Various lengths are stored - read each from disk + uncompressedBlockLength += compressedData.readVInt(); + } else { + // Only one length + uncompressedBlockLength += onlyLength; + } + } + uncompressedDocStarts[i + 1] = uncompressedBlockLength; + } + + if (uncompressedBlockLength == 0) { + uncompressedBytesRef.offset = 0; + uncompressedBytesRef.length = 0; + return uncompressedBytesRef; + } + + assert uncompressedBlockLength <= uncompressedBlock.length; + LZ4.decompress(EndiannessReverserUtil.wrapDataInput(compressedData), uncompressedBlockLength, uncompressedBlock, 0); + } + + uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId]; + uncompressedBytesRef.length = uncompressedDocStarts[docInBlockId + 1] - uncompressedBytesRef.offset; + return uncompressedBytesRef; + } + } + // END: Copied fom LUCENE-9211 + private abstract static class DenseBinaryDocValues extends BinaryDocValues { final int maxDoc; @@ -1087,7 +1226,7 @@ static int primarySortFieldNumber(SegmentInfo segmentInfo, FieldInfos fieldInfos return -1; } - private void readFields(IndexInput meta, FieldInfos infos) throws IOException { + private void readFields(IndexInput meta, FieldInfos infos, int version) throws IOException { for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { FieldInfo info = infos.fieldInfo(fieldNumber); if (info == null) { @@ -1100,7 +1239,7 @@ private void readFields(IndexInput meta, FieldInfos infos) throws IOException { if (type == ES819TSDBDocValuesFormat.NUMERIC) { numerics.put(info.number, readNumeric(meta)); } else if (type == ES819TSDBDocValuesFormat.BINARY) { - binaries.put(info.number, readBinary(meta)); + binaries.put(info.number, readBinary(meta, version)); } else if (type == ES819TSDBDocValuesFormat.SORTED) { sorted.put(info.number, readSorted(meta)); } else if (type == ES819TSDBDocValuesFormat.SORTED_SET) { @@ -1162,8 +1301,15 @@ private static void readNumeric(IndexInput meta, NumericEntry entry) throws IOEx entry.denseRankPower = meta.readByte(); } - private BinaryEntry readBinary(IndexInput meta) throws IOException { - final BinaryEntry entry = new BinaryEntry(); + private BinaryEntry readBinary(IndexInput meta, int version) throws IOException { + final BinaryDVCompressionMode compression; + if (version >= ES819TSDBDocValuesFormat.VERSION_BINARY_DV_COMPRESSION) { + compression = BinaryDVCompressionMode.fromMode(meta.readByte()); + } else { + compression = BinaryDVCompressionMode.NO_COMPRESS; + } + final BinaryEntry entry = new BinaryEntry(compression); + entry.dataOffset = meta.readLong(); entry.dataLength = meta.readLong(); entry.docsWithFieldOffset = meta.readLong(); @@ -1173,15 +1319,27 @@ private BinaryEntry readBinary(IndexInput meta) throws IOException { entry.numDocsWithField = meta.readInt(); entry.minLength = meta.readInt(); entry.maxLength = meta.readInt(); - if (entry.minLength < entry.maxLength) { - entry.addressesOffset = meta.readLong(); - - // Old count of uncompressed addresses - long numAddresses = entry.numDocsWithField + 1L; - - final int blockShift = meta.readVInt(); - entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift); - entry.addressesLength = meta.readLong(); + if (compression == BinaryDVCompressionMode.NO_COMPRESS) { + if (entry.minLength < entry.maxLength) { + entry.addressesOffset = meta.readLong(); + // Old count of uncompressed addresses + long numAddresses = entry.numDocsWithField + 1L; + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift); + entry.addressesLength = meta.readLong(); + } + } else { + if (entry.numDocsWithField > 0 || entry.minLength < entry.maxLength) { + entry.addressesOffset = meta.readLong(); + // New count of compressed addresses - the number of compresseed blocks + int numCompressedChunks = meta.readVInt(); + entry.docsPerChunkShift = meta.readVInt(); + entry.maxUncompressedChunkSize = meta.readVInt(); + + final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks, blockShift); + entry.addressesLength = meta.readLong(); + } } return entry; } @@ -1846,6 +2004,7 @@ static class NumericEntry { } static class BinaryEntry { + final BinaryDVCompressionMode compression; long dataOffset; long dataLength; long docsWithFieldOffset; @@ -1857,7 +2016,14 @@ static class BinaryEntry { int maxLength; long addressesOffset; long addressesLength; + // compression mode + int maxUncompressedChunkSize; + int docsPerChunkShift; DirectMonotonicReader.Meta addressesMeta; + + BinaryEntry(BinaryDVCompressionMode compression) { + this.compression = compression; + } } static class SortedNumericEntry extends NumericEntry { diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java index ee9351ed51b97..2eee88c414733 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.index.codec.Elasticsearch92Lucene103Codec; import org.elasticsearch.index.codec.tsdb.ES87TSDBDocValuesFormatTests.TestES87TSDBDocValuesFormat; import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat; +import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormatTests; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -61,7 +62,8 @@ public void testDuel() throws IOException { ? new ES819TSDBDocValuesFormat( ESTestCase.randomIntBetween(1, 4096), ESTestCase.randomIntBetween(1, 512), - random().nextBoolean() + random().nextBoolean(), + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() ) : new TestES87TSDBDocValuesFormat(); diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index d2c8aae601977..d07175822be81 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.codec.perfield.XPerFieldDocValuesFormat; import org.elasticsearch.index.codec.tsdb.ES87TSDBDocValuesFormatTests.TestES87TSDBDocValuesFormat; import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat; +import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormatTests; import org.elasticsearch.test.ESTestCase; import org.hamcrest.Matchers; @@ -291,7 +292,8 @@ public void testEncodeOrdinalRange() throws IOException { new ES819TSDBDocValuesFormat( random().nextInt(16, 128), nextOrdinalRangeThreshold.getAsInt(), - random().nextBoolean() + random().nextBoolean(), + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() ) ) ); diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 003124ab4b6f4..1b97d7f341586 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.index.codec.Elasticsearch900Lucene101Codec; import org.elasticsearch.index.codec.Elasticsearch92Lucene103Codec; +import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.ES87TSDBDocValuesFormatTests; import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesProducer.BaseDenseNumericValues; import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesProducer.BaseSortedDocValues; @@ -73,7 +74,8 @@ public class ES819TSDBDocValuesFormatTests extends ES87TSDBDocValuesFormatTests final ES819TSDBDocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat( ESTestCase.randomIntBetween(2, 4096), ESTestCase.randomIntBetween(1, 512), - random().nextBoolean() + random().nextBoolean(), + randomBinaryCompressionMode() ); @Override @@ -1541,4 +1543,9 @@ private IndexWriterConfig getTimeSeriesIndexWriterConfig(String hostnameField, b return config; } + public static BinaryDVCompressionMode randomBinaryCompressionMode() { + BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); + return modes[random().nextInt(modes.length)]; + } + } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java index 247b75f2977b5..b607e68ea755a 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java @@ -19,14 +19,14 @@ public class ES819TSDBDocValuesFormatVariableSkipIntervalTests extends ES87TSDBD protected Codec getCodec() { // small interval size to test with many intervals return TestUtil.alwaysDocValuesFormat( - new ES819TSDBDocValuesFormat(random().nextInt(4, 16), random().nextInt(1, 32), random().nextBoolean()) + new ES819TSDBDocValuesFormat(random().nextInt(4, 16), random().nextInt(1, 32), random().nextBoolean(), ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode()) ); } public void testSkipIndexIntervalSize() { IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> new ES819TSDBDocValuesFormat(random().nextInt(Integer.MIN_VALUE, 2), random().nextInt(1, 32), random().nextBoolean()) + () -> new ES819TSDBDocValuesFormat(random().nextInt(Integer.MIN_VALUE, 2), random().nextInt(1, 32), random().nextBoolean(), ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode()) ); assertTrue(ex.getMessage().contains("skipIndexIntervalSize must be > 1")); } From a97371338d3ab2cc2746a6a03bb4da64f979e1ad Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 23 Oct 2025 13:54:13 -0500 Subject: [PATCH 02/64] Initial version of block withs variable number values --- .../es819/ES819TSDBDocValuesConsumer.java | 265 +++++++++++------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 11 +- .../es819/ES819TSDBDocValuesProducer.java | 197 ++++++++----- .../es819/ES819TSDBDocValuesFormatTests.java | 5 +- 4 files changed, 308 insertions(+), 170 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 94177f655ded3..4890309fa6a4f 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -32,10 +32,12 @@ import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.LongsRef; @@ -70,8 +72,8 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { private final int minDocsPerOrdinalForOrdinalRangeEncoding; final boolean enableOptimizedMerge; private final int primarySortFieldNumber; - private final SegmentWriteState state; - private final BinaryDVCompressionMode binaryDVCompressionMode; + final SegmentWriteState state; + final BinaryDVCompressionMode binaryDVCompressionMode; ES819TSDBDocValuesConsumer( BinaryDVCompressionMode binaryDVCompressionMode, @@ -332,7 +334,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th } } - private void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + public void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { if (valuesProducer instanceof TsdbDocValuesProducer tsdbValuesProducer && tsdbValuesProducer.mergeStats.supported()) { final int numDocsWithField = tsdbValuesProducer.mergeStats.sumNumDocsWithField(); final int minLength = tsdbValuesProducer.mergeStats.minLength(); @@ -461,8 +463,7 @@ private void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesPr } } - // BEGIN: Copied fom LUCENE-9211 - private void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) { BinaryDocValues values = valuesProducer.getBinary(field); long start = data.getFilePointer(); @@ -511,52 +512,89 @@ private void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProd } } - static final int BINARY_BLOCK_SHIFT = 5; - static final int BINARY_DOCS_PER_COMPRESSED_BLOCK = 1 << BINARY_BLOCK_SHIFT; + static final int MIN_BLOCK_BYTES = 256 * 1024; + static final int START_BLOCK_DOCS = 1024; // likely needs to grow private class CompressedBinaryBlockWriter implements Closeable { final LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; - final int[] docLengths = new int[BINARY_DOCS_PER_COMPRESSED_BLOCK]; + byte[] docLengths = new byte[START_BLOCK_DOCS * Integer.BYTES]; // really want ints, but need 0 copy way to convert to ByteBuffer byte[] block = BytesRef.EMPTY_BYTES; int totalChunks = 0; long maxPointer = 0; final long blockAddressesStart; + int maxDocInBlock = -1; + int maxNumDocsInAnyBlock = 0; + final IndexOutput tempBinaryOffsets; + final IndexOutput tempDocRanges; CompressedBinaryBlockWriter() throws IOException { - tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( - state.directory, - state.segmentInfo.name, - "binary_pointers", - state.context - ); - boolean success = false; - try { - CodecUtil.writeHeader( - tempBinaryOffsets, - ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", - ES819TSDBDocValuesFormat.VERSION_CURRENT + { + tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( + state.directory, + state.segmentInfo.name, + "binary_pointers", + state.context ); - blockAddressesStart = data.getFilePointer(); - success = true; - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + boolean success = false; + try { + CodecUtil.writeHeader( + tempBinaryOffsets, + ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + blockAddressesStart = data.getFilePointer(); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } + } + + { + tempDocRanges = EndiannessReverserUtil.createTempOutput( + state.directory, + state.segmentInfo.name, + "doc_ranges", + state.context + ); + boolean success = false; + try { + CodecUtil.writeHeader( + tempDocRanges, + ES819TSDBDocValuesFormat.META_CODEC + "DocRanges", + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } } } } - void addDoc(int doc, BytesRef v) throws IOException { - docLengths[numDocsInCurrentBlock] = v.length; + /** + * Confusingly we do not use doc. This is because docId may not be dense. + * But we can guarantee that the lookup value is dense on the range of inserted values. + */ + void addDoc(int _docId, BytesRef v) throws IOException { + docLengths = ArrayUtil.grow(docLengths, (numDocsInCurrentBlock + 1) * Integer.BYTES); + BitUtil.VH_LE_INT.set(docLengths, numDocsInCurrentBlock * Integer.BYTES, v.length); + block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; + maxDocInBlock++; numDocsInCurrentBlock++; - if (numDocsInCurrentBlock == BINARY_DOCS_PER_COMPRESSED_BLOCK) { + +// int totalUncompressedSize = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; + if (uncompressedBlockLength > MIN_BLOCK_BYTES) { flushData(); } } @@ -567,34 +605,21 @@ private void flushData() throws IOException { totalChunks++; long thisBlockStartPointer = data.getFilePointer(); - // Optimisation - check if all lengths are same - boolean allLengthsSame = true; - for (int i = 1; i < BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { - if (docLengths[i] != docLengths[i - 1]) { - allLengthsSame = false; - break; - } - } - if (allLengthsSame) { - // Only write one value shifted. Steal a bit to indicate all other lengths are the same - int onlyOneLength = (docLengths[0] << 1) | 1; - data.writeVInt(onlyOneLength); - } else { - for (int i = 0; i < BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) { - if (i == 0) { - // Write first value shifted and steal a bit to indicate other lengths are to follow - int multipleLengths = (docLengths[0] << 1); - data.writeVInt(multipleLengths); - } else { - data.writeVInt(docLengths[i]); - } - } - } + // write length of string data + data.writeInt(uncompressedBlockLength); + maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); - LZ4.compress(block, 0, uncompressedBlockLength, EndiannessReverserUtil.wrapDataOutput(data), ht); + maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); + + DataOutput output = EndiannessReverserUtil.wrapDataOutput(data); + LZ4.compress(docLengths, 0, numDocsInCurrentBlock * Integer.BYTES, output, ht); + LZ4.compress(block, 0, uncompressedBlockLength, output, ht); + + int minDocInBlock = maxDocInBlock - numDocsInCurrentBlock + 1; + tempDocRanges.writeVInt(minDocInBlock); + tempDocRanges.writeVInt(maxDocInBlock); + numDocsInCurrentBlock = 0; - // Ensure initialized with zeroes because full array is always written - Arrays.fill(docLengths, 0); uncompressedBlockLength = 0; maxPointer = data.getFilePointer(); tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer); @@ -607,57 +632,104 @@ void writeMetaData() throws IOException { } long startDMW = data.getFilePointer(); - meta.writeLong(startDMW); + meta.writeLong(startDMW); // this is where the block pointer data will start - meta.writeVInt(totalChunks); - meta.writeVInt(BINARY_BLOCK_SHIFT); + meta.writeVInt(totalChunks); // numCompressedChunks meta.writeVInt(maxUncompressedBlockLength); + meta.writeVInt(maxNumDocsInAnyBlock); meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - CodecUtil.writeFooter(tempBinaryOffsets); - IOUtils.close(tempBinaryOffsets); - // write the compressed block offsets info to the meta file by reading from temp file - try ( - ChecksumIndexInput filePointersIn = EndiannessReverserUtil.openChecksumInput( - state.directory, - tempBinaryOffsets.getName(), - IOContext.READONCE - ) - ) { - CodecUtil.checkHeader( - filePointersIn, - ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", - ES819TSDBDocValuesFormat.VERSION_CURRENT, - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - Throwable priorE = null; - try { - final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance( - meta, - data, - totalChunks, - ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + { + CodecUtil.writeFooter(tempBinaryOffsets); + IOUtils.close(tempBinaryOffsets); + // write the compressed block offsets info to the meta file by reading from temp file + try ( + ChecksumIndexInput filePointersIn = EndiannessReverserUtil.openChecksumInput( + state.directory, + tempBinaryOffsets.getName(), + IOContext.READONCE + ) + ) { + CodecUtil.checkHeader( + filePointersIn, + ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", + ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_CURRENT ); - long fp = blockAddressesStart; - for (int i = 0; i < totalChunks; ++i) { - filePointers.add(fp); - fp += filePointersIn.readVLong(); - } - if (maxPointer < fp) { - throw new CorruptIndexException( - "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")", - filePointersIn + Throwable priorE = null; + try { + final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance( + meta, + data, + totalChunks, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT ); + long fp = blockAddressesStart; + for (int i = 0; i < totalChunks; ++i) { + filePointers.add(fp); + fp += filePointersIn.readVLong(); + } + if (maxPointer < fp) { + throw new CorruptIndexException( + "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")", + filePointersIn + ); + } + filePointers.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(filePointersIn, priorE); } - filePointers.finish(); - } catch (Throwable e) { - priorE = e; - } finally { - CodecUtil.checkFooter(filePointersIn, priorE); } } - // Write the length of the DMW block in the data + + // Write the length of the block point block in the data meta.writeLong(data.getFilePointer() - startDMW); + + long startDocRanges = data.getFilePointer(); + meta.writeLong(startDocRanges); + { + CodecUtil.writeFooter(tempDocRanges); + IOUtils.close(tempDocRanges); + try ( + ChecksumIndexInput docRangesIn = EndiannessReverserUtil.openChecksumInput( + state.directory, + tempDocRanges.getName(), + IOContext.READONCE + ) + ) { + CodecUtil.checkHeader( + docRangesIn, + ES819TSDBDocValuesFormat.META_CODEC + "DocRanges", + ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + Throwable priorE = null; + try { + long numDocRangeBounds = totalChunks * 2L; + final DirectMonotonicWriter docRanges = DirectMonotonicWriter.getInstance( + meta, + data, + numDocRangeBounds, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + ); + long bound = 0; + for (int i = 0; i < numDocRangeBounds; ++i) { + bound = docRangesIn.readVInt(); + docRanges.add(bound); + } + docRanges.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(docRangesIn, priorE); + } + } + } + + long lenDocRanges = data.getFilePointer() - startDocRanges; + meta.writeLong(lenDocRanges); } @Override @@ -665,6 +737,9 @@ public void close() throws IOException { if (tempBinaryOffsets != null) { IOUtils.close(tempBinaryOffsets, () -> state.directory.deleteFile(tempBinaryOffsets.getName())); } + if (tempDocRanges != null) { + IOUtils.close(tempDocRanges, () -> state.directory.deleteFile(tempDocRanges.getName())); + } } } // END: Copied fom LUCENE-9211 diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index bd2865df2f1e2..3c530bf1a9e72 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -48,7 +48,7 @@ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValues static final byte SORTED_NUMERIC = 4; static final int VERSION_START = 0; - static final int VERSION_BINARY_DV_COMPRESSION = VERSION_START; + static final int VERSION_BINARY_DV_COMPRESSION = 1; static final int VERSION_CURRENT = VERSION_BINARY_DV_COMPRESSION; static final int TERMS_DICT_BLOCK_LZ4_SHIFT = 6; @@ -125,11 +125,16 @@ private static boolean getOptimizedMergeEnabledDefault() { /** Default constructor. */ public ES819TSDBDocValuesFormat() { - this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.NO_COMPRESS); + this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.COMPRESSED_WITH_LZ4); } /** Doc values fields format with specified skipIndexIntervalSize. */ - public ES819TSDBDocValuesFormat(int skipIndexIntervalSize, int minDocsPerOrdinalForRangeEncoding, boolean enableOptimizedMerge, BinaryDVCompressionMode binaryDVCompressionMode) { + public ES819TSDBDocValuesFormat( + int skipIndexIntervalSize, + int minDocsPerOrdinalForRangeEncoding, + boolean enableOptimizedMerge, + BinaryDVCompressionMode binaryDVCompressionMode + ) { super(CODEC_NAME); if (skipIndexIntervalSize < 2) { throw new IllegalArgumentException("skipIndexIntervalSize must be > 1, got [" + skipIndexIntervalSize + "]"); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index cf4cce7bceb6f..3e215e33cd527 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -39,6 +39,7 @@ import org.apache.lucene.store.DataInput; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LongValues; import org.apache.lucene.util.compress.LZ4; @@ -53,8 +54,6 @@ import java.io.IOException; -import static org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode.COMPRESSED_WITH_LZ4; -import static org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode.NO_COMPRESS; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -203,8 +202,7 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { }; } - private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException { - + public BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException { final RandomAccessInput bytesSlice = data.randomAccessSlice(entry.dataOffset, entry.dataLength); if (entry.docsWithFieldOffset == -1) { @@ -285,17 +283,21 @@ private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOExceptio // dense final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + + final RandomAccessInput docRangeData = this.data.randomAccessSlice(entry.docRangeOffset, entry.docRangeLength); + final LongValues docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new DenseBinaryDocValues(maxDoc) { final BinaryDecoder decoder = new BinaryDecoder( addresses, + docRanges, data.clone(), entry.maxUncompressedChunkSize, - entry.docsPerChunkShift + entry.maxNumDocsInAnyBlock ); @Override public BytesRef binaryValue() throws IOException { - return decoder.decode(doc); + return decoder.decode(doc, entry.numCompressedBlocks); } }; } else { @@ -310,17 +312,21 @@ public BytesRef binaryValue() throws IOException { ); final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength); final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); + + final RandomAccessInput docRangeData = this.data.randomAccessSlice(entry.docRangeOffset, entry.docRangeLength); + final LongValues docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new SparseBinaryDocValues(disi) { final BinaryDecoder decoder = new BinaryDecoder( addresses, + docRanges, data.clone(), entry.maxUncompressedChunkSize, - entry.docsPerChunkShift + entry.maxNumDocsInAnyBlock ); @Override public BytesRef binaryValue() throws IOException { - return decoder.decode(disi.index()); + return decoder.decode(disi.index(), entry.numCompressedBlocks); } }; } @@ -331,80 +337,121 @@ public BytesRef binaryValue() throws IOException { static final class BinaryDecoder { private final LongValues addresses; + private final LongValues docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; + private final byte[] uncompressedDocLengths; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; - private final int docsPerChunk; - private final int docsPerChunkShift; - - BinaryDecoder(LongValues addresses, IndexInput compressedData, int biggestUncompressedBlockSize, int docsPerChunkShift) { + private final long[] offsets; + private long minDocIdInBlock = -1; + private long maxDocIdInBlock = -1; + + BinaryDecoder( + LongValues addresses, + LongValues docRanges, + IndexInput compressedData, + int biggestUncompressedBlockSize, + int maxNumDocsInAnyBlock + ) { super(); this.addresses = addresses; + this.docRanges = docRanges; this.compressedData = compressedData; // pre-allocate a byte array large enough for the biggest uncompressed block needed. this.uncompressedBlock = new byte[biggestUncompressedBlockSize]; uncompressedBytesRef = new BytesRef(uncompressedBlock); - this.docsPerChunk = 1 << docsPerChunkShift; - this.docsPerChunkShift = docsPerChunkShift; - uncompressedDocStarts = new int[docsPerChunk + 1]; + uncompressedDocLengths = new byte[(maxNumDocsInAnyBlock + 1) * Integer.BYTES]; + uncompressedDocStarts = new int[maxNumDocsInAnyBlock + 1]; + offsets = new long[maxNumDocsInAnyBlock + 1]; } - BytesRef decode(int docNumber) throws IOException { - int blockId = docNumber >> docsPerChunkShift; - int docInBlockId = docNumber % docsPerChunk; - assert docInBlockId < docsPerChunk; + // unconditionally decompress blockId into + // uncompressedDocStarts + // uncompressedBlck + private void decompressBlock(int blockId, int numDocsInBlock) throws IOException { + long blockStartOffset = addresses.get(blockId); + compressedData.seek(blockStartOffset); - // already read and uncompressed? - if (blockId != lastBlockId) { - lastBlockId = blockId; - long blockStartOffset = addresses.get(blockId); - compressedData.seek(blockStartOffset); - - int uncompressedBlockLength = 0; - - int onlyLength = -1; - for (int i = 0; i < docsPerChunk; i++) { - if (i == 0) { - // The first length value is special. It is shifted and has a bit to denote if - // all other values are the same length - int lengthPlusSameInd = compressedData.readVInt(); - int sameIndicator = lengthPlusSameInd & 1; - int firstValLength = lengthPlusSameInd >>> 1; - if (sameIndicator == 1) { - onlyLength = firstValLength; - } - uncompressedBlockLength += firstValLength; - } else { - if (onlyLength == -1) { - // Various lengths are stored - read each from disk - uncompressedBlockLength += compressedData.readVInt(); - } else { - // Only one length - uncompressedBlockLength += onlyLength; - } - } - uncompressedDocStarts[i + 1] = uncompressedBlockLength; - } + int uncompressedBlockLength = compressedData.readInt(); + + if (uncompressedBlockLength == 0) { + return; + } + + DataInput input = EndiannessReverserUtil.wrapDataInput(compressedData); + + int offsetBytesLen = numDocsInBlock * Integer.BYTES; + assert offsetBytesLen <= uncompressedDocLengths.length; + BytesRef offsetOut = new BytesRef(uncompressedDocLengths, 0, offsetBytesLen); + LZ4.decompress(input, offsetBytesLen, offsetOut.bytes, offsetOut.offset); + + int docStart = 0; + for (int i = 0; i < numDocsInBlock; i++) { + int len = getOffsetDocStart(i); + docStart += len; + uncompressedDocStarts[i+1] = docStart; + } + + assert uncompressedBlockLength <= uncompressedBlock.length; + BytesRef dataOut = new BytesRef(uncompressedBlock, 0, uncompressedBlockLength); + LZ4.decompress(input, uncompressedBlockLength, dataOut.bytes, dataOut.offset); + } + + // Find range containing docId that is within or after lastBlockId + // Could change to binary search, though since we usually scan forward this would probably be slower + long getBlockContainingDoc(LongValues docRanges, long lastBlockId, int docNumber, int numBlocks) { + long blockId = lastBlockId + 1; + + while (blockId < numBlocks) { + minDocIdInBlock = docRanges.get(2L * blockId); + maxDocIdInBlock = docRanges.get(2L * blockId + 1); - if (uncompressedBlockLength == 0) { - uncompressedBytesRef.offset = 0; - uncompressedBytesRef.length = 0; - return uncompressedBytesRef; + if (docNumber < minDocIdInBlock) { + break; } + if (docNumber <= maxDocIdInBlock) { + return blockId; + } + blockId++; + } + return -1; + } + + private int getOffsetDocStart(int idx) { + return (int) BitUtil.VH_LE_INT.get(uncompressedDocLengths, idx * Integer.BYTES); + } + + static final BytesRef EMPTY_BYTES_REF = new BytesRef(); + BytesRef decode(int docNumber, int numBlocks) throws IOException { + // docNumber because these are dense and could be indices from a DISI + long blockId = docNumber <= maxDocIdInBlock ? lastBlockId : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); - assert uncompressedBlockLength <= uncompressedBlock.length; - LZ4.decompress(EndiannessReverserUtil.wrapDataInput(compressedData), uncompressedBlockLength, uncompressedBlock, 0); + if (blockId < 0) { + return EMPTY_BYTES_REF; } - uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId]; - uncompressedBytesRef.length = uncompressedDocStarts[docInBlockId + 1] - uncompressedBytesRef.offset; + int numDocsInBlock = (int) (maxDocIdInBlock - minDocIdInBlock + 1); + + int idxInBlock = (int) (docNumber - minDocIdInBlock); + assert idxInBlock < numDocsInBlock; + + // already read and uncompressed? + if (blockId != lastBlockId) { + decompressBlock((int) blockId, numDocsInBlock); + // uncompressedBytesRef and uncompressedDocStarts now populated + lastBlockId = blockId; + } + + int start = uncompressedDocStarts[idxInBlock]; + int end = uncompressedDocStarts[idxInBlock + 1]; + uncompressedBytesRef.offset = start; + uncompressedBytesRef.length = end - start; return uncompressedBytesRef; } } - // END: Copied fom LUCENE-9211 private abstract static class DenseBinaryDocValues extends BinaryDocValues { @@ -1333,12 +1380,18 @@ private BinaryEntry readBinary(IndexInput meta, int version) throws IOException entry.addressesOffset = meta.readLong(); // New count of compressed addresses - the number of compresseed blocks int numCompressedChunks = meta.readVInt(); - entry.docsPerChunkShift = meta.readVInt(); entry.maxUncompressedChunkSize = meta.readVInt(); - + entry.maxNumDocsInAnyBlock = meta.readVInt(); final int blockShift = meta.readVInt(); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks, blockShift); entry.addressesLength = meta.readLong(); + + entry.docRangeOffset = meta.readLong(); + entry.docRangeMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks * 2L, blockShift); + entry.docRangeLength = meta.readLong(); + + entry.numCompressedBlocks = numCompressedChunks; } } return entry; @@ -1638,14 +1691,6 @@ long lookAheadValueAt(int targetDoc) throws IOException { } return lookaheadBlock[valueIndex]; } - - static boolean isDense(int firstDocId, int lastDocId, int length) { - // This does not detect duplicate docids (e.g [1, 1, 2, 4] would be detected as dense), - // this can happen with enrich or lookup. However this codec isn't used for enrich / lookup. - // This codec is only used in the context of logsdb and tsdb, so this is fine here. - return lastDocId - firstDocId == length - 1; - } - }; } else { final IndexedDISI disi = new IndexedDISI( @@ -1760,6 +1805,13 @@ public BlockLoader.Block tryRead( } } + static boolean isDense(int firstDocId, int lastDocId, int length) { + // This does not detect duplicate docids (e.g [1, 1, 2, 4] would be detected as dense), + // this can happen with enrich or lookup. However this codec isn't used for enrich / lookup. + // This codec is only used in the context of logsdb and tsdb, so this is fine here. + return lastDocId - firstDocId == length - 1; + } + private NumericDocValues getRangeEncodedNumericDocValues(NumericEntry entry, long maxOrd) throws IOException { final var ordinalsReader = new SortedOrdinalReader( maxOrd, @@ -2005,6 +2057,7 @@ static class NumericEntry { static class BinaryEntry { final BinaryDVCompressionMode compression; + long dataOffset; long dataLength; long docsWithFieldOffset; @@ -2016,10 +2069,14 @@ static class BinaryEntry { int maxLength; long addressesOffset; long addressesLength; + long docRangeOffset; + long docRangeLength; // compression mode int maxUncompressedChunkSize; - int docsPerChunkShift; + int maxNumDocsInAnyBlock; + int numCompressedBlocks; DirectMonotonicReader.Meta addressesMeta; + DirectMonotonicReader.Meta docRangeMeta; BinaryEntry(BinaryDVCompressionMode compression) { this.compression = compression; diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 1b97d7f341586..f6f5f9fecf870 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -1544,8 +1544,9 @@ private IndexWriterConfig getTimeSeriesIndexWriterConfig(String hostnameField, b } public static BinaryDVCompressionMode randomBinaryCompressionMode() { - BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); - return modes[random().nextInt(modes.length)]; +// BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); +// return modes[random().nextInt(modes.length)]; + return BinaryDVCompressionMode.COMPRESSED_WITH_LZ4; } } From 3fc95dc56772a48f8c2dc9cb521b7b42fb4d9754 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 23 Oct 2025 15:43:15 -0500 Subject: [PATCH 03/64] Fix issue with index output unclosed --- .../es819/ES819TSDBDocValuesConsumer.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 4890309fa6a4f..722874ab7956e 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -534,14 +534,14 @@ private class CompressedBinaryBlockWriter implements Closeable { CompressedBinaryBlockWriter() throws IOException { { - tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( - state.directory, - state.segmentInfo.name, - "binary_pointers", - state.context - ); boolean success = false; try { + tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( + state.directory, + state.segmentInfo.name, + "binary_pointers", + state.context + ); CodecUtil.writeHeader( tempBinaryOffsets, ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", @@ -557,14 +557,14 @@ private class CompressedBinaryBlockWriter implements Closeable { } { - tempDocRanges = EndiannessReverserUtil.createTempOutput( - state.directory, - state.segmentInfo.name, - "doc_ranges", - state.context - ); boolean success = false; try { + tempDocRanges = EndiannessReverserUtil.createTempOutput( + state.directory, + state.segmentInfo.name, + "doc_ranges", + state.context + ); CodecUtil.writeHeader( tempDocRanges, ES819TSDBDocValuesFormat.META_CODEC + "DocRanges", From c302cc26bc04f1316eb49fa4d5e10ecb13e8d4e6 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 23 Oct 2025 17:00:55 -0500 Subject: [PATCH 04/64] Changes docRanges to single limit per block, plus start of 0 --- .../es819/ES819TSDBDocValuesConsumer.java | 27 ++++++++------- .../es819/ES819TSDBDocValuesProducer.java | 34 ++++++------------- 2 files changed, 24 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 722874ab7956e..dec5c2e1b629a 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -615,14 +615,13 @@ private void flushData() throws IOException { LZ4.compress(docLengths, 0, numDocsInCurrentBlock * Integer.BYTES, output, ht); LZ4.compress(block, 0, uncompressedBlockLength, output, ht); - int minDocInBlock = maxDocInBlock - numDocsInCurrentBlock + 1; - tempDocRanges.writeVInt(minDocInBlock); - tempDocRanges.writeVInt(maxDocInBlock); - + tempDocRanges.writeVInt(numDocsInCurrentBlock); numDocsInCurrentBlock = 0; + uncompressedBlockLength = 0; maxPointer = data.getFilePointer(); - tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer); + long blockLenBytes = maxPointer - thisBlockStartPointer; + tempBinaryOffsets.writeVLong(blockLenBytes); } } @@ -661,13 +660,14 @@ void writeMetaData() throws IOException { final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance( meta, data, - totalChunks, + totalChunks + 1, ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT ); long fp = blockAddressesStart; + filePointers.add(fp); for (int i = 0; i < totalChunks; ++i) { - filePointers.add(fp); fp += filePointersIn.readVLong(); + filePointers.add(fp); } if (maxPointer < fp) { throw new CorruptIndexException( @@ -707,17 +707,18 @@ void writeMetaData() throws IOException { ); Throwable priorE = null; try { - long numDocRangeBounds = totalChunks * 2L; final DirectMonotonicWriter docRanges = DirectMonotonicWriter.getInstance( meta, data, - numDocRangeBounds, + totalChunks + 1, ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT ); - long bound = 0; - for (int i = 0; i < numDocRangeBounds; ++i) { - bound = docRangesIn.readVInt(); - docRanges.add(bound); + + long docOffset = 0; + docRanges.add(docOffset); + for (int i = 0; i < totalChunks; ++i) { + docOffset += docRangesIn.readVLong(); + docRanges.add(docOffset); } docRanges.finish(); } catch (Throwable e) { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 3e215e33cd527..eb9c09a1f16e3 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -345,9 +345,8 @@ static final class BinaryDecoder { private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; - private final long[] offsets; - private long minDocIdInBlock = -1; - private long maxDocIdInBlock = -1; + private long startDocNumForBlock = -1; + private long limitDocNumForBlock = -1; BinaryDecoder( LongValues addresses, @@ -356,7 +355,6 @@ static final class BinaryDecoder { int biggestUncompressedBlockSize, int maxNumDocsInAnyBlock ) { - super(); this.addresses = addresses; this.docRanges = docRanges; this.compressedData = compressedData; @@ -365,7 +363,6 @@ static final class BinaryDecoder { uncompressedBytesRef = new BytesRef(uncompressedBlock); uncompressedDocLengths = new byte[(maxNumDocsInAnyBlock + 1) * Integer.BYTES]; uncompressedDocStarts = new int[maxNumDocsInAnyBlock + 1]; - offsets = new long[maxNumDocsInAnyBlock + 1]; } // unconditionally decompress blockId into @@ -403,19 +400,12 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException // Find range containing docId that is within or after lastBlockId // Could change to binary search, though since we usually scan forward this would probably be slower long getBlockContainingDoc(LongValues docRanges, long lastBlockId, int docNumber, int numBlocks) { - long blockId = lastBlockId + 1; - - while (blockId < numBlocks) { - minDocIdInBlock = docRanges.get(2L * blockId); - maxDocIdInBlock = docRanges.get(2L * blockId + 1); - - if (docNumber < minDocIdInBlock) { - break; - } - if (docNumber <= maxDocIdInBlock) { + for (long blockId = lastBlockId + 1; blockId < numBlocks; blockId++) { + startDocNumForBlock = docRanges.get(blockId); + limitDocNumForBlock = docRanges.get(blockId + 1); + if (docNumber < limitDocNumForBlock) { return blockId; } - blockId++; } return -1; } @@ -427,15 +417,11 @@ private int getOffsetDocStart(int idx) { static final BytesRef EMPTY_BYTES_REF = new BytesRef(); BytesRef decode(int docNumber, int numBlocks) throws IOException { // docNumber because these are dense and could be indices from a DISI - long blockId = docNumber <= maxDocIdInBlock ? lastBlockId : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); - - if (blockId < 0) { - return EMPTY_BYTES_REF; - } - - int numDocsInBlock = (int) (maxDocIdInBlock - minDocIdInBlock + 1); + long blockId = docNumber < limitDocNumForBlock ? lastBlockId : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); + assert blockId >= 0; - int idxInBlock = (int) (docNumber - minDocIdInBlock); + int numDocsInBlock = (int) (limitDocNumForBlock - startDocNumForBlock); + int idxInBlock = (int) (docNumber - startDocNumForBlock); assert idxInBlock < numDocsInBlock; // already read and uncompressed? From 99748c8d833b175ac992bd99e530415cf9ec365f Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 23 Oct 2025 18:44:26 -0500 Subject: [PATCH 05/64] Factor block address and block doc offset to accumulator class --- .../es819/ES819TSDBDocValuesConsumer.java | 202 ++++-------------- .../es819/ES819TSDBDocValuesProducer.java | 1 - .../OffsetsAccumulatorUnknownLength.java | 111 ++++++++++ 3 files changed, 154 insertions(+), 160 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index dec5c2e1b629a..4ca61c31e5c70 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -14,7 +14,6 @@ import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene90.IndexedDISI; import org.apache.lucene.index.BinaryDocValues; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValuesSkipIndexType; import org.apache.lucene.index.FieldInfo; @@ -31,7 +30,6 @@ import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; -import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -512,75 +510,51 @@ public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProdu } } - static final int MIN_BLOCK_BYTES = 256 * 1024; - static final int START_BLOCK_DOCS = 1024; // likely needs to grow - private class CompressedBinaryBlockWriter implements Closeable { + static final int MIN_BLOCK_BYTES = 256 * 1024; + static final int START_BLOCK_DOCS = 1024; // likely needs to grow + final LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; - byte[] docLengths = new byte[START_BLOCK_DOCS * Integer.BYTES]; // really want ints, but need 0 copy way to convert to ByteBuffer + + // really want ints, but need zero-copy way to convert to ByteBuffer + byte[] docLengths = new byte[START_BLOCK_DOCS * Integer.BYTES]; byte[] block = BytesRef.EMPTY_BYTES; int totalChunks = 0; long maxPointer = 0; - final long blockAddressesStart; - - int maxDocInBlock = -1; int maxNumDocsInAnyBlock = 0; - final IndexOutput tempBinaryOffsets; - final IndexOutput tempDocRanges; + final OffsetsAccumulatorUnknownLength blockAddressAcc; + final OffsetsAccumulatorUnknownLength blockDocRangeAcc; CompressedBinaryBlockWriter() throws IOException { - { - boolean success = false; - try { - tempBinaryOffsets = EndiannessReverserUtil.createTempOutput( - state.directory, - state.segmentInfo.name, - "binary_pointers", - state.context - ); - CodecUtil.writeHeader( - tempBinaryOffsets, - ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - blockAddressesStart = data.getFilePointer(); - success = true; - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't - } - } - } + long blockAddressesStart = data.getFilePointer(); + blockAddressAcc = new OffsetsAccumulatorUnknownLength( + state.directory, + state.context, + data, + "block_addresses", + blockAddressesStart + ); - { - boolean success = false; - try { - tempDocRanges = EndiannessReverserUtil.createTempOutput( - state.directory, - state.segmentInfo.name, - "doc_ranges", - state.context - ); - CodecUtil.writeHeader( - tempDocRanges, - ES819TSDBDocValuesFormat.META_CODEC + "DocRanges", - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - success = true; - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't - } - } + try { + blockDocRangeAcc = new OffsetsAccumulatorUnknownLength( + state.directory, + state.context, + data, + "block_doc_ranges", + 0 + ); + } catch (IOException e) { + blockAddressAcc.close(); + throw e; } } /** - * Confusingly we do not use doc. This is because docId may not be dense. + * _docId is unused. This is because docId may not be dense. * But we can guarantee that the lookup value is dense on the range of inserted values. */ void addDoc(int _docId, BytesRef v) throws IOException { @@ -590,7 +564,6 @@ void addDoc(int _docId, BytesRef v) throws IOException { block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; - maxDocInBlock++; numDocsInCurrentBlock++; // int totalUncompressedSize = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; @@ -615,13 +588,13 @@ private void flushData() throws IOException { LZ4.compress(docLengths, 0, numDocsInCurrentBlock * Integer.BYTES, output, ht); LZ4.compress(block, 0, uncompressedBlockLength, output, ht); - tempDocRanges.writeVInt(numDocsInCurrentBlock); + blockDocRangeAcc.addDoc(numDocsInCurrentBlock); numDocsInCurrentBlock = 0; uncompressedBlockLength = 0; maxPointer = data.getFilePointer(); long blockLenBytes = maxPointer - thisBlockStartPointer; - tempBinaryOffsets.writeVLong(blockLenBytes); + blockAddressAcc.addDoc(blockLenBytes); } } @@ -630,120 +603,31 @@ void writeMetaData() throws IOException { return; } - long startDMW = data.getFilePointer(); - meta.writeLong(startDMW); // this is where the block pointer data will start + long dataAddressesStart = data.getFilePointer(); - meta.writeVInt(totalChunks); // numCompressedChunks + meta.writeLong(dataAddressesStart); + meta.writeVInt(totalChunks); meta.writeVInt(maxUncompressedBlockLength); meta.writeVInt(maxNumDocsInAnyBlock); meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - { - CodecUtil.writeFooter(tempBinaryOffsets); - IOUtils.close(tempBinaryOffsets); - // write the compressed block offsets info to the meta file by reading from temp file - try ( - ChecksumIndexInput filePointersIn = EndiannessReverserUtil.openChecksumInput( - state.directory, - tempBinaryOffsets.getName(), - IOContext.READONCE - ) - ) { - CodecUtil.checkHeader( - filePointersIn, - ES819TSDBDocValuesFormat.META_CODEC + "FilePointers", - ES819TSDBDocValuesFormat.VERSION_CURRENT, - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - Throwable priorE = null; - try { - final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance( - meta, - data, - totalChunks + 1, - ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT - ); - long fp = blockAddressesStart; - filePointers.add(fp); - for (int i = 0; i < totalChunks; ++i) { - fp += filePointersIn.readVLong(); - filePointers.add(fp); - } - if (maxPointer < fp) { - throw new CorruptIndexException( - "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")", - filePointersIn - ); - } - filePointers.finish(); - } catch (Throwable e) { - priorE = e; - } finally { - CodecUtil.checkFooter(filePointersIn, priorE); - } - } - } - - // Write the length of the block point block in the data - meta.writeLong(data.getFilePointer() - startDMW); - - long startDocRanges = data.getFilePointer(); - meta.writeLong(startDocRanges); - { - CodecUtil.writeFooter(tempDocRanges); - IOUtils.close(tempDocRanges); - try ( - ChecksumIndexInput docRangesIn = EndiannessReverserUtil.openChecksumInput( - state.directory, - tempDocRanges.getName(), - IOContext.READONCE - ) - ) { - CodecUtil.checkHeader( - docRangesIn, - ES819TSDBDocValuesFormat.META_CODEC + "DocRanges", - ES819TSDBDocValuesFormat.VERSION_CURRENT, - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - Throwable priorE = null; - try { - final DirectMonotonicWriter docRanges = DirectMonotonicWriter.getInstance( - meta, - data, - totalChunks + 1, - ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT - ); - - long docOffset = 0; - docRanges.add(docOffset); - for (int i = 0; i < totalChunks; ++i) { - docOffset += docRangesIn.readVLong(); - docRanges.add(docOffset); - } - docRanges.finish(); - } catch (Throwable e) { - priorE = e; - } finally { - CodecUtil.checkFooter(docRangesIn, priorE); - } - } - } + blockAddressAcc.build(meta, data); + long dataDocRangeStart = data.getFilePointer(); + long addressesLength = dataDocRangeStart - dataAddressesStart; + meta.writeLong(addressesLength); - long lenDocRanges = data.getFilePointer() - startDocRanges; - meta.writeLong(lenDocRanges); + meta.writeLong(dataDocRangeStart); + blockDocRangeAcc.build(meta, data); + long docRangesLen = data.getFilePointer() - dataAddressesStart; + meta.writeLong(docRangesLen); } @Override public void close() throws IOException { - if (tempBinaryOffsets != null) { - IOUtils.close(tempBinaryOffsets, () -> state.directory.deleteFile(tempBinaryOffsets.getName())); - } - if (tempDocRanges != null) { - IOUtils.close(tempDocRanges, () -> state.directory.deleteFile(tempDocRanges.getName())); - } + blockDocRangeAcc.close(); + blockAddressAcc.close(); } } - // END: Copied fom LUCENE-9211 @Override public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index eb9c09a1f16e3..bffdb987bf525 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -414,7 +414,6 @@ private int getOffsetDocStart(int idx) { return (int) BitUtil.VH_LE_INT.get(uncompressedDocLengths, idx * Integer.BYTES); } - static final BytesRef EMPTY_BYTES_REF = new BytesRef(); BytesRef decode(int docNumber, int numBlocks) throws IOException { // docNumber because these are dense and could be indices from a DISI long blockId = docNumber < limitDocNumForBlock ? lastBlockId : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java new file mode 100644 index 0000000000000..1f87f95413961 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java @@ -0,0 +1,111 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.tsdb.es819; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Like OffsetsAccumulator builds offsets and stores in a DirectMonotonicWriter. But write to temp file + * rather than directly to a DirectMonotonicWriter because the number of values is unknown. If number of + * values if known prefer OffsetsWriter. + */ +final class OffsetsAccumulatorUnknownLength implements Closeable { + private final Directory dir; + private final long startOffset; + + private int numValues = 0; + private final IndexOutput tempOutput; + private final String suffix; + + OffsetsAccumulatorUnknownLength( + Directory dir, + IOContext context, + IndexOutput data, + String suffix, + long startOffset + ) throws IOException { + this.dir = dir; + this.startOffset = startOffset; + this.suffix = suffix; + + boolean success = false; + try { + tempOutput = dir.createTempOutput(data.getName(), suffix, context); + CodecUtil.writeHeader( + tempOutput, + ES819TSDBDocValuesFormat.META_CODEC + suffix, + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + success = true; + } + finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } + } + + public void addDoc(long value) throws IOException { + tempOutput.writeVLong(value); + numValues++; + } + + public void build(IndexOutput meta, IndexOutput data) throws IOException { + CodecUtil.writeFooter(tempOutput); + IOUtils.close(tempOutput); + + // write the offsets info to the meta file by reading from temp file + try (ChecksumIndexInput tempInput = dir.openChecksumInput(tempOutput.getName());) { + CodecUtil.checkHeader( + tempInput, + ES819TSDBDocValuesFormat.META_CODEC + suffix, + ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + Throwable priorE = null; + try { + final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( + meta, + data, + numValues + 1, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + ); + + long offset = startOffset; + writer.add(offset); + for (int i = 0; i < numValues; ++i) { + offset += tempInput.readVLong(); + writer.add(offset); + } + writer.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(tempInput, priorE); + } + } + } + + @Override + public void close() throws IOException { + if (tempOutput != null) { + IOUtils.close(tempOutput, () -> dir.deleteFile(tempOutput.getName())); + } + } +} From fa2ea1169806e7e35181852cd879f2b747329c58 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 10:14:40 -0500 Subject: [PATCH 06/64] Rename offset accumulator --- ...nknownLength.java => DelayedOffsetAccumulator.java} | 8 ++++---- .../codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 10 +++++----- 2 files changed, 9 insertions(+), 9 deletions(-) rename server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/{OffsetsAccumulatorUnknownLength.java => DelayedOffsetAccumulator.java} (94%) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java similarity index 94% rename from server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java rename to server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java index 1f87f95413961..32efb0e2dfd5e 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/OffsetsAccumulatorUnknownLength.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java @@ -25,7 +25,7 @@ * rather than directly to a DirectMonotonicWriter because the number of values is unknown. If number of * values if known prefer OffsetsWriter. */ -final class OffsetsAccumulatorUnknownLength implements Closeable { +final class DelayedOffsetAccumulator implements Closeable { private final Directory dir; private final long startOffset; @@ -33,7 +33,7 @@ final class OffsetsAccumulatorUnknownLength implements Closeable { private final IndexOutput tempOutput; private final String suffix; - OffsetsAccumulatorUnknownLength( + DelayedOffsetAccumulator( Directory dir, IOContext context, IndexOutput data, @@ -61,8 +61,8 @@ final class OffsetsAccumulatorUnknownLength implements Closeable { } } - public void addDoc(long value) throws IOException { - tempOutput.writeVLong(value); + public void addDoc(long delta) throws IOException { + tempOutput.writeVLong(delta); numValues++; } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 4ca61c31e5c70..e066ddfb7890a 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -519,19 +519,19 @@ private class CompressedBinaryBlockWriter implements Closeable { int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; - // really want ints, but need zero-copy way to convert to ByteBuffer + // Store ints, but use byte[] so can be zero-copy converted to ByteBuffer byte[] docLengths = new byte[START_BLOCK_DOCS * Integer.BYTES]; byte[] block = BytesRef.EMPTY_BYTES; int totalChunks = 0; long maxPointer = 0; int maxNumDocsInAnyBlock = 0; - final OffsetsAccumulatorUnknownLength blockAddressAcc; - final OffsetsAccumulatorUnknownLength blockDocRangeAcc; + final DelayedOffsetAccumulator blockAddressAcc; + final DelayedOffsetAccumulator blockDocRangeAcc; CompressedBinaryBlockWriter() throws IOException { long blockAddressesStart = data.getFilePointer(); - blockAddressAcc = new OffsetsAccumulatorUnknownLength( + blockAddressAcc = new DelayedOffsetAccumulator( state.directory, state.context, data, @@ -540,7 +540,7 @@ private class CompressedBinaryBlockWriter implements Closeable { ); try { - blockDocRangeAcc = new OffsetsAccumulatorUnknownLength( + blockDocRangeAcc = new DelayedOffsetAccumulator( state.directory, state.context, data, From b67dd580f43f2714a327f204623ace010837a374 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 10:37:53 -0500 Subject: [PATCH 07/64] Change lz4 to zstd --- .../codec/tsdb/BinaryDVCompressionMode.java | 4 ++-- .../es819/ES819TSDBDocValuesConsumer.java | 20 +++++++++++++------ .../tsdb/es819/ES819TSDBDocValuesFormat.java | 2 +- .../es819/ES819TSDBDocValuesProducer.java | 17 ++++++++-------- .../codec/zstd/Zstd814StoredFieldsFormat.java | 8 ++++---- .../es819/ES819TSDBDocValuesFormatTests.java | 2 +- 6 files changed, 31 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 534c8117eaae0..0be015de97521 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -12,7 +12,7 @@ public enum BinaryDVCompressionMode { NO_COMPRESS((byte) 0), - COMPRESSED_WITH_LZ4((byte) 1); + COMPRESSED_WITH_ZSTD((byte) 1); public final byte code; @@ -23,7 +23,7 @@ public enum BinaryDVCompressionMode { public static BinaryDVCompressionMode fromMode(byte mode) { return switch (mode) { case 0 -> NO_COMPRESS; - case 1 -> COMPRESSED_WITH_LZ4; + case 1 -> COMPRESSED_WITH_ZSTD; default -> throw new IllegalStateException("unknown compression mode [" + mode + "]"); }; } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index e066ddfb7890a..31feba055d257 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -9,7 +9,6 @@ package org.elasticsearch.index.codec.tsdb.es819; -import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene90.IndexedDISI; @@ -28,6 +27,7 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.SortedSetSelector; import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; import org.apache.lucene.store.DataOutput; @@ -46,9 +46,11 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; +import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat; import java.io.Closeable; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -328,7 +330,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th meta.writeByte(binaryDVCompressionMode.code); switch (binaryDVCompressionMode) { case NO_COMPRESS -> doAddUncompressedBinary(field, valuesProducer); - case COMPRESSED_WITH_LZ4 -> doAddCompressedBinary(field, valuesProducer); + case COMPRESSED_WITH_ZSTD -> doAddCompressedBinary(field, valuesProducer); } } @@ -513,8 +515,9 @@ public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProdu private class CompressedBinaryBlockWriter implements Closeable { static final int MIN_BLOCK_BYTES = 256 * 1024; static final int START_BLOCK_DOCS = 1024; // likely needs to grow + private static final int ZSTD_LEVEL = 1; - final LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable(); + private final Zstd814StoredFieldsFormat.ZstdCompressor compressor = new Zstd814StoredFieldsFormat.ZstdCompressor(ZSTD_LEVEL); int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; @@ -584,9 +587,8 @@ private void flushData() throws IOException { maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); - DataOutput output = EndiannessReverserUtil.wrapDataOutput(data); - LZ4.compress(docLengths, 0, numDocsInCurrentBlock * Integer.BYTES, output, ht); - LZ4.compress(block, 0, uncompressedBlockLength, output, ht); + compress(docLengths, numDocsInCurrentBlock * Integer.BYTES, data); + compress(block, uncompressedBlockLength, data); blockDocRangeAcc.addDoc(numDocsInCurrentBlock); numDocsInCurrentBlock = 0; @@ -598,6 +600,12 @@ private void flushData() throws IOException { } } + void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { + ByteBuffer inputBuffer = ByteBuffer.wrap(data, 0, uncompressedLength); + ByteBuffersDataInput input = new ByteBuffersDataInput(List.of(inputBuffer)); + compressor.compress(input, output); + } + void writeMetaData() throws IOException { if (totalChunks == 0) { return; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 3c530bf1a9e72..779dc38893861 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -125,7 +125,7 @@ private static boolean getOptimizedMergeEnabledDefault() { /** Default constructor. */ public ES819TSDBDocValuesFormat() { - this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.COMPRESSED_WITH_LZ4); + this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD); } /** Doc values fields format with specified skipIndexIntervalSize. */ diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index bffdb987bf525..892975cad40a1 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -9,7 +9,6 @@ package org.elasticsearch.index.codec.tsdb.es819; -import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene90.IndexedDISI; @@ -49,6 +48,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; +import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat; import org.elasticsearch.index.mapper.BlockDocValuesReader; import org.elasticsearch.index.mapper.BlockLoader; @@ -198,7 +198,7 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return switch (entry.compression) { case NO_COMPRESS -> getUncompressedBinary(entry); - case COMPRESSED_WITH_LZ4 -> getCompressedBinary(entry); + case COMPRESSED_WITH_ZSTD -> getCompressedBinary(entry); }; } @@ -347,6 +347,7 @@ static final class BinaryDecoder { private final BytesRef uncompressedBytesRef; private long startDocNumForBlock = -1; private long limitDocNumForBlock = -1; + private final Zstd814StoredFieldsFormat.ZstdDecompressor decompressor = new Zstd814StoredFieldsFormat.ZstdDecompressor(); BinaryDecoder( LongValues addresses, @@ -367,7 +368,7 @@ static final class BinaryDecoder { // unconditionally decompress blockId into // uncompressedDocStarts - // uncompressedBlck + // uncompressedBlock private void decompressBlock(int blockId, int numDocsInBlock) throws IOException { long blockStartOffset = addresses.get(blockId); compressedData.seek(blockStartOffset); @@ -378,12 +379,11 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - DataInput input = EndiannessReverserUtil.wrapDataInput(compressedData); - + DataInput input = compressedData; int offsetBytesLen = numDocsInBlock * Integer.BYTES; assert offsetBytesLen <= uncompressedDocLengths.length; BytesRef offsetOut = new BytesRef(uncompressedDocLengths, 0, offsetBytesLen); - LZ4.decompress(input, offsetBytesLen, offsetOut.bytes, offsetOut.offset); + decompressor.decompress(input, offsetBytesLen, 0, offsetBytesLen, offsetOut); int docStart = 0; for (int i = 0; i < numDocsInBlock; i++) { @@ -393,8 +393,9 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException } assert uncompressedBlockLength <= uncompressedBlock.length; - BytesRef dataOut = new BytesRef(uncompressedBlock, 0, uncompressedBlockLength); - LZ4.decompress(input, uncompressedBlockLength, dataOut.bytes, dataOut.offset); + uncompressedBytesRef.offset = 0; + uncompressedBytesRef.length = uncompressedBlock.length; + decompressor.decompress(input, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); } // Find range containing docId that is within or after lastBlockId diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java index f40ed5baf74d6..2efcf3c79db20 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java @@ -112,13 +112,13 @@ public String toString() { } } - private static final class ZstdDecompressor extends Decompressor { + public static final class ZstdDecompressor extends Decompressor { // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough // to benefit from bulk copying and low enough to keep heap usage under control. final byte[] copyBuffer = new byte[4096]; - ZstdDecompressor() {} + public ZstdDecompressor() {} @Override public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException { @@ -163,14 +163,14 @@ public Decompressor clone() { } } - private static class ZstdCompressor extends Compressor { + public static class ZstdCompressor extends Compressor { final int level; // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough // to benefit from bulk copying and low enough to keep heap usage under control. final byte[] copyBuffer = new byte[4096]; - ZstdCompressor(int level) { + public ZstdCompressor(int level) { this.level = level; } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index f6f5f9fecf870..0108de392a9ea 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -1546,7 +1546,7 @@ private IndexWriterConfig getTimeSeriesIndexWriterConfig(String hostnameField, b public static BinaryDVCompressionMode randomBinaryCompressionMode() { // BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); // return modes[random().nextInt(modes.length)]; - return BinaryDVCompressionMode.COMPRESSED_WITH_LZ4; + return BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD; } } From 638dbbc2887ee395bafeb27fa5ed545eaded57cd Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 14:25:40 -0500 Subject: [PATCH 08/64] Fix direct monotonic reader size --- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 892975cad40a1..d8732865dfb83 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -277,7 +277,6 @@ public BytesRef binaryValue() throws IOException { } } - // START: Copied fom LUCENE-9211 private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException { if (entry.docsWithFieldOffset == -1) { // dense @@ -1364,17 +1363,17 @@ private BinaryEntry readBinary(IndexInput meta, int version) throws IOException } else { if (entry.numDocsWithField > 0 || entry.minLength < entry.maxLength) { entry.addressesOffset = meta.readLong(); - // New count of compressed addresses - the number of compresseed blocks + // New count of compressed addresses - the number of compressed blocks int numCompressedChunks = meta.readVInt(); entry.maxUncompressedChunkSize = meta.readVInt(); entry.maxNumDocsInAnyBlock = meta.readVInt(); final int blockShift = meta.readVInt(); - entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks, blockShift); + entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks + 1, blockShift); entry.addressesLength = meta.readLong(); entry.docRangeOffset = meta.readLong(); - entry.docRangeMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks * 2L, blockShift); + entry.docRangeMeta = DirectMonotonicReader.loadMeta(meta, numCompressedChunks + 1, blockShift); entry.docRangeLength = meta.readLong(); entry.numCompressedBlocks = numCompressedChunks; From fdf34288dcab0fc15ed7a023ef2ea872e889c20e Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 15:17:17 -0500 Subject: [PATCH 09/64] Fix docRangeLen bug, use for non-logsdb wildcards --- .../index/codec/PerFieldFormatSupplier.java | 12 +++++++++++- .../codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 2 +- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java index 2ed1aa6c9f17f..0c752c8ffd833 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java +++ b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java @@ -127,12 +127,22 @@ public KnnVectorsFormat getKnnVectorsFormatForField(String field) { } public DocValuesFormat getDocValuesFormatForField(String field) { - if (useTSDBDocValuesFormat(field)) { + if (useTSDBDocValuesFormat(field) || isBinaryDocValueField(field)) { return tsdbDocValuesFormat; } return docValuesFormat; } + boolean isBinaryDocValueField(final String field) { + if (mapperService != null) { + Mapper mapper = mapperService.mappingLookup().getMapper(field); + if (mapper != null && "wildcard".equals(mapper.typeName())) { + return true; + } + } + return false; + } + boolean useTSDBDocValuesFormat(final String field) { if (excludeFields(field)) { return false; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 31feba055d257..27c0acc6b9a20 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -626,7 +626,7 @@ void writeMetaData() throws IOException { meta.writeLong(dataDocRangeStart); blockDocRangeAcc.build(meta, data); - long docRangesLen = data.getFilePointer() - dataAddressesStart; + long docRangesLen = data.getFilePointer() - dataDocRangeStart; meta.writeLong(docRangesLen); } From 36b3e100e43ebe76516598f007625ecb1379e663 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 17:45:33 -0500 Subject: [PATCH 10/64] Change offset encoding from zstd to numeric --- .../es819/ES819TSDBDocValuesConsumer.java | 36 +++++++++---- .../es819/ES819TSDBDocValuesProducer.java | 51 +++++++++---------- 2 files changed, 52 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 27c0acc6b9a20..134c7a0e5c8e3 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -57,6 +57,7 @@ import static org.elasticsearch.index.codec.tsdb.es819.DocValuesConsumerUtil.compatibleWithOptimizedMerge; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SORTED_SET; @@ -514,16 +515,19 @@ public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProdu private class CompressedBinaryBlockWriter implements Closeable { static final int MIN_BLOCK_BYTES = 256 * 1024; - static final int START_BLOCK_DOCS = 1024; // likely needs to grow - private static final int ZSTD_LEVEL = 1; + static final int START_BLOCK_DOCS = 1024; + static final int ZSTD_LEVEL = 1; + + final Zstd814StoredFieldsFormat.ZstdCompressor compressor = new Zstd814StoredFieldsFormat.ZstdCompressor(ZSTD_LEVEL); + + final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); + final long[] docLengthCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; + int[] docLengths = new int[START_BLOCK_DOCS]; - private final Zstd814StoredFieldsFormat.ZstdCompressor compressor = new Zstd814StoredFieldsFormat.ZstdCompressor(ZSTD_LEVEL); int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; int numDocsInCurrentBlock = 0; - // Store ints, but use byte[] so can be zero-copy converted to ByteBuffer - byte[] docLengths = new byte[START_BLOCK_DOCS * Integer.BYTES]; byte[] block = BytesRef.EMPTY_BYTES; int totalChunks = 0; long maxPointer = 0; @@ -561,15 +565,14 @@ private class CompressedBinaryBlockWriter implements Closeable { * But we can guarantee that the lookup value is dense on the range of inserted values. */ void addDoc(int _docId, BytesRef v) throws IOException { - docLengths = ArrayUtil.grow(docLengths, (numDocsInCurrentBlock + 1) * Integer.BYTES); - BitUtil.VH_LE_INT.set(docLengths, numDocsInCurrentBlock * Integer.BYTES, v.length); + docLengths = ArrayUtil.grow(docLengths, numDocsInCurrentBlock + 1); + docLengths[numDocsInCurrentBlock] = v.length; block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; numDocsInCurrentBlock++; -// int totalUncompressedSize = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; if (uncompressedBlockLength > MIN_BLOCK_BYTES) { flushData(); } @@ -587,7 +590,7 @@ private void flushData() throws IOException { maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); - compress(docLengths, numDocsInCurrentBlock * Integer.BYTES, data); + compressOffsets(data, numDocsInCurrentBlock); compress(block, uncompressedBlockLength, data); blockDocRangeAcc.addDoc(numDocsInCurrentBlock); @@ -600,6 +603,21 @@ private void flushData() throws IOException { } } + void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { + int batchStart = 0; + while (batchStart < numDocsInCurrentBlock) { + int batchLength = Math.min(numDocsInCurrentBlock - batchStart, NUMERIC_BLOCK_SIZE); + for (int i = 0; i < batchLength; i++) { + docLengthCompressBuffer[i] = docLengths[batchStart + i]; + } + if (batchLength < docLengthCompressBuffer.length) { + Arrays.fill(docLengthCompressBuffer, batchLength, docLengthCompressBuffer.length,0); + } + encoder.encode(docLengthCompressBuffer, output); + batchStart += batchLength; + } + } + void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { ByteBuffer inputBuffer = ByteBuffer.wrap(data, 0, uncompressedLength); ByteBuffersDataInput input = new ByteBuffersDataInput(List.of(inputBuffer)); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index d8732865dfb83..73c87e973e2cb 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -54,6 +54,7 @@ import java.io.IOException; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -335,12 +336,13 @@ public BytesRef binaryValue() throws IOException { // Decompresses blocks of binary values to retrieve content static final class BinaryDecoder { + private final TSDBDocValuesEncoder decoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); private final LongValues addresses; private final LongValues docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final byte[] uncompressedDocLengths; + private final long[] docLengthDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -361,13 +363,10 @@ static final class BinaryDecoder { // pre-allocate a byte array large enough for the biggest uncompressed block needed. this.uncompressedBlock = new byte[biggestUncompressedBlockSize]; uncompressedBytesRef = new BytesRef(uncompressedBlock); - uncompressedDocLengths = new byte[(maxNumDocsInAnyBlock + 1) * Integer.BYTES]; uncompressedDocStarts = new int[maxNumDocsInAnyBlock + 1]; } - // unconditionally decompress blockId into - // uncompressedDocStarts - // uncompressedBlock + // unconditionally decompress blockId into uncompressedDocStarts and uncompressedBlock private void decompressBlock(int blockId, int numDocsInBlock) throws IOException { long blockStartOffset = addresses.get(blockId); compressedData.seek(blockStartOffset); @@ -378,23 +377,26 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - DataInput input = compressedData; - int offsetBytesLen = numDocsInBlock * Integer.BYTES; - assert offsetBytesLen <= uncompressedDocLengths.length; - BytesRef offsetOut = new BytesRef(uncompressedDocLengths, 0, offsetBytesLen); - decompressor.decompress(input, offsetBytesLen, 0, offsetBytesLen, offsetOut); - - int docStart = 0; - for (int i = 0; i < numDocsInBlock; i++) { - int len = getOffsetDocStart(i); - docStart += len; - uncompressedDocStarts[i+1] = docStart; - } + decompressDocOffsets(numDocsInBlock, compressedData); assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; uncompressedBytesRef.length = uncompressedBlock.length; - decompressor.decompress(input, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); + decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); + } + + void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { + int batchStart = 0; + while (batchStart < numDocsInBlock) { + decoder.decode(input, docLengthDecompBuffer); + int lenToCopy = Math.min(numDocsInBlock - batchStart, NUMERIC_BLOCK_SIZE); + for (int i = 0; i < lenToCopy; i++) { + // convert compressed length to offsets + int docLength = (int) docLengthDecompBuffer[i]; + uncompressedDocStarts[batchStart + i + 1] = docLength + uncompressedDocStarts[batchStart + i]; + } + batchStart += NUMERIC_BLOCK_SIZE; + } } // Find range containing docId that is within or after lastBlockId @@ -407,17 +409,14 @@ long getBlockContainingDoc(LongValues docRanges, long lastBlockId, int docNumber return blockId; } } - return -1; - } - - private int getOffsetDocStart(int idx) { - return (int) BitUtil.VH_LE_INT.get(uncompressedDocLengths, idx * Integer.BYTES); + throw new AssertionError("No block found containing document: " + docNumber + ", this should never happen."); } BytesRef decode(int docNumber, int numBlocks) throws IOException { - // docNumber because these are dense and could be indices from a DISI - long blockId = docNumber < limitDocNumForBlock ? lastBlockId : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); - assert blockId >= 0; + // docNumber, rather than docId, because these are dense and could be indices from a DISI + long blockId = docNumber < limitDocNumForBlock + ? lastBlockId + : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); int numDocsInBlock = (int) (limitDocNumForBlock - startDocNumForBlock); int idxInBlock = (int) (docNumber - startDocNumForBlock); From eeded36af34baa24e0a69b72aac53a533bf5b7d0 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 24 Oct 2025 23:22:50 +0000 Subject: [PATCH 11/64] [CI] Auto commit changes from spotless --- .../tsdb/es819/DelayedOffsetAccumulator.java | 17 +++-------------- .../es819/ES819TSDBDocValuesConsumer.java | 19 +++---------------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 7 ++++++- .../es819/ES819TSDBDocValuesProducer.java | 5 ++--- .../es819/ES819TSDBDocValuesFormatTests.java | 4 ++-- ...ValuesFormatVariableSkipIntervalTests.java | 14 ++++++++++++-- 6 files changed, 28 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java index 32efb0e2dfd5e..01e90bd8dd5ae 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java @@ -33,13 +33,7 @@ final class DelayedOffsetAccumulator implements Closeable { private final IndexOutput tempOutput; private final String suffix; - DelayedOffsetAccumulator( - Directory dir, - IOContext context, - IndexOutput data, - String suffix, - long startOffset - ) throws IOException { + DelayedOffsetAccumulator(Directory dir, IOContext context, IndexOutput data, String suffix, long startOffset) throws IOException { this.dir = dir; this.startOffset = startOffset; this.suffix = suffix; @@ -47,14 +41,9 @@ final class DelayedOffsetAccumulator implements Closeable { boolean success = false; try { tempOutput = dir.createTempOutput(data.getName(), suffix, context); - CodecUtil.writeHeader( - tempOutput, - ES819TSDBDocValuesFormat.META_CODEC + suffix, - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); + CodecUtil.writeHeader(tempOutput, ES819TSDBDocValuesFormat.META_CODEC + suffix, ES819TSDBDocValuesFormat.VERSION_CURRENT); success = true; - } - finally { + } finally { if (success == false) { IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 134c7a0e5c8e3..e33cfe81dbe7f 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -35,7 +35,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.LongsRef; @@ -538,22 +537,10 @@ private class CompressedBinaryBlockWriter implements Closeable { CompressedBinaryBlockWriter() throws IOException { long blockAddressesStart = data.getFilePointer(); - blockAddressAcc = new DelayedOffsetAccumulator( - state.directory, - state.context, - data, - "block_addresses", - blockAddressesStart - ); + blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block_addresses", blockAddressesStart); try { - blockDocRangeAcc = new DelayedOffsetAccumulator( - state.directory, - state.context, - data, - "block_doc_ranges", - 0 - ); + blockDocRangeAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block_doc_ranges", 0); } catch (IOException e) { blockAddressAcc.close(); throw e; @@ -611,7 +598,7 @@ void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOExce docLengthCompressBuffer[i] = docLengths[batchStart + i]; } if (batchLength < docLengthCompressBuffer.length) { - Arrays.fill(docLengthCompressBuffer, batchLength, docLengthCompressBuffer.length,0); + Arrays.fill(docLengthCompressBuffer, batchLength, docLengthCompressBuffer.length, 0); } encoder.encode(docLengthCompressBuffer, output); batchStart += batchLength; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 779dc38893861..e4de45f0e0822 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -125,7 +125,12 @@ private static boolean getOptimizedMergeEnabledDefault() { /** Default constructor. */ public ES819TSDBDocValuesFormat() { - this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD); + this( + DEFAULT_SKIP_INDEX_INTERVAL_SIZE, + ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, + OPTIMIZED_MERGE_ENABLE_DEFAULT, + BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD + ); } /** Doc values fields format with specified skipIndexIntervalSize. */ diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 73c87e973e2cb..9e27ed5742626 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -38,7 +38,6 @@ import org.apache.lucene.store.DataInput; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RandomAccessInput; -import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LongValues; import org.apache.lucene.util.compress.LZ4; @@ -392,7 +391,7 @@ void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOExceptio int lenToCopy = Math.min(numDocsInBlock - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < lenToCopy; i++) { // convert compressed length to offsets - int docLength = (int) docLengthDecompBuffer[i]; + int docLength = (int) docLengthDecompBuffer[i]; uncompressedDocStarts[batchStart + i + 1] = docLength + uncompressedDocStarts[batchStart + i]; } batchStart += NUMERIC_BLOCK_SIZE; @@ -430,7 +429,7 @@ BytesRef decode(int docNumber, int numBlocks) throws IOException { } int start = uncompressedDocStarts[idxInBlock]; - int end = uncompressedDocStarts[idxInBlock + 1]; + int end = uncompressedDocStarts[idxInBlock + 1]; uncompressedBytesRef.offset = start; uncompressedBytesRef.length = end - start; return uncompressedBytesRef; diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 0108de392a9ea..0124fc9071d80 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -1544,8 +1544,8 @@ private IndexWriterConfig getTimeSeriesIndexWriterConfig(String hostnameField, b } public static BinaryDVCompressionMode randomBinaryCompressionMode() { -// BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); -// return modes[random().nextInt(modes.length)]; + // BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); + // return modes[random().nextInt(modes.length)]; return BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD; } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java index b607e68ea755a..cca5764aa5982 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java @@ -19,14 +19,24 @@ public class ES819TSDBDocValuesFormatVariableSkipIntervalTests extends ES87TSDBD protected Codec getCodec() { // small interval size to test with many intervals return TestUtil.alwaysDocValuesFormat( - new ES819TSDBDocValuesFormat(random().nextInt(4, 16), random().nextInt(1, 32), random().nextBoolean(), ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode()) + new ES819TSDBDocValuesFormat( + random().nextInt(4, 16), + random().nextInt(1, 32), + random().nextBoolean(), + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ) ); } public void testSkipIndexIntervalSize() { IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> new ES819TSDBDocValuesFormat(random().nextInt(Integer.MIN_VALUE, 2), random().nextInt(1, 32), random().nextBoolean(), ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode()) + () -> new ES819TSDBDocValuesFormat( + random().nextInt(Integer.MIN_VALUE, 2), + random().nextInt(1, 32), + random().nextBoolean(), + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ) ); assertTrue(ex.getMessage().contains("skipIndexIntervalSize must be > 1")); } From 2d8e6dc6695f4d3ec74e9264a7b17b8c53ce4133 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 21:00:46 -0500 Subject: [PATCH 12/64] Fix missing compression in es819 format --- .../index/codec/tsdb/TSDBDocValuesMergeBenchmark.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index a3b2fd3633adf..534d5ba500af6 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.common.logging.LogConfigurator; import org.elasticsearch.index.codec.Elasticsearch92Lucene103Codec; +import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -257,7 +258,7 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE ); config.setLeafSorter(DataStream.TIMESERIES_LEAF_READERS_SORTER); config.setMergePolicy(new LogByteSizeMergePolicy()); - var docValuesFormat = new ES819TSDBDocValuesFormat(4096, 512, optimizedMergeEnabled); + var docValuesFormat = new ES819TSDBDocValuesFormat(4096, 512, optimizedMergeEnabled, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override public DocValuesFormat getDocValuesFormatForField(String field) { From c4d67e56526c9118a5beaa568854bcea44637a1d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 21:36:12 -0500 Subject: [PATCH 13/64] Store offsets rather than lengths --- .../es819/ES819TSDBDocValuesConsumer.java | 23 ++++++++++--------- .../es819/ES819TSDBDocValuesProducer.java | 13 +++++------ 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index e33cfe81dbe7f..fdd6d22c4ef1e 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -520,8 +520,8 @@ private class CompressedBinaryBlockWriter implements Closeable { final Zstd814StoredFieldsFormat.ZstdCompressor compressor = new Zstd814StoredFieldsFormat.ZstdCompressor(ZSTD_LEVEL); final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); - final long[] docLengthCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; - int[] docLengths = new int[START_BLOCK_DOCS]; + final long[] docOffsetsCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; + int[] docOffsets = new int[START_BLOCK_DOCS]; int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -552,13 +552,13 @@ private class CompressedBinaryBlockWriter implements Closeable { * But we can guarantee that the lookup value is dense on the range of inserted values. */ void addDoc(int _docId, BytesRef v) throws IOException { - docLengths = ArrayUtil.grow(docLengths, numDocsInCurrentBlock + 1); - docLengths[numDocsInCurrentBlock] = v.length; - block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; + numDocsInCurrentBlock++; + docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block + docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; if (uncompressedBlockLength > MIN_BLOCK_BYTES) { flushData(); @@ -592,15 +592,16 @@ private void flushData() throws IOException { void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { int batchStart = 0; - while (batchStart < numDocsInCurrentBlock) { - int batchLength = Math.min(numDocsInCurrentBlock - batchStart, NUMERIC_BLOCK_SIZE); + int numOffsets = numDocsInCurrentBlock + 1; + while (batchStart < numOffsets) { + int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < batchLength; i++) { - docLengthCompressBuffer[i] = docLengths[batchStart + i]; + docOffsetsCompressBuffer[i] = docOffsets[batchStart + i]; } - if (batchLength < docLengthCompressBuffer.length) { - Arrays.fill(docLengthCompressBuffer, batchLength, docLengthCompressBuffer.length, 0); + if (batchLength < docOffsetsCompressBuffer.length) { + Arrays.fill(docOffsetsCompressBuffer, batchLength, docOffsetsCompressBuffer.length, 0); } - encoder.encode(docLengthCompressBuffer, output); + encoder.encode(docOffsetsCompressBuffer, output); batchStart += batchLength; } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 63a815d23d40b..a6f2b5d7d5593 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -341,7 +341,7 @@ static final class BinaryDecoder { private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final long[] docLengthDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; + private final long[] docOffsetDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -386,13 +386,12 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { int batchStart = 0; - while (batchStart < numDocsInBlock) { - decoder.decode(input, docLengthDecompBuffer); - int lenToCopy = Math.min(numDocsInBlock - batchStart, NUMERIC_BLOCK_SIZE); + int numOffsets = numDocsInBlock + 1; + while (batchStart < numOffsets) { + decoder.decode(input, docOffsetDecompBuffer); + int lenToCopy = Math.min(numOffsets- batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < lenToCopy; i++) { - // convert compressed length to offsets - int docLength = (int) docLengthDecompBuffer[i]; - uncompressedDocStarts[batchStart + i + 1] = docLength + uncompressedDocStarts[batchStart + i]; + uncompressedDocStarts[batchStart + i] = (int) docOffsetDecompBuffer[i]; } batchStart += NUMERIC_BLOCK_SIZE; } From 06a2035e3ec3c4549237d0a35b062cecd7db0e0b Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Sat, 25 Oct 2025 02:42:24 +0000 Subject: [PATCH 14/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index a6f2b5d7d5593..0d7bf67605caf 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -389,7 +389,7 @@ void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOExceptio int numOffsets = numDocsInBlock + 1; while (batchStart < numOffsets) { decoder.decode(input, docOffsetDecompBuffer); - int lenToCopy = Math.min(numOffsets- batchStart, NUMERIC_BLOCK_SIZE); + int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < lenToCopy; i++) { uncompressedDocStarts[batchStart + i] = (int) docOffsetDecompBuffer[i]; } From 7ccb18d0be53058391511ecf964c6e55681a83dc Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 24 Oct 2025 21:44:35 -0500 Subject: [PATCH 15/64] Remove forbidden APIs --- .../index/codec/tsdb/es819/DelayedOffsetAccumulator.java | 2 +- .../index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java index 01e90bd8dd5ae..7986804317ad6 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java @@ -14,7 +14,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.IOUtils; +import org.elasticsearch.core.IOUtils; import org.apache.lucene.util.packed.DirectMonotonicWriter; import java.io.Closeable; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index fdd6d22c4ef1e..d42e4250cd310 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -567,7 +567,6 @@ void addDoc(int _docId, BytesRef v) throws IOException { private void flushData() throws IOException { if (numDocsInCurrentBlock > 0) { - // Write offset to this block to temporary offsets file totalChunks++; long thisBlockStartPointer = data.getFilePointer(); From a57e0d47ab78cf532d4e6c2622ba4d5342f39ca1 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Sat, 25 Oct 2025 02:53:10 +0000 Subject: [PATCH 16/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/es819/DelayedOffsetAccumulator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java index 7986804317ad6..d172d8837b47e 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java @@ -14,8 +14,8 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; -import org.elasticsearch.core.IOUtils; import org.apache.lucene.util.packed.DirectMonotonicWriter; +import org.elasticsearch.core.IOUtils; import java.io.Closeable; import java.io.IOException; From f156e554636ea22c7a24408ae8b90523138a6354 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Sun, 26 Oct 2025 19:46:23 -0500 Subject: [PATCH 17/64] Binary search to find block containing docNum --- .../es819/ES819TSDBDocValuesProducer.java | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 0d7bf67605caf..17aad6165fd13 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -284,7 +284,7 @@ private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOExceptio final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); final RandomAccessInput docRangeData = this.data.randomAccessSlice(entry.docRangeOffset, entry.docRangeLength); - final LongValues docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); + final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new DenseBinaryDocValues(maxDoc) { final BinaryDecoder decoder = new BinaryDecoder( addresses, @@ -313,7 +313,7 @@ public BytesRef binaryValue() throws IOException { final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData); final RandomAccessInput docRangeData = this.data.randomAccessSlice(entry.docRangeOffset, entry.docRangeLength); - final LongValues docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); + final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new SparseBinaryDocValues(disi) { final BinaryDecoder decoder = new BinaryDecoder( addresses, @@ -337,7 +337,7 @@ static final class BinaryDecoder { private final TSDBDocValuesEncoder decoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); private final LongValues addresses; - private final LongValues docRanges; + private final DirectMonotonicReader docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; @@ -351,7 +351,7 @@ static final class BinaryDecoder { BinaryDecoder( LongValues addresses, - LongValues docRanges, + DirectMonotonicReader docRanges, IndexInput compressedData, int biggestUncompressedBlockSize, int maxNumDocsInAnyBlock @@ -397,24 +397,26 @@ void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOExceptio } } - // Find range containing docId that is within or after lastBlockId - // Could change to binary search, though since we usually scan forward this would probably be slower - long getBlockContainingDoc(LongValues docRanges, long lastBlockId, int docNumber, int numBlocks) { - for (long blockId = lastBlockId + 1; blockId < numBlocks; blockId++) { - startDocNumForBlock = docRanges.get(blockId); - limitDocNumForBlock = docRanges.get(blockId + 1); - if (docNumber < limitDocNumForBlock) { - return blockId; - } + long findAndUpdateBlock(DirectMonotonicReader docOffsets, long lastBlockId, int docNumber, int numBlocks) { + long index = docOffsets.binarySearch(lastBlockId + 1, numBlocks, docNumber); + // If index is found, index is inclusive lower bound of docNum range, so docNum is in blockId == index + if (index < 0) { + // If index was not found, insertion point (-index - 1) will be upper bound of docNum range. + // Subtract additional 1 so that index points to lower bound of doc range, which is the blockId + index = -2 - index; } - throw new AssertionError("No block found containing document: " + docNumber + ", this should never happen."); + assert index < numBlocks: "invalid range " + index + " for doc " + docNumber + " in numBlocks " + numBlocks; + + startDocNumForBlock = docOffsets.get(index); + limitDocNumForBlock = docOffsets.get(index + 1); + return index; } BytesRef decode(int docNumber, int numBlocks) throws IOException { // docNumber, rather than docId, because these are dense and could be indices from a DISI long blockId = docNumber < limitDocNumForBlock ? lastBlockId - : getBlockContainingDoc(docRanges, lastBlockId, docNumber, numBlocks); + : findAndUpdateBlock(docRanges, lastBlockId, docNumber, numBlocks); int numDocsInBlock = (int) (limitDocNumForBlock - startDocNumForBlock); int idxInBlock = (int) (docNumber - startDocNumForBlock); From 91e5842e34d695cb370e5273920380ddaaeb1aa6 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 27 Oct 2025 00:53:18 +0000 Subject: [PATCH 18/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 17aad6165fd13..d3c90b28b423c 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -405,7 +405,7 @@ long findAndUpdateBlock(DirectMonotonicReader docOffsets, long lastBlockId, int // Subtract additional 1 so that index points to lower bound of doc range, which is the blockId index = -2 - index; } - assert index < numBlocks: "invalid range " + index + " for doc " + docNumber + " in numBlocks " + numBlocks; + assert index < numBlocks : "invalid range " + index + " for doc " + docNumber + " in numBlocks " + numBlocks; startDocNumForBlock = docOffsets.get(index); limitDocNumForBlock = docOffsets.get(index + 1); @@ -414,9 +414,7 @@ long findAndUpdateBlock(DirectMonotonicReader docOffsets, long lastBlockId, int BytesRef decode(int docNumber, int numBlocks) throws IOException { // docNumber, rather than docId, because these are dense and could be indices from a DISI - long blockId = docNumber < limitDocNumForBlock - ? lastBlockId - : findAndUpdateBlock(docRanges, lastBlockId, docNumber, numBlocks); + long blockId = docNumber < limitDocNumForBlock ? lastBlockId : findAndUpdateBlock(docRanges, lastBlockId, docNumber, numBlocks); int numDocsInBlock = (int) (limitDocNumForBlock - startDocNumForBlock); int idxInBlock = (int) (docNumber - startDocNumForBlock); From 401a041c580ce3dfb5f5647ea55db048820b1864 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 27 Oct 2025 09:43:00 -0500 Subject: [PATCH 19/64] do not mmap temp offset files --- .../tsdb/es819/ES819TSDBDocValuesConsumer.java | 4 ++-- .../index/store/FsDirectoryFactory.java | 15 ++++++++++++--- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index d42e4250cd310..13c59dc7bc044 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -537,10 +537,10 @@ private class CompressedBinaryBlockWriter implements Closeable { CompressedBinaryBlockWriter() throws IOException { long blockAddressesStart = data.getFilePointer(); - blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block_addresses", blockAddressesStart); + blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-addresses", blockAddressesStart); try { - blockDocRangeAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block_doc_ranges", 0); + blockDocRangeAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-doc-ranges", 0); } catch (IOException e) { blockAddressAcc.close(); throw e; diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java index c235516d6c363..67762541a783e 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java @@ -41,6 +41,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -290,8 +291,8 @@ static boolean useDelegate(String name, IOContext ioContext) { * mmap-ing that should still be ok even is memory is scarce. * The fdt file is large and tends to cause more page faults when memory is scarce. * - * For disi and address-data files, in es819 tsdb doc values codec, docids and offsets are first written to a tmp file and - * read and written into new segment. + * For disi, address-data, block-addresses, and block-doc-ranges files, in es819 tsdb doc values codec, + * docids and offsets are first written to a tmp file and read and written into new segment. * * @param name The name of the file in Lucene index * @param extension The extension of the in Lucene index @@ -299,9 +300,17 @@ static boolean useDelegate(String name, IOContext ioContext) { */ static boolean avoidDelegateForFdtTempFiles(String name, LuceneFilesExtensions extension) { return extension == LuceneFilesExtensions.TMP - && (name.contains("fdt") || name.contains("disi") || name.contains("address-data")); + && NO_MMAP_FILE_SUFFIXES.stream().anyMatch(name::contains); } + static final List NO_MMAP_FILE_SUFFIXES = List.of( + "fdt", + "disi", + "address-data", + "block-addresses", + "block-doc-ranges" + ); + MMapDirectory getDelegate() { return delegate; } From ad55bc3cc3bc8700c9c271f03b13510e3be29c94 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 27 Oct 2025 10:50:31 -0500 Subject: [PATCH 20/64] feedback --- .../index/codec/PerFieldFormatSupplier.java | 12 +----------- .../tsdb/es819/ES819TSDBDocValuesConsumer.java | 8 ++------ .../tsdb/es819/ES819TSDBDocValuesProducer.java | 16 ++++++++-------- .../es819/ES819TSDBDocValuesFormatTests.java | 5 ++--- 4 files changed, 13 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java index 0c752c8ffd833..2ed1aa6c9f17f 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java +++ b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java @@ -127,22 +127,12 @@ public KnnVectorsFormat getKnnVectorsFormatForField(String field) { } public DocValuesFormat getDocValuesFormatForField(String field) { - if (useTSDBDocValuesFormat(field) || isBinaryDocValueField(field)) { + if (useTSDBDocValuesFormat(field)) { return tsdbDocValuesFormat; } return docValuesFormat; } - boolean isBinaryDocValueField(final String field) { - if (mapperService != null) { - Mapper mapper = mapperService.mappingLookup().getMapper(field); - if (mapper != null && "wildcard".equals(mapper.typeName())) { - return true; - } - } - return false; - } - boolean useTSDBDocValuesFormat(final String field) { if (excludeFields(field)) { return false; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 13c59dc7bc044..9ee46ea98f391 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -474,7 +474,7 @@ public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProdu for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { numDocsWithField++; BytesRef v = values.binaryValue(); - blockWriter.addDoc(doc, v); + blockWriter.addDoc(v); int length = v.length; minLength = Math.min(length, minLength); maxLength = Math.max(length, maxLength); @@ -547,11 +547,7 @@ private class CompressedBinaryBlockWriter implements Closeable { } } - /** - * _docId is unused. This is because docId may not be dense. - * But we can guarantee that the lookup value is dense on the range of inserted values. - */ - void addDoc(int _docId, BytesRef v) throws IOException { + void addDoc(BytesRef v) throws IOException { block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index d3c90b28b423c..691068a1fe207 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -420,7 +420,6 @@ BytesRef decode(int docNumber, int numBlocks) throws IOException { int idxInBlock = (int) (docNumber - startDocNumForBlock); assert idxInBlock < numDocsInBlock; - // already read and uncompressed? if (blockId != lastBlockId) { decompressBlock((int) blockId, numDocsInBlock); // uncompressedBytesRef and uncompressedDocStarts now populated @@ -1673,6 +1672,14 @@ long lookAheadValueAt(int targetDoc) throws IOException { } return lookaheadBlock[valueIndex]; } + + static boolean isDense(int firstDocId, int lastDocId, int length) { + // This does not detect duplicate docids (e.g [1, 1, 2, 4] would be detected as dense), + // this can happen with enrich or lookup. However this codec isn't used for enrich / lookup. + // This codec is only used in the context of logsdb and tsdb, so this is fine here. + return lastDocId - firstDocId == length - 1; + } + }; } else { final IndexedDISI disi = new IndexedDISI( @@ -1787,13 +1794,6 @@ public BlockLoader.Block tryRead( } } - static boolean isDense(int firstDocId, int lastDocId, int length) { - // This does not detect duplicate docids (e.g [1, 1, 2, 4] would be detected as dense), - // this can happen with enrich or lookup. However this codec isn't used for enrich / lookup. - // This codec is only used in the context of logsdb and tsdb, so this is fine here. - return lastDocId - firstDocId == length - 1; - } - private NumericDocValues getRangeEncodedNumericDocValues(NumericEntry entry, long maxOrd) throws IOException { final var ordinalsReader = new SortedOrdinalReader( maxOrd, diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 0124fc9071d80..1b97d7f341586 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -1544,9 +1544,8 @@ private IndexWriterConfig getTimeSeriesIndexWriterConfig(String hostnameField, b } public static BinaryDVCompressionMode randomBinaryCompressionMode() { - // BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); - // return modes[random().nextInt(modes.length)]; - return BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD; + BinaryDVCompressionMode[] modes = BinaryDVCompressionMode.values(); + return modes[random().nextInt(modes.length)]; } } From 4d4e15369cb5b34fc91eb91c3d50f072820a1366 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 27 Oct 2025 16:16:33 +0000 Subject: [PATCH 21/64] [CI] Auto commit changes from spotless --- .../elasticsearch/index/store/FsDirectoryFactory.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java index 67762541a783e..cd1deec68a342 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java @@ -299,17 +299,10 @@ static boolean useDelegate(String name, IOContext ioContext) { * @return whether to avoid using delegate if the file is a tmp fdt file. */ static boolean avoidDelegateForFdtTempFiles(String name, LuceneFilesExtensions extension) { - return extension == LuceneFilesExtensions.TMP - && NO_MMAP_FILE_SUFFIXES.stream().anyMatch(name::contains); + return extension == LuceneFilesExtensions.TMP && NO_MMAP_FILE_SUFFIXES.stream().anyMatch(name::contains); } - static final List NO_MMAP_FILE_SUFFIXES = List.of( - "fdt", - "disi", - "address-data", - "block-addresses", - "block-doc-ranges" - ); + static final List NO_MMAP_FILE_SUFFIXES = List.of("fdt", "disi", "address-data", "block-addresses", "block-doc-ranges"); MMapDirectory getDelegate() { return delegate; From f1ff1828e4f622246c0d2df12e20a10ffbf77300 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 27 Oct 2025 11:22:56 -0500 Subject: [PATCH 22/64] Move zstd (de)compressor to separate class --- .../tsdb/es819/DelayedOffsetAccumulator.java | 2 +- .../es819/ES819TSDBDocValuesConsumer.java | 4 +- .../es819/ES819TSDBDocValuesProducer.java | 4 +- .../codec/zstd/Zstd814StoredFieldsFormat.java | 117 ------------------ .../index/codec/zstd/ZstdCompressor.java | 77 ++++++++++++ .../index/codec/zstd/ZstdDecompressor.java | 73 +++++++++++ 6 files changed, 155 insertions(+), 122 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressor.java create mode 100644 server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java index d172d8837b47e..1e9d4e46d2b70 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java @@ -23,7 +23,7 @@ /** * Like OffsetsAccumulator builds offsets and stores in a DirectMonotonicWriter. But write to temp file * rather than directly to a DirectMonotonicWriter because the number of values is unknown. If number of - * values if known prefer OffsetsWriter. + * values is known prefer OffsetsWriter. */ final class DelayedOffsetAccumulator implements Closeable { private final Directory dir; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 9ee46ea98f391..f6933a847f7a1 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -45,7 +45,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; -import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat; +import org.elasticsearch.index.codec.zstd.ZstdCompressor; import java.io.Closeable; import java.io.IOException; @@ -517,7 +517,7 @@ private class CompressedBinaryBlockWriter implements Closeable { static final int START_BLOCK_DOCS = 1024; static final int ZSTD_LEVEL = 1; - final Zstd814StoredFieldsFormat.ZstdCompressor compressor = new Zstd814StoredFieldsFormat.ZstdCompressor(ZSTD_LEVEL); + final ZstdCompressor compressor = new ZstdCompressor(ZSTD_LEVEL); final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); final long[] docOffsetsCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 691068a1fe207..da4538b0973af 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -47,7 +47,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; -import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat; +import org.elasticsearch.index.codec.zstd.ZstdDecompressor; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.blockloader.docvalues.BlockDocValuesReader; @@ -347,7 +347,7 @@ static final class BinaryDecoder { private final BytesRef uncompressedBytesRef; private long startDocNumForBlock = -1; private long limitDocNumForBlock = -1; - private final Zstd814StoredFieldsFormat.ZstdDecompressor decompressor = new Zstd814StoredFieldsFormat.ZstdDecompressor(); + private final ZstdDecompressor decompressor = new ZstdDecompressor(); BinaryDecoder( LongValues addresses, diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java index 2efcf3c79db20..cb7b570ff303d 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java @@ -14,18 +14,9 @@ import org.apache.lucene.codecs.compressing.Compressor; import org.apache.lucene.codecs.compressing.Decompressor; import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.store.ByteBuffersDataInput; -import org.apache.lucene.store.DataInput; -import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.nativeaccess.CloseableByteBuffer; -import org.elasticsearch.nativeaccess.NativeAccess; -import org.elasticsearch.nativeaccess.Zstd; import java.io.IOException; @@ -112,112 +103,4 @@ public String toString() { } } - public static final class ZstdDecompressor extends Decompressor { - - // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough - // to benefit from bulk copying and low enough to keep heap usage under control. - final byte[] copyBuffer = new byte[4096]; - - public ZstdDecompressor() {} - - @Override - public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException { - if (originalLength == 0) { - bytes.offset = 0; - bytes.length = 0; - return; - } - - final NativeAccess nativeAccess = NativeAccess.instance(); - final Zstd zstd = nativeAccess.getZstd(); - - final int compressedLength = in.readVInt(); - - try ( - CloseableByteBuffer src = nativeAccess.newConfinedBuffer(compressedLength); - CloseableByteBuffer dest = nativeAccess.newConfinedBuffer(originalLength) - ) { - - while (src.buffer().position() < compressedLength) { - final int numBytes = Math.min(copyBuffer.length, compressedLength - src.buffer().position()); - in.readBytes(copyBuffer, 0, numBytes); - src.buffer().put(copyBuffer, 0, numBytes); - } - src.buffer().flip(); - - final int decompressedLen = zstd.decompress(dest, src); - if (decompressedLen != originalLength) { - throw new CorruptIndexException("Expected " + originalLength + " decompressed bytes, got " + decompressedLen, in); - } - - bytes.bytes = ArrayUtil.growNoCopy(bytes.bytes, length); - dest.buffer().get(offset, bytes.bytes, 0, length); - bytes.offset = 0; - bytes.length = length; - } - } - - @Override - public Decompressor clone() { - return new ZstdDecompressor(); - } - } - - public static class ZstdCompressor extends Compressor { - - final int level; - // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough - // to benefit from bulk copying and low enough to keep heap usage under control. - final byte[] copyBuffer = new byte[4096]; - - public ZstdCompressor(int level) { - this.level = level; - } - - @Override - public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException { - final NativeAccess nativeAccess = NativeAccess.instance(); - final Zstd zstd = nativeAccess.getZstd(); - - final int srcLen = Math.toIntExact(buffersInput.length()); - if (srcLen == 0) { - return; - } - - final int compressBound = zstd.compressBound(srcLen); - - // NOTE: We are allocating/deallocating native buffers on each call. We could save allocations by reusing these buffers, though - // this would come at the expense of higher permanent memory usage. Benchmarks suggested that there is some performance to save - // there, but it wouldn't be a game changer either. - // Also note that calls to #compress implicitly allocate memory under the hood for e.g. hash tables and chain tables that help - // identify duplicate strings. So if we wanted to avoid allocating memory on every compress call, we should also look into - // reusing compression contexts, which are not small and would increase permanent memory usage as well. - try ( - CloseableByteBuffer src = nativeAccess.newConfinedBuffer(srcLen); - CloseableByteBuffer dest = nativeAccess.newConfinedBuffer(compressBound) - ) { - - while (buffersInput.position() < buffersInput.length()) { - final int numBytes = Math.min(copyBuffer.length, (int) (buffersInput.length() - buffersInput.position())); - buffersInput.readBytes(copyBuffer, 0, numBytes); - src.buffer().put(copyBuffer, 0, numBytes); - } - src.buffer().flip(); - - final int compressedLen = zstd.compress(dest, src, level); - out.writeVInt(compressedLen); - - for (int written = 0; written < compressedLen;) { - final int numBytes = Math.min(copyBuffer.length, compressedLen - written); - dest.buffer().get(copyBuffer, 0, numBytes); - out.writeBytes(copyBuffer, 0, numBytes); - written += numBytes; - assert written == dest.buffer().position(); - } - } - } - - @Override - public void close() throws IOException {} - } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressor.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressor.java new file mode 100644 index 0000000000000..f7af543c17199 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressor.java @@ -0,0 +1,77 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.zstd; + +import org.apache.lucene.codecs.compressing.Compressor; +import org.apache.lucene.store.ByteBuffersDataInput; +import org.apache.lucene.store.DataOutput; +import org.elasticsearch.nativeaccess.CloseableByteBuffer; +import org.elasticsearch.nativeaccess.NativeAccess; +import org.elasticsearch.nativeaccess.Zstd; + +import java.io.IOException; + +public class ZstdCompressor extends Compressor { + + final int level; + // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough + // to benefit from bulk copying and low enough to keep heap usage under control. + final byte[] copyBuffer = new byte[4096]; + + public ZstdCompressor(int level) { + this.level = level; + } + + @Override + public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException { + final NativeAccess nativeAccess = NativeAccess.instance(); + final Zstd zstd = nativeAccess.getZstd(); + + final int srcLen = Math.toIntExact(buffersInput.length()); + if (srcLen == 0) { + return; + } + + final int compressBound = zstd.compressBound(srcLen); + + // NOTE: We are allocating/deallocating native buffers on each call. We could save allocations by reusing these buffers, though + // this would come at the expense of higher permanent memory usage. Benchmarks suggested that there is some performance to save + // there, but it wouldn't be a game changer either. + // Also note that calls to #compress implicitly allocate memory under the hood for e.g. hash tables and chain tables that help + // identify duplicate strings. So if we wanted to avoid allocating memory on every compress call, we should also look into + // reusing compression contexts, which are not small and would increase permanent memory usage as well. + try ( + CloseableByteBuffer src = nativeAccess.newConfinedBuffer(srcLen); + CloseableByteBuffer dest = nativeAccess.newConfinedBuffer(compressBound) + ) { + + while (buffersInput.position() < buffersInput.length()) { + final int numBytes = Math.min(copyBuffer.length, (int) (buffersInput.length() - buffersInput.position())); + buffersInput.readBytes(copyBuffer, 0, numBytes); + src.buffer().put(copyBuffer, 0, numBytes); + } + src.buffer().flip(); + + final int compressedLen = zstd.compress(dest, src, level); + out.writeVInt(compressedLen); + + for (int written = 0; written < compressedLen;) { + final int numBytes = Math.min(copyBuffer.length, compressedLen - written); + dest.buffer().get(copyBuffer, 0, numBytes); + out.writeBytes(copyBuffer, 0, numBytes); + written += numBytes; + assert written == dest.buffer().position(); + } + } + } + + @Override + public void close() throws IOException {} +} diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java new file mode 100644 index 0000000000000..e9762b67410c2 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java @@ -0,0 +1,73 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.zstd; + +import org.apache.lucene.codecs.compressing.Decompressor; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.nativeaccess.CloseableByteBuffer; +import org.elasticsearch.nativeaccess.NativeAccess; +import org.elasticsearch.nativeaccess.Zstd; + +import java.io.IOException; + +public final class ZstdDecompressor extends Decompressor { + + // Buffer for copying between the DataInput and native memory. No hard science behind this number, it just tries to be high enough + // to benefit from bulk copying and low enough to keep heap usage under control. + final byte[] copyBuffer = new byte[4096]; + + public ZstdDecompressor() { + } + + @Override + public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException { + if (originalLength == 0) { + bytes.offset = 0; + bytes.length = 0; + return; + } + + final NativeAccess nativeAccess = NativeAccess.instance(); + final Zstd zstd = nativeAccess.getZstd(); + + final int compressedLength = in.readVInt(); + + try ( + CloseableByteBuffer src = nativeAccess.newConfinedBuffer(compressedLength); + CloseableByteBuffer dest = nativeAccess.newConfinedBuffer(originalLength) + ) { + + while (src.buffer().position() < compressedLength) { + final int numBytes = Math.min(copyBuffer.length, compressedLength - src.buffer().position()); + in.readBytes(copyBuffer, 0, numBytes); + src.buffer().put(copyBuffer, 0, numBytes); + } + src.buffer().flip(); + + final int decompressedLen = zstd.decompress(dest, src); + if (decompressedLen != originalLength) { + throw new CorruptIndexException("Expected " + originalLength + " decompressed bytes, got " + decompressedLen, in); + } + + bytes.bytes = ArrayUtil.growNoCopy(bytes.bytes, length); + dest.buffer().get(offset, bytes.bytes, 0, length); + bytes.offset = 0; + bytes.length = length; + } + } + + @Override + public Decompressor clone() { + return new ZstdDecompressor(); + } +} From 9d2f2378821a1c616c13848d1ada895a3dbc0cc5 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 27 Oct 2025 18:37:41 -0500 Subject: [PATCH 23/64] Combine doAddCompressedBinary and doAddUncompressedBinary --- .../tsdb/TSDBDocValuesMergeBenchmark.java | 2 +- .../codec/tsdb/BinaryDVCompressionMode.java | 17 +- .../es819/ES819TSDBDocValuesConsumer.java | 246 +++++++++--------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 2 +- .../es819/ES819TSDBDocValuesProducer.java | 5 +- 5 files changed, 148 insertions(+), 124 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index 534d5ba500af6..5078415e55485 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -258,7 +258,7 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE ); config.setLeafSorter(DataStream.TIMESERIES_LEAF_READERS_SORTER); config.setMergePolicy(new LogByteSizeMergePolicy()); - var docValuesFormat = new ES819TSDBDocValuesFormat(4096, 512, optimizedMergeEnabled, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD); + var docValuesFormat = new ES819TSDBDocValuesFormat(4096, 512, optimizedMergeEnabled, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override public DocValuesFormat getDocValuesFormatForField(String field) { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 0be015de97521..ef3fc86debb74 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -9,21 +9,30 @@ package org.elasticsearch.index.codec.tsdb; +import org.apache.lucene.codecs.compressing.Compressor; +import org.apache.lucene.codecs.compressing.Decompressor; +import org.elasticsearch.index.codec.zstd.ZstdCompressor; +import org.elasticsearch.index.codec.zstd.ZstdDecompressor; + public enum BinaryDVCompressionMode { - NO_COMPRESS((byte) 0), - COMPRESSED_WITH_ZSTD((byte) 1); + NO_COMPRESS((byte) 0, null, null), + COMPRESSED_WITH_ZSTD_1((byte) 1, new ZstdCompressor(1), new ZstdDecompressor()); public final byte code; + public final Compressor compressor; + public final Decompressor decompressor; - BinaryDVCompressionMode(byte code) { + BinaryDVCompressionMode(byte code, Compressor compressor, Decompressor decompressor) { this.code = code; + this.compressor = compressor; + this.decompressor = decompressor; } public static BinaryDVCompressionMode fromMode(byte mode) { return switch (mode) { case 0 -> NO_COMPRESS; - case 1 -> COMPRESSED_WITH_ZSTD; + case 1 -> COMPRESSED_WITH_ZSTD_1; default -> throw new IllegalStateException("unknown compression mode [" + mode + "]"); }; } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index f6933a847f7a1..3b197e4579cfa 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -45,7 +45,6 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; -import org.elasticsearch.index.codec.zstd.ZstdCompressor; import java.io.Closeable; import java.io.IOException; @@ -328,13 +327,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th meta.writeInt(field.number); meta.writeByte(ES819TSDBDocValuesFormat.BINARY); meta.writeByte(binaryDVCompressionMode.code); - switch (binaryDVCompressionMode) { - case NO_COMPRESS -> doAddUncompressedBinary(field, valuesProducer); - case COMPRESSED_WITH_ZSTD -> doAddCompressedBinary(field, valuesProducer); - } - } - public void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { if (valuesProducer instanceof TsdbDocValuesProducer tsdbValuesProducer && tsdbValuesProducer.mergeStats.supported()) { final int numDocsWithField = tsdbValuesProducer.mergeStats.sumNumDocsWithField(); final int minLength = tsdbValuesProducer.mergeStats.minLength(); @@ -346,28 +339,29 @@ public void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesPro long start = data.getFilePointer(); meta.writeLong(start); // dataOffset - OffsetsAccumulator offsetsAccumulator = null; DISIAccumulator disiAccumulator = null; + BinaryWriter binaryWriter = null; try { if (numDocsWithField > 0 && numDocsWithField < maxDoc) { disiAccumulator = new DISIAccumulator(dir, context, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER); } assert maxLength >= minLength; - if (maxLength > minLength) { - offsetsAccumulator = new OffsetsAccumulator(dir, context, data, numDocsWithField); + if (binaryDVCompressionMode == BinaryDVCompressionMode.NO_COMPRESS) { + var offsetsAccumulator = maxLength > minLength ? new OffsetsAccumulator(dir, context, data, numDocsWithField) : null; + binaryWriter = new DirectBinaryWriterOptimized(offsetsAccumulator); + } else { + binaryWriter = new CompressedBinaryBlockWriter(binaryDVCompressionMode); } for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { BytesRef v = values.binaryValue(); - data.writeBytes(v.bytes, v.offset, v.length); + binaryWriter.addDoc(v); if (disiAccumulator != null) { disiAccumulator.addDocId(doc); } - if (offsetsAccumulator != null) { - offsetsAccumulator.addDoc(v.length); - } } + binaryWriter.flushData(); meta.writeLong(data.getFilePointer() - start); // dataLength if (numDocsWithField == 0) { @@ -392,56 +386,123 @@ public void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesPro meta.writeInt(numDocsWithField); meta.writeInt(minLength); meta.writeInt(maxLength); - if (offsetsAccumulator != null) { - offsetsAccumulator.build(meta, data); - } + + binaryWriter.writeAddressMetadata(minLength, maxLength, numDocsWithField); } finally { - IOUtils.close(disiAccumulator, offsetsAccumulator); + IOUtils.close(disiAccumulator, binaryWriter); } } else { - 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); + BinaryWriter binaryWriter = null; + try { + if (binaryDVCompressionMode == BinaryDVCompressionMode.NO_COMPRESS) { + binaryWriter = new DirectBinaryWriterUnoptimized(valuesProducer, field); + } else { + binaryWriter = new CompressedBinaryBlockWriter(binaryDVCompressionMode); + } + + 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; + binaryWriter.addDoc(v); + minLength = Math.min(length, minLength); + maxLength = Math.max(length, maxLength); + } + binaryWriter.flushData(); + + 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); + + binaryWriter.writeAddressMetadata(minLength, maxLength, numDocsWithField); + } finally { + IOUtils.close(binaryWriter); } - 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); + private interface BinaryWriter extends Closeable { + void addDoc(BytesRef v) throws IOException; + + default void flushData() throws IOException {} + + default void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException {} + + @Override + default void close() throws IOException {} + } + + private class DirectBinaryWriterOptimized implements BinaryWriter { + final OffsetsAccumulator offsetsAccumulator; + + private DirectBinaryWriterOptimized(OffsetsAccumulator offsetsAccumulator) { + this.offsetsAccumulator = offsetsAccumulator; + } + + @Override + public void addDoc(BytesRef v) throws IOException { + data.writeBytes(v.bytes, v.offset, v.length); + if (offsetsAccumulator != null) { + offsetsAccumulator.addDoc(v.length); + } + } + + @Override + public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException { + if (offsetsAccumulator != null) { + offsetsAccumulator.build(meta, data); } + } + } + + private class DirectBinaryWriterUnoptimized implements BinaryWriter { + private final DocValuesProducer valuesProducer; + private final FieldInfo field; - meta.writeInt(numDocsWithField); - meta.writeInt(minLength); - meta.writeInt(maxLength); + private DirectBinaryWriterUnoptimized(DocValuesProducer valuesProducer, FieldInfo field) { + this.valuesProducer = valuesProducer; + this.field = field; + } + + @Override + public void addDoc(BytesRef v) throws IOException { + data.writeBytes(v.bytes, v.offset, v.length); + } + + @Override + public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException { if (maxLength > minLength) { - start = data.getFilePointer(); - meta.writeLong(start); + long addressStart = data.getFilePointer(); + meta.writeLong(addressStart); meta.writeVInt(ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT); final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( @@ -452,72 +513,22 @@ public void doAddUncompressedBinary(FieldInfo field, DocValuesProducer valuesPro ); long addr = 0; writer.add(addr); - values = valuesProducer.getBinary(field); + var 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); + meta.writeLong(data.getFilePointer() - addressStart); } } } - public void doAddCompressedBinary(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { - try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) { - 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(); - blockWriter.addDoc(v); - int length = v.length; - minLength = Math.min(length, minLength); - maxLength = Math.max(length, maxLength); - } - blockWriter.flushData(); - - 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); - - blockWriter.writeMetaData(); - } - } - - private class CompressedBinaryBlockWriter implements Closeable { + private class CompressedBinaryBlockWriter implements BinaryWriter { static final int MIN_BLOCK_BYTES = 256 * 1024; static final int START_BLOCK_DOCS = 1024; - static final int ZSTD_LEVEL = 1; - final ZstdCompressor compressor = new ZstdCompressor(ZSTD_LEVEL); + final BinaryDVCompressionMode compressionMode; final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); final long[] docOffsetsCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; @@ -535,7 +546,8 @@ private class CompressedBinaryBlockWriter implements Closeable { final DelayedOffsetAccumulator blockAddressAcc; final DelayedOffsetAccumulator blockDocRangeAcc; - CompressedBinaryBlockWriter() throws IOException { + CompressedBinaryBlockWriter(BinaryDVCompressionMode compressionMode) throws IOException { + this.compressionMode = compressionMode; long blockAddressesStart = data.getFilePointer(); blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-addresses", blockAddressesStart); @@ -547,7 +559,8 @@ private class CompressedBinaryBlockWriter implements Closeable { } } - void addDoc(BytesRef v) throws IOException { + @Override + public void addDoc(BytesRef v) throws IOException { block = ArrayUtil.grow(block, uncompressedBlockLength + v.length); System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length); uncompressedBlockLength += v.length; @@ -561,7 +574,8 @@ void addDoc(BytesRef v) throws IOException { } } - private void flushData() throws IOException { + @Override + public void flushData() throws IOException { if (numDocsInCurrentBlock > 0) { totalChunks++; long thisBlockStartPointer = data.getFilePointer(); @@ -604,16 +618,16 @@ void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOExce void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { ByteBuffer inputBuffer = ByteBuffer.wrap(data, 0, uncompressedLength); ByteBuffersDataInput input = new ByteBuffersDataInput(List.of(inputBuffer)); - compressor.compress(input, output); + compressionMode.compressor.compress(input, output); } - void writeMetaData() throws IOException { + @Override + public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException { if (totalChunks == 0) { return; } long dataAddressesStart = data.getFilePointer(); - meta.writeLong(dataAddressesStart); meta.writeVInt(totalChunks); meta.writeVInt(maxUncompressedBlockLength); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index e4de45f0e0822..a8c013b85eae2 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -129,7 +129,7 @@ public ES819TSDBDocValuesFormat() { DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD + BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1 ); } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index da4538b0973af..891774341a7c9 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -198,7 +198,7 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return switch (entry.compression) { case NO_COMPRESS -> getUncompressedBinary(entry); - case COMPRESSED_WITH_ZSTD -> getCompressedBinary(entry); + case COMPRESSED_WITH_ZSTD_1 -> getCompressedBinary(entry); }; } @@ -1347,6 +1347,7 @@ private BinaryEntry readBinary(IndexInput meta, int version) throws IOException entry.numDocsWithField = meta.readInt(); entry.minLength = meta.readInt(); entry.maxLength = meta.readInt(); + if (compression == BinaryDVCompressionMode.NO_COMPRESS) { if (entry.minLength < entry.maxLength) { entry.addressesOffset = meta.readLong(); @@ -1357,7 +1358,7 @@ private BinaryEntry readBinary(IndexInput meta, int version) throws IOException entry.addressesLength = meta.readLong(); } } else { - if (entry.numDocsWithField > 0 || entry.minLength < entry.maxLength) { + if (entry.numDocsWithField > 0) { entry.addressesOffset = meta.readLong(); // New count of compressed addresses - the number of compressed blocks int numCompressedChunks = meta.readVInt(); From 2269f9c7f53661b6dfe97495dbd90281b50ac18a Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 27 Oct 2025 23:48:44 +0000 Subject: [PATCH 24/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/TSDBDocValuesMergeBenchmark.java | 7 ++++++- .../elasticsearch/index/codec/zstd/ZstdDecompressor.java | 3 +-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index 5078415e55485..8373e6735ec53 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -258,7 +258,12 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE ); config.setLeafSorter(DataStream.TIMESERIES_LEAF_READERS_SORTER); config.setMergePolicy(new LogByteSizeMergePolicy()); - var docValuesFormat = new ES819TSDBDocValuesFormat(4096, 512, optimizedMergeEnabled, BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1); + var docValuesFormat = new ES819TSDBDocValuesFormat( + 4096, + 512, + optimizedMergeEnabled, + BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1 + ); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override public DocValuesFormat getDocValuesFormatForField(String field) { diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java index e9762b67410c2..91995e5944ed4 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdDecompressor.java @@ -26,8 +26,7 @@ public final class ZstdDecompressor extends Decompressor { // to benefit from bulk copying and low enough to keep heap usage under control. final byte[] copyBuffer = new byte[4096]; - public ZstdDecompressor() { - } + public ZstdDecompressor() {} @Override public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException { From 1c4e9dc5f2a163115ca39356afa158c08772e17d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Tue, 28 Oct 2025 13:04:42 -0500 Subject: [PATCH 25/64] feedback --- .../tsdb/TSDBDocValuesMergeBenchmark.java | 2 +- .../codec/tsdb/BinaryDVCompressionMode.java | 4 +-- .../es819/ES819TSDBDocValuesConsumer.java | 34 ++++++++++--------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 2 +- .../es819/ES819TSDBDocValuesProducer.java | 2 +- 5 files changed, 23 insertions(+), 21 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index 8373e6735ec53..671505d43dbac 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -262,7 +262,7 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE 4096, 512, optimizedMergeEnabled, - BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1 + BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_LEVEL_1 ); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index ef3fc86debb74..5e01af6d4e4b5 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -17,7 +17,7 @@ public enum BinaryDVCompressionMode { NO_COMPRESS((byte) 0, null, null), - COMPRESSED_WITH_ZSTD_1((byte) 1, new ZstdCompressor(1), new ZstdDecompressor()); + COMPRESSED_WITH_ZSTD_LEVEL_1((byte) 1, new ZstdCompressor(1), new ZstdDecompressor()); public final byte code; public final Compressor compressor; @@ -32,7 +32,7 @@ public enum BinaryDVCompressionMode { public static BinaryDVCompressionMode fromMode(byte mode) { return switch (mode) { case 0 -> NO_COMPRESS; - case 1 -> COMPRESSED_WITH_ZSTD_1; + case 1 -> COMPRESSED_WITH_ZSTD_LEVEL_1; default -> throw new IllegalStateException("unknown compression mode [" + mode + "]"); }; } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 3b197e4579cfa..eddd1de30c632 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -576,27 +576,29 @@ public void addDoc(BytesRef v) throws IOException { @Override public void flushData() throws IOException { - if (numDocsInCurrentBlock > 0) { - totalChunks++; - long thisBlockStartPointer = data.getFilePointer(); + if (numDocsInCurrentBlock == 0) { + return; + } - // write length of string data - data.writeInt(uncompressedBlockLength); + totalChunks++; + long thisBlockStartPointer = data.getFilePointer(); - maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); - maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); + // write length of string data + data.writeInt(uncompressedBlockLength); - compressOffsets(data, numDocsInCurrentBlock); - compress(block, uncompressedBlockLength, data); + maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); + maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); - blockDocRangeAcc.addDoc(numDocsInCurrentBlock); - numDocsInCurrentBlock = 0; + compressOffsets(data, numDocsInCurrentBlock); + compress(block, uncompressedBlockLength, data); - uncompressedBlockLength = 0; - maxPointer = data.getFilePointer(); - long blockLenBytes = maxPointer - thisBlockStartPointer; - blockAddressAcc.addDoc(blockLenBytes); - } + blockDocRangeAcc.addDoc(numDocsInCurrentBlock); + numDocsInCurrentBlock = 0; + + uncompressedBlockLength = 0; + maxPointer = data.getFilePointer(); + long blockLenBytes = maxPointer - thisBlockStartPointer; + blockAddressAcc.addDoc(blockLenBytes); } void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index a8c013b85eae2..634af5135423b 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -129,7 +129,7 @@ public ES819TSDBDocValuesFormat() { DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_1 + BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_LEVEL_1 ); } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 891774341a7c9..2139bec290bf8 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -198,7 +198,7 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return switch (entry.compression) { case NO_COMPRESS -> getUncompressedBinary(entry); - case COMPRESSED_WITH_ZSTD_1 -> getCompressedBinary(entry); + case COMPRESSED_WITH_ZSTD_LEVEL_1 -> getCompressedBinary(entry); }; } From 3ddb6491061d2e5ee35e1c3dc5bbe3605cc98ebf Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Tue, 28 Oct 2025 13:26:33 -0500 Subject: [PATCH 26/64] Add WildcardRollingUpgradeIT --- .../AbstractStringTypeRollingUpgradeIT.java | 332 ++++++++++++++++++ .../MatchOnlyTextRollingUpgradeIT.java | 306 +--------------- .../upgrades/TextRollingUpgradeIT.java | 322 +---------------- .../upgrades/WildcardRollingUpgradeIT.java | 25 ++ 4 files changed, 374 insertions(+), 611 deletions(-) create mode 100644 x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java create mode 100644 x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java new file mode 100644 index 0000000000000..af3602a3433a5 --- /dev/null +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java @@ -0,0 +1,332 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.upgrades; + +import com.carrotsearch.randomizedtesting.annotations.Name; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.time.FormatNames; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.test.rest.ObjectPath; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.io.InputStream; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; + +public abstract class AbstractStringTypeRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { + + private static final String DATA_STREAM = "logs-bwc-test"; + + private static final int IGNORE_ABOVE_MAX = 256; + private static final int NUM_REQUESTS = 4; + private static final int NUM_DOCS_PER_REQUEST = 1024; + + static String BULK_ITEM_TEMPLATE = + """ + { "create": {} } + {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} + """; + + private static final String TEMPLATE = """ + { + "mappings": { + "properties": { + "@timestamp" : { + "type": "date" + }, + "method": { + "type": "keyword" + }, + "message": { + "type": "$STRING_TYPE", + "fields": { + "keyword": { + "ignore_above": $IGNORE_ABOVE, + "type": "keyword" + } + } + }, + "ip": { + "type": "ip" + }, + "length": { + "type": "long" + }, + "factor": { + "type": "double" + } + } + } + }"""; + + // when sorted, this message will appear at the top and hence can be used to validate query results + private static String smallestMessage; + + public AbstractStringTypeRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { + super(upgradedNodes); + } + + abstract String stringType(); + + public void testIndexing() throws Exception { + if (isOldCluster()) { + // given - enable logsdb and create a template + startTrial(); + enableLogsdbByDefault(); + String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); + createTemplate(DATA_STREAM, templateId, prepareTemplate()); + + // when - index some documents + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // then - verify that logsdb and synthetic source are both enabled + String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); + var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); + assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); + assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); + + // then continued - verify that the created data stream using the created template + LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); + + // when/then - run some queries and verify results + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isMixedCluster()) { + // when + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // when/then + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isUpgradedCluster()) { + // when/then + ensureGreen(DATA_STREAM); + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + + // when/then continued - force merge all shard segments into one + var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); + forceMergeRequest.addParameter("max_num_segments", "1"); + assertOK(client().performRequest(forceMergeRequest)); + + // then continued + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + query(DATA_STREAM); + } + } + + private String prepareTemplate() { + boolean shouldSetIgnoreAbove = randomBoolean(); + String templateWithType = TEMPLATE.replace("$STRING_TYPE", stringType()); + if (shouldSetIgnoreAbove) { + return templateWithType.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); + } + + // removes the entire line that defines ignore_above + return templateWithType.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); + } + + static void createTemplate(String dataStreamName, String id, String template) throws IOException { + final String INDEX_TEMPLATE = """ + { + "priority": 500, + "index_patterns": ["$DATASTREAM"], + "template": $TEMPLATE, + "data_stream": { + } + }"""; + var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); + putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); + assertOK(client().performRequest(putIndexTemplateRequest)); + } + + private void bulkIndex(int numRequest, int numDocs) throws Exception { + String firstIndex = null; + Instant startTime = Instant.now().minusSeconds(60 * 60); + + for (int i = 0; i < numRequest; i++) { + var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); + bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); + bulkRequest.addParameter("refresh", "true"); + + var response = client().performRequest(bulkRequest); + var responseBody = entityAsMap(response); + + assertOK(response); + assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); + if (firstIndex == null) { + firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); + } + } + } + + private String bulkIndexRequestBody(int numDocs, Instant startTime) { + StringBuilder requestBody = new StringBuilder(); + + for (int j = 0; j < numDocs; j++) { + String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. + String methodName = "method" + j % 5; + String ip = NetworkAddress.format(randomIp(true)); + String message = randomAlphasDelimitedBySpace(10, 1, 15); + recordSmallestMessage(message); + long length = randomLong(); + double factor = randomDouble(); + + requestBody.append( + BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) + .replace("$host", hostName) + .replace("$method", methodName) + .replace("$ip", ip) + .replace("$message", message) + .replace("$length", Long.toString(length)) + .replace("$factor", Double.toString(factor)) + ); + requestBody.append('\n'); + + startTime = startTime.plusMillis(1); + } + + return requestBody.toString(); + } + + /** + * Generates a string containing a random number of random length alphas, all delimited by space. + */ + public static String randomAlphasDelimitedBySpace(int maxAlphas, int minCodeUnits, int maxCodeUnits) { + int numAlphas = randomIntBetween(1, maxAlphas); + List alphas = new ArrayList<>(numAlphas); + for (int i = 0; i < numAlphas; i++) { + alphas.add(randomAlphaOfLengthBetween(minCodeUnits, maxCodeUnits)); + } + return String.join(" ", alphas); + } + + private void recordSmallestMessage(final String message) { + if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { + smallestMessage = message; + } + } + + private void search(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "size": 500 + } + """); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); + assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); + } + + private void phraseSearch(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "query": { + "match_phrase": { + "message": "$smallestMessage" + } + } + } + """.replace("$smallestMessage", smallestMessage)); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); + } + + private void query(String dataStreamName) throws Exception { + var queryRequest = new Request("POST", "/_query"); + queryRequest.addParameter("pretty", "true"); + queryRequest.setJsonEntity(""" + { + "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" + } + """.replace("$ds", dataStreamName)); + var response = client().performRequest(queryRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); + assertThat(column1, equalTo("max(length)")); + String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); + assertThat(column2, equalTo("max(factor)")); + String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); + assertThat(column3, equalTo("message")); + + Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); + assertThat(maxRx, notNullValue()); + Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); + assertThat(maxTx, notNullValue()); + String key = ObjectPath.evaluate(responseBody, "values.0.2"); + assertThat(key, equalTo(smallestMessage)); + } + + protected static void startTrial() throws IOException { + Request startTrial = new Request("POST", "/_license/start_trial"); + startTrial.addParameter("acknowledge", "true"); + try { + assertOK(client().performRequest(startTrial)); + } catch (ResponseException e) { + var responseBody = entityAsMap(e.getResponse()); + String error = ObjectPath.evaluate(responseBody, "error_message"); + assertThat(error, containsString("Trial was already activated.")); + } + } + + static Map getIndexSettingsWithDefaults(String index) throws IOException { + Request request = new Request("GET", "/" + index + "/_settings"); + request.addParameter("flat_settings", "true"); + request.addParameter("include_defaults", "true"); + Response response = client().performRequest(request); + try (InputStream is = response.getEntity().getContent()) { + return XContentHelper.convertToMap( + XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), + is, + true + ); + } + } + + static String formatInstant(Instant instant) { + return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); + } + +} diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java index 42138462f1397..09a65f93c9857 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java @@ -11,315 +11,25 @@ import com.carrotsearch.randomizedtesting.annotations.Name; -import org.elasticsearch.client.Request; -import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.common.time.FormatNames; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.mapper.MapperFeatures; -import org.elasticsearch.test.rest.ObjectPath; -import org.elasticsearch.xcontent.XContentType; -import java.io.IOException; -import java.io.InputStream; -import java.time.Instant; -import java.util.List; -import java.util.Locale; -import java.util.Map; - -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; -import static org.elasticsearch.upgrades.TextRollingUpgradeIT.randomAlphasDelimitedBySpace; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.notNullValue; - -public class MatchOnlyTextRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { - - private static final String DATA_STREAM = "logs-bwc-test"; - - private static final int IGNORE_ABOVE_MAX = 256; - private static final int NUM_REQUESTS = 4; - private static final int NUM_DOCS_PER_REQUEST = 1024; - - static String BULK_ITEM_TEMPLATE = - """ - { "create": {} } - {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} - """; - - private static final String TEMPLATE = """ - { - "mappings": { - "properties": { - "@timestamp" : { - "type": "date" - }, - "method": { - "type": "keyword" - }, - "message": { - "type": "match_only_text", - "fields": { - "keyword": { - "ignore_above": $IGNORE_ABOVE, - "type": "keyword" - } - } - }, - "ip": { - "type": "ip" - }, - "length": { - "type": "long" - }, - "factor": { - "type": "double" - } - } - } - }"""; - - // when sorted, this message will appear at the top and hence can be used to validate query results - private static String smallestMessage; +public class MatchOnlyTextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public MatchOnlyTextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { super(upgradedNodes); } + @Override + public String stringType() { + return "match_only_text"; + } + + @Override public void testIndexing() throws Exception { assumeTrue( "Match only text block loader fix is not present in this cluster", oldClusterHasFeature(MapperFeatures.MATCH_ONLY_TEXT_BLOCK_LOADER_FIX) ); - - if (isOldCluster()) { - // given - enable logsdb and create a template - startTrial(); - enableLogsdbByDefault(); - String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); - createTemplate(DATA_STREAM, templateId, prepareTemplate()); - - // when - index some documents - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // then - verify that logsdb and synthetic source are both enabled - String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); - var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); - assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); - assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); - - // then continued - verify that the created data stream uses the created template - LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); - - // when/then - run some queries and verify results - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isMixedCluster()) { - // when - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // when/then - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isUpgradedCluster()) { - // when/then - ensureGreen(DATA_STREAM); - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - - // when/then continued - force merge all shard segments into one - var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); - forceMergeRequest.addParameter("max_num_segments", "1"); - assertOK(client().performRequest(forceMergeRequest)); - - // then continued - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - query(DATA_STREAM); - } - } - - private String prepareTemplate() { - boolean shouldSetIgnoreAbove = randomBoolean(); - if (shouldSetIgnoreAbove) { - return TEMPLATE.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); - } - - // removes the entire line that defines ignore_above - return TEMPLATE.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); + super.testIndexing(); } - - static void createTemplate(String dataStreamName, String id, String template) throws IOException { - final String INDEX_TEMPLATE = """ - { - "priority": 500, - "index_patterns": ["$DATASTREAM"], - "template": $TEMPLATE, - "data_stream": { - } - }"""; - var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); - putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); - assertOK(client().performRequest(putIndexTemplateRequest)); - } - - private void bulkIndex(int numRequest, int numDocs) throws Exception { - String firstIndex = null; - Instant startTime = Instant.now().minusSeconds(60 * 60); - - for (int i = 0; i < numRequest; i++) { - var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); - bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); - bulkRequest.addParameter("refresh", "true"); - - var response = client().performRequest(bulkRequest); - var responseBody = entityAsMap(response); - - assertOK(response); - assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); - if (firstIndex == null) { - firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); - } - } - } - - private String bulkIndexRequestBody(int numDocs, Instant startTime) { - StringBuilder requestBody = new StringBuilder(); - - for (int j = 0; j < numDocs; j++) { - String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. - String methodName = "method" + j % 5; - String ip = NetworkAddress.format(randomIp(true)); - String message = randomAlphasDelimitedBySpace(10, 1, 15); - recordSmallestMessage(message); - long length = randomLong(); - double factor = randomDouble(); - - requestBody.append( - BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) - .replace("$host", hostName) - .replace("$method", methodName) - .replace("$ip", ip) - .replace("$message", message) - .replace("$length", Long.toString(length)) - .replace("$factor", Double.toString(factor)) - ); - requestBody.append('\n'); - - startTime = startTime.plusMillis(1); - } - - return requestBody.toString(); - } - - private void recordSmallestMessage(final String message) { - if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { - smallestMessage = message; - } - } - - void search(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "size": 500 - } - """); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); - assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); - } - - private void phraseSearch(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "query": { - "match_phrase": { - "message": "$smallestMessage" - } - } - } - """.replace("$smallestMessage", smallestMessage)); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); - } - - private void query(String dataStreamName) throws Exception { - var queryRequest = new Request("POST", "/_query"); - queryRequest.addParameter("pretty", "true"); - queryRequest.setJsonEntity(""" - { - "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" - } - """.replace("$ds", dataStreamName)); - var response = client().performRequest(queryRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); - assertThat(column1, equalTo("max(length)")); - String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); - assertThat(column2, equalTo("max(factor)")); - String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); - assertThat(column3, equalTo("message")); - - Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); - assertThat(maxRx, notNullValue()); - Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); - assertThat(maxTx, notNullValue()); - String key = ObjectPath.evaluate(responseBody, "values.0.2"); - assertThat(key, equalTo(smallestMessage)); - } - - protected static void startTrial() throws IOException { - Request startTrial = new Request("POST", "/_license/start_trial"); - startTrial.addParameter("acknowledge", "true"); - try { - assertOK(client().performRequest(startTrial)); - } catch (ResponseException e) { - var responseBody = entityAsMap(e.getResponse()); - String error = ObjectPath.evaluate(responseBody, "error_message"); - assertThat(error, containsString("Trial was already activated.")); - } - } - - static Map getIndexSettingsWithDefaults(String index) throws IOException { - Request request = new Request("GET", "/" + index + "/_settings"); - request.addParameter("flat_settings", "true"); - request.addParameter("include_defaults", "true"); - Response response = client().performRequest(request); - try (InputStream is = response.getEntity().getContent()) { - return XContentHelper.convertToMap( - XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), - is, - true - ); - } - } - - static String formatInstant(Instant instant) { - return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); - } - } diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java index a2bbd4471bab1..c6c35625375da 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java @@ -1,329 +1,25 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.upgrades; import com.carrotsearch.randomizedtesting.annotations.Name; -import org.elasticsearch.client.Request; -import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.common.time.FormatNames; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.test.rest.ObjectPath; -import org.elasticsearch.xcontent.XContentType; -import java.io.IOException; -import java.io.InputStream; -import java.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.Locale; -import java.util.Map; - -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.notNullValue; - -public class TextRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { - - private static final String DATA_STREAM = "logs-bwc-test"; - - private static final int IGNORE_ABOVE_MAX = 256; - private static final int NUM_REQUESTS = 4; - private static final int NUM_DOCS_PER_REQUEST = 1024; - - static String BULK_ITEM_TEMPLATE = - """ - { "create": {} } - {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} - """; - - private static final String TEMPLATE = """ - { - "mappings": { - "properties": { - "@timestamp" : { - "type": "date" - }, - "method": { - "type": "keyword" - }, - "message": { - "type": "text", - "fields": { - "keyword": { - "ignore_above": $IGNORE_ABOVE, - "type": "keyword" - } - } - }, - "ip": { - "type": "ip" - }, - "length": { - "type": "long" - }, - "factor": { - "type": "double" - } - } - } - }"""; - - // when sorted, this message will appear at the top and hence can be used to validate query results - private static String smallestMessage; +public class TextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public TextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { super(upgradedNodes); } - public void testIndexing() throws Exception { - if (isOldCluster()) { - // given - enable logsdb and create a template - startTrial(); - enableLogsdbByDefault(); - String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); - createTemplate(DATA_STREAM, templateId, prepareTemplate()); - - // when - index some documents - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // then - verify that logsdb and synthetic source are both enabled - String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); - var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); - assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); - assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); - - // then continued - verify that the created data stream using the created template - LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); - - // when/then - run some queries and verify results - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isMixedCluster()) { - // when - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // when/then - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isUpgradedCluster()) { - // when/then - ensureGreen(DATA_STREAM); - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - - // when/then continued - force merge all shard segments into one - var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); - forceMergeRequest.addParameter("max_num_segments", "1"); - assertOK(client().performRequest(forceMergeRequest)); - - // then continued - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - query(DATA_STREAM); - } - } - - private String prepareTemplate() { - boolean shouldSetIgnoreAbove = randomBoolean(); - if (shouldSetIgnoreAbove) { - return TEMPLATE.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); - } - - // removes the entire line that defines ignore_above - return TEMPLATE.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); - } - - static void createTemplate(String dataStreamName, String id, String template) throws IOException { - final String INDEX_TEMPLATE = """ - { - "priority": 500, - "index_patterns": ["$DATASTREAM"], - "template": $TEMPLATE, - "data_stream": { - } - }"""; - var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); - putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); - assertOK(client().performRequest(putIndexTemplateRequest)); - } - - private void bulkIndex(int numRequest, int numDocs) throws Exception { - String firstIndex = null; - Instant startTime = Instant.now().minusSeconds(60 * 60); - - for (int i = 0; i < numRequest; i++) { - var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); - bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); - bulkRequest.addParameter("refresh", "true"); - - var response = client().performRequest(bulkRequest); - var responseBody = entityAsMap(response); - - assertOK(response); - assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); - if (firstIndex == null) { - firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); - } - } - } - - private String bulkIndexRequestBody(int numDocs, Instant startTime) { - StringBuilder requestBody = new StringBuilder(); - - for (int j = 0; j < numDocs; j++) { - String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. - String methodName = "method" + j % 5; - String ip = NetworkAddress.format(randomIp(true)); - String message = randomAlphasDelimitedBySpace(10, 1, 15); - recordSmallestMessage(message); - long length = randomLong(); - double factor = randomDouble(); - - requestBody.append( - BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) - .replace("$host", hostName) - .replace("$method", methodName) - .replace("$ip", ip) - .replace("$message", message) - .replace("$length", Long.toString(length)) - .replace("$factor", Double.toString(factor)) - ); - requestBody.append('\n'); - - startTime = startTime.plusMillis(1); - } - - return requestBody.toString(); - } - - /** - * Generates a string containing a random number of random length alphas, all delimited by space. - */ - public static String randomAlphasDelimitedBySpace(int maxAlphas, int minCodeUnits, int maxCodeUnits) { - int numAlphas = randomIntBetween(1, maxAlphas); - List alphas = new ArrayList<>(numAlphas); - for (int i = 0; i < numAlphas; i++) { - alphas.add(randomAlphaOfLengthBetween(minCodeUnits, maxCodeUnits)); - } - return String.join(" ", alphas); + @Override + public String stringType() { + return "text"; } - - private void recordSmallestMessage(final String message) { - if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { - smallestMessage = message; - } - } - - private void search(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "size": 500 - } - """); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); - assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); - } - - private void phraseSearch(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "query": { - "match_phrase": { - "message": "$smallestMessage" - } - } - } - """.replace("$smallestMessage", smallestMessage)); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); - } - - private void query(String dataStreamName) throws Exception { - var queryRequest = new Request("POST", "/_query"); - queryRequest.addParameter("pretty", "true"); - queryRequest.setJsonEntity(""" - { - "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" - } - """.replace("$ds", dataStreamName)); - var response = client().performRequest(queryRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); - assertThat(column1, equalTo("max(length)")); - String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); - assertThat(column2, equalTo("max(factor)")); - String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); - assertThat(column3, equalTo("message")); - - Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); - assertThat(maxRx, notNullValue()); - Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); - assertThat(maxTx, notNullValue()); - String key = ObjectPath.evaluate(responseBody, "values.0.2"); - assertThat(key, equalTo(smallestMessage)); - } - - protected static void startTrial() throws IOException { - Request startTrial = new Request("POST", "/_license/start_trial"); - startTrial.addParameter("acknowledge", "true"); - try { - assertOK(client().performRequest(startTrial)); - } catch (ResponseException e) { - var responseBody = entityAsMap(e.getResponse()); - String error = ObjectPath.evaluate(responseBody, "error_message"); - assertThat(error, containsString("Trial was already activated.")); - } - } - - static Map getIndexSettingsWithDefaults(String index) throws IOException { - Request request = new Request("GET", "/" + index + "/_settings"); - request.addParameter("flat_settings", "true"); - request.addParameter("include_defaults", "true"); - Response response = client().performRequest(request); - try (InputStream is = response.getEntity().getContent()) { - return XContentHelper.convertToMap( - XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), - is, - true - ); - } - } - - static String formatInstant(Instant instant) { - return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); - } - } diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java new file mode 100644 index 0000000000000..e2ae2bf23e955 --- /dev/null +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java @@ -0,0 +1,25 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.upgrades; + +import com.carrotsearch.randomizedtesting.annotations.Name; + + +public class WildcardRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { + + public WildcardRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { + super(upgradedNodes); + } + + @Override + public String stringType() { + return "wildcard"; + } +} From dbcd1c6adac4b220691d8fc1e5b2eadc4981f453 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Tue, 28 Oct 2025 19:32:21 -0500 Subject: [PATCH 27/64] need new compressor/decompressor for new block writer --- .../codec/tsdb/BinaryDVCompressionMode.java | 18 ++++----- .../es819/ES819TSDBDocValuesConsumer.java | 7 ++-- .../es819/ES819TSDBDocValuesProducer.java | 7 +++- .../codec/zstd/Zstd814StoredFieldsFormat.java | 26 ------------- .../index/codec/zstd/ZstdCompressionMode.java | 37 +++++++++++++++++++ 5 files changed, 54 insertions(+), 41 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressionMode.java diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 5e01af6d4e4b5..9122340fa0755 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -9,24 +9,20 @@ package org.elasticsearch.index.codec.tsdb; -import org.apache.lucene.codecs.compressing.Compressor; -import org.apache.lucene.codecs.compressing.Decompressor; -import org.elasticsearch.index.codec.zstd.ZstdCompressor; -import org.elasticsearch.index.codec.zstd.ZstdDecompressor; +import org.apache.lucene.codecs.compressing.CompressionMode; +import org.elasticsearch.index.codec.zstd.ZstdCompressionMode; public enum BinaryDVCompressionMode { - NO_COMPRESS((byte) 0, null, null), - COMPRESSED_WITH_ZSTD_LEVEL_1((byte) 1, new ZstdCompressor(1), new ZstdDecompressor()); + NO_COMPRESS((byte) 0, null), + COMPRESSED_WITH_ZSTD_LEVEL_1((byte) 1, new ZstdCompressionMode(1)); public final byte code; - public final Compressor compressor; - public final Decompressor decompressor; + public final CompressionMode compressionMode; - BinaryDVCompressionMode(byte code, Compressor compressor, Decompressor decompressor) { + BinaryDVCompressionMode(byte code, CompressionMode compressionMode) { this.code = code; - this.compressor = compressor; - this.decompressor = decompressor; + this.compressionMode = compressionMode; } public static BinaryDVCompressionMode fromMode(byte mode) { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index eddd1de30c632..3bf951285b6d7 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -11,6 +11,7 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.compressing.Compressor; import org.apache.lucene.codecs.lucene90.IndexedDISI; import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.DocValues; @@ -528,7 +529,7 @@ private class CompressedBinaryBlockWriter implements BinaryWriter { static final int MIN_BLOCK_BYTES = 256 * 1024; static final int START_BLOCK_DOCS = 1024; - final BinaryDVCompressionMode compressionMode; + final Compressor compressor; final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); final long[] docOffsetsCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; @@ -547,7 +548,7 @@ private class CompressedBinaryBlockWriter implements BinaryWriter { final DelayedOffsetAccumulator blockDocRangeAcc; CompressedBinaryBlockWriter(BinaryDVCompressionMode compressionMode) throws IOException { - this.compressionMode = compressionMode; + this.compressor = compressionMode.compressionMode.newCompressor(); long blockAddressesStart = data.getFilePointer(); blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-addresses", blockAddressesStart); @@ -620,7 +621,7 @@ void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOExce void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { ByteBuffer inputBuffer = ByteBuffer.wrap(data, 0, uncompressedLength); ByteBuffersDataInput input = new ByteBuffersDataInput(List.of(inputBuffer)); - compressionMode.compressor.compress(input, output); + compressor.compress(input, output); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 2139bec290bf8..fd5ebde9f2396 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -11,6 +11,7 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.compressing.Decompressor; import org.apache.lucene.codecs.lucene90.IndexedDISI; import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.BinaryDocValues; @@ -287,6 +288,7 @@ private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOExceptio final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new DenseBinaryDocValues(maxDoc) { final BinaryDecoder decoder = new BinaryDecoder( + entry.compression.compressionMode.newDecompressor(), addresses, docRanges, data.clone(), @@ -316,6 +318,7 @@ public BytesRef binaryValue() throws IOException { final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new SparseBinaryDocValues(disi) { final BinaryDecoder decoder = new BinaryDecoder( + entry.compression.compressionMode.newDecompressor(), addresses, docRanges, data.clone(), @@ -347,15 +350,17 @@ static final class BinaryDecoder { private final BytesRef uncompressedBytesRef; private long startDocNumForBlock = -1; private long limitDocNumForBlock = -1; - private final ZstdDecompressor decompressor = new ZstdDecompressor(); + private final Decompressor decompressor; BinaryDecoder( + Decompressor decompressor, LongValues addresses, DirectMonotonicReader docRanges, IndexInput compressedData, int biggestUncompressedBlockSize, int maxNumDocsInAnyBlock ) { + this.decompressor = decompressor; this.addresses = addresses; this.docRanges = docRanges; this.compressedData = compressedData; diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java index cb7b570ff303d..8fc8042fa7fac 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/Zstd814StoredFieldsFormat.java @@ -10,9 +10,6 @@ package org.elasticsearch.index.codec.zstd; import org.apache.lucene.codecs.StoredFieldsWriter; -import org.apache.lucene.codecs.compressing.CompressionMode; -import org.apache.lucene.codecs.compressing.Compressor; -import org.apache.lucene.codecs.compressing.Decompressor; import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.store.Directory; @@ -80,27 +77,4 @@ public Mode getMode() { return mode; } - private static class ZstdCompressionMode extends CompressionMode { - private final int level; - - ZstdCompressionMode(int level) { - this.level = level; - } - - @Override - public Compressor newCompressor() { - return new ZstdCompressor(level); - } - - @Override - public Decompressor newDecompressor() { - return new ZstdDecompressor(); - } - - @Override - public String toString() { - return "ZSTD(level=" + level + ")"; - } - } - } diff --git a/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressionMode.java new file mode 100644 index 0000000000000..81d1c6fde5301 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/zstd/ZstdCompressionMode.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.zstd; + +import org.apache.lucene.codecs.compressing.CompressionMode; +import org.apache.lucene.codecs.compressing.Compressor; +import org.apache.lucene.codecs.compressing.Decompressor; + +public class ZstdCompressionMode extends CompressionMode { + private final int level; + + public ZstdCompressionMode(int level) { + this.level = level; + } + + @Override + public Compressor newCompressor() { + return new ZstdCompressor(level); + } + + @Override + public Decompressor newDecompressor() { + return new ZstdDecompressor(); + } + + @Override + public String toString() { + return "ZSTD(level=" + level + ")"; + } +} From 5537d8c6f85e2b8c669df2a729744efc452c28fa Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 29 Oct 2025 02:53:41 +0000 Subject: [PATCH 28/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 1 - .../java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java | 1 - .../org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java | 1 - 3 files changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index fd5ebde9f2396..4bd9056e683df 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -48,7 +48,6 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; -import org.elasticsearch.index.codec.zstd.ZstdDecompressor; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.blockloader.docvalues.BlockDocValuesReader; diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java index c6c35625375da..4d13777699d36 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java @@ -11,7 +11,6 @@ import com.carrotsearch.randomizedtesting.annotations.Name; - public class TextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public TextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java index e2ae2bf23e955..afc4217a6d2f3 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java @@ -11,7 +11,6 @@ import com.carrotsearch.randomizedtesting.annotations.Name; - public class WildcardRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public WildcardRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { From d7fce75c868272cf4c1b93f4f92213178a6adaf9 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 11:38:54 -0500 Subject: [PATCH 29/64] Cleanup binaryWriter interface --- .../es819/ES819TSDBDocValuesConsumer.java | 73 ++++++++----------- 1 file changed, 29 insertions(+), 44 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 3bf951285b6d7..5dec4fe36b1f3 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -350,7 +350,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th assert maxLength >= minLength; if (binaryDVCompressionMode == BinaryDVCompressionMode.NO_COMPRESS) { var offsetsAccumulator = maxLength > minLength ? new OffsetsAccumulator(dir, context, data, numDocsWithField) : null; - binaryWriter = new DirectBinaryWriterOptimized(offsetsAccumulator); + binaryWriter = new DirectBinaryWriter(offsetsAccumulator, null); } else { binaryWriter = new CompressedBinaryBlockWriter(binaryDVCompressionMode); } @@ -396,7 +396,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th BinaryWriter binaryWriter = null; try { if (binaryDVCompressionMode == BinaryDVCompressionMode.NO_COMPRESS) { - binaryWriter = new DirectBinaryWriterUnoptimized(valuesProducer, field); + binaryWriter = new DirectBinaryWriter(null, valuesProducer.getBinary(field)); } else { binaryWriter = new CompressedBinaryBlockWriter(binaryDVCompressionMode); } @@ -451,7 +451,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th } } - private interface BinaryWriter extends Closeable { + private sealed interface BinaryWriter extends Closeable { void addDoc(BytesRef v) throws IOException; default void flushData() throws IOException {} @@ -462,11 +462,13 @@ default void writeAddressMetadata(int minLength, int maxLength, int numDocsWithF default void close() throws IOException {} } - private class DirectBinaryWriterOptimized implements BinaryWriter { + private final class DirectBinaryWriter implements BinaryWriter { final OffsetsAccumulator offsetsAccumulator; + final BinaryDocValues values; - private DirectBinaryWriterOptimized(OffsetsAccumulator offsetsAccumulator) { + private DirectBinaryWriter(OffsetsAccumulator offsetsAccumulator, BinaryDocValues values) { this.offsetsAccumulator = offsetsAccumulator; + this.values = values; } @Override @@ -480,52 +482,35 @@ public void addDoc(BytesRef v) throws IOException { @Override public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException { if (offsetsAccumulator != null) { + // If optimized merging and minLength > maxLength offsetsAccumulator.build(meta, data); - } - } - } - - private class DirectBinaryWriterUnoptimized implements BinaryWriter { - private final DocValuesProducer valuesProducer; - private final FieldInfo field; - - private DirectBinaryWriterUnoptimized(DocValuesProducer valuesProducer, FieldInfo field) { - this.valuesProducer = valuesProducer; - this.field = field; - } - - @Override - public void addDoc(BytesRef v) throws IOException { - data.writeBytes(v.bytes, v.offset, v.length); - } - - @Override - public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithField) throws IOException { - if (maxLength > minLength) { - long addressStart = data.getFilePointer(); - meta.writeLong(addressStart); - meta.writeVInt(ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT); - - final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( - meta, - data, - numDocsWithField + 1, - ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT - ); - long addr = 0; - writer.add(addr); - var values = valuesProducer.getBinary(field); - for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { - addr += values.binaryValue().length; + } else if (values != null) { + if (maxLength > minLength) { + // If optimized merging and minLength > maxLength + long addressStart = data.getFilePointer(); + meta.writeLong(addressStart); + meta.writeVInt(ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT); + + final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( + meta, + data, + numDocsWithField + 1, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + ); + long addr = 0; writer.add(addr); + 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() - addressStart); } - writer.finish(); - meta.writeLong(data.getFilePointer() - addressStart); } } } - private class CompressedBinaryBlockWriter implements BinaryWriter { + private final class CompressedBinaryBlockWriter implements BinaryWriter { static final int MIN_BLOCK_BYTES = 256 * 1024; static final int START_BLOCK_DOCS = 1024; From bb8361cfc48993482f928a970b1452e7dd99f4e4 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 11:40:27 -0500 Subject: [PATCH 30/64] Revert "[CI] Auto commit changes from spotless" This reverts commit 5537d8c6f85e2b8c669df2a729744efc452c28fa. --- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 1 + .../java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java | 1 + .../org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java | 1 + 3 files changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 4bd9056e683df..fd5ebde9f2396 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -48,6 +48,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; +import org.elasticsearch.index.codec.zstd.ZstdDecompressor; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.blockloader.docvalues.BlockDocValuesReader; diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java index 4d13777699d36..c6c35625375da 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java @@ -11,6 +11,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; + public class TextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public TextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java index afc4217a6d2f3..e2ae2bf23e955 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java @@ -11,6 +11,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; + public class WildcardRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { public WildcardRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { From aa3d44fec64452bde0030264866bba2a3f8d21ac Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 11:40:49 -0500 Subject: [PATCH 31/64] Revert "Add WildcardRollingUpgradeIT" This reverts commit 3ddb6491061d2e5ee35e1c3dc5bbe3605cc98ebf. --- .../AbstractStringTypeRollingUpgradeIT.java | 332 ------------------ .../MatchOnlyTextRollingUpgradeIT.java | 306 +++++++++++++++- .../upgrades/TextRollingUpgradeIT.java | 322 ++++++++++++++++- .../upgrades/WildcardRollingUpgradeIT.java | 25 -- 4 files changed, 611 insertions(+), 374 deletions(-) delete mode 100644 x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java delete mode 100644 x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java deleted file mode 100644 index af3602a3433a5..0000000000000 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/AbstractStringTypeRollingUpgradeIT.java +++ /dev/null @@ -1,332 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.upgrades; - -import com.carrotsearch.randomizedtesting.annotations.Name; - -import org.elasticsearch.client.Request; -import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.common.time.FormatNames; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.test.rest.ObjectPath; -import org.elasticsearch.xcontent.XContentType; - -import java.io.IOException; -import java.io.InputStream; -import java.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.Locale; -import java.util.Map; - -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; -import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.notNullValue; - -public abstract class AbstractStringTypeRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { - - private static final String DATA_STREAM = "logs-bwc-test"; - - private static final int IGNORE_ABOVE_MAX = 256; - private static final int NUM_REQUESTS = 4; - private static final int NUM_DOCS_PER_REQUEST = 1024; - - static String BULK_ITEM_TEMPLATE = - """ - { "create": {} } - {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} - """; - - private static final String TEMPLATE = """ - { - "mappings": { - "properties": { - "@timestamp" : { - "type": "date" - }, - "method": { - "type": "keyword" - }, - "message": { - "type": "$STRING_TYPE", - "fields": { - "keyword": { - "ignore_above": $IGNORE_ABOVE, - "type": "keyword" - } - } - }, - "ip": { - "type": "ip" - }, - "length": { - "type": "long" - }, - "factor": { - "type": "double" - } - } - } - }"""; - - // when sorted, this message will appear at the top and hence can be used to validate query results - private static String smallestMessage; - - public AbstractStringTypeRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { - super(upgradedNodes); - } - - abstract String stringType(); - - public void testIndexing() throws Exception { - if (isOldCluster()) { - // given - enable logsdb and create a template - startTrial(); - enableLogsdbByDefault(); - String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); - createTemplate(DATA_STREAM, templateId, prepareTemplate()); - - // when - index some documents - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // then - verify that logsdb and synthetic source are both enabled - String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); - var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); - assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); - assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); - - // then continued - verify that the created data stream using the created template - LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); - - // when/then - run some queries and verify results - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isMixedCluster()) { - // when - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - - // when/then - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - } else if (isUpgradedCluster()) { - // when/then - ensureGreen(DATA_STREAM); - bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); - search(DATA_STREAM); - phraseSearch(DATA_STREAM); - query(DATA_STREAM); - - // when/then continued - force merge all shard segments into one - var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); - forceMergeRequest.addParameter("max_num_segments", "1"); - assertOK(client().performRequest(forceMergeRequest)); - - // then continued - ensureGreen(DATA_STREAM); - search(DATA_STREAM); - query(DATA_STREAM); - } - } - - private String prepareTemplate() { - boolean shouldSetIgnoreAbove = randomBoolean(); - String templateWithType = TEMPLATE.replace("$STRING_TYPE", stringType()); - if (shouldSetIgnoreAbove) { - return templateWithType.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); - } - - // removes the entire line that defines ignore_above - return templateWithType.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); - } - - static void createTemplate(String dataStreamName, String id, String template) throws IOException { - final String INDEX_TEMPLATE = """ - { - "priority": 500, - "index_patterns": ["$DATASTREAM"], - "template": $TEMPLATE, - "data_stream": { - } - }"""; - var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); - putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); - assertOK(client().performRequest(putIndexTemplateRequest)); - } - - private void bulkIndex(int numRequest, int numDocs) throws Exception { - String firstIndex = null; - Instant startTime = Instant.now().minusSeconds(60 * 60); - - for (int i = 0; i < numRequest; i++) { - var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); - bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); - bulkRequest.addParameter("refresh", "true"); - - var response = client().performRequest(bulkRequest); - var responseBody = entityAsMap(response); - - assertOK(response); - assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); - if (firstIndex == null) { - firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); - } - } - } - - private String bulkIndexRequestBody(int numDocs, Instant startTime) { - StringBuilder requestBody = new StringBuilder(); - - for (int j = 0; j < numDocs; j++) { - String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. - String methodName = "method" + j % 5; - String ip = NetworkAddress.format(randomIp(true)); - String message = randomAlphasDelimitedBySpace(10, 1, 15); - recordSmallestMessage(message); - long length = randomLong(); - double factor = randomDouble(); - - requestBody.append( - BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) - .replace("$host", hostName) - .replace("$method", methodName) - .replace("$ip", ip) - .replace("$message", message) - .replace("$length", Long.toString(length)) - .replace("$factor", Double.toString(factor)) - ); - requestBody.append('\n'); - - startTime = startTime.plusMillis(1); - } - - return requestBody.toString(); - } - - /** - * Generates a string containing a random number of random length alphas, all delimited by space. - */ - public static String randomAlphasDelimitedBySpace(int maxAlphas, int minCodeUnits, int maxCodeUnits) { - int numAlphas = randomIntBetween(1, maxAlphas); - List alphas = new ArrayList<>(numAlphas); - for (int i = 0; i < numAlphas; i++) { - alphas.add(randomAlphaOfLengthBetween(minCodeUnits, maxCodeUnits)); - } - return String.join(" ", alphas); - } - - private void recordSmallestMessage(final String message) { - if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { - smallestMessage = message; - } - } - - private void search(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "size": 500 - } - """); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); - assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); - } - - private void phraseSearch(String dataStreamName) throws Exception { - var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); - searchRequest.addParameter("pretty", "true"); - searchRequest.setJsonEntity(""" - { - "query": { - "match_phrase": { - "message": "$smallestMessage" - } - } - } - """.replace("$smallestMessage", smallestMessage)); - var response = client().performRequest(searchRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); - } - - private void query(String dataStreamName) throws Exception { - var queryRequest = new Request("POST", "/_query"); - queryRequest.addParameter("pretty", "true"); - queryRequest.setJsonEntity(""" - { - "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" - } - """.replace("$ds", dataStreamName)); - var response = client().performRequest(queryRequest); - assertOK(response); - var responseBody = entityAsMap(response); - logger.info("{}", responseBody); - - String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); - assertThat(column1, equalTo("max(length)")); - String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); - assertThat(column2, equalTo("max(factor)")); - String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); - assertThat(column3, equalTo("message")); - - Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); - assertThat(maxRx, notNullValue()); - Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); - assertThat(maxTx, notNullValue()); - String key = ObjectPath.evaluate(responseBody, "values.0.2"); - assertThat(key, equalTo(smallestMessage)); - } - - protected static void startTrial() throws IOException { - Request startTrial = new Request("POST", "/_license/start_trial"); - startTrial.addParameter("acknowledge", "true"); - try { - assertOK(client().performRequest(startTrial)); - } catch (ResponseException e) { - var responseBody = entityAsMap(e.getResponse()); - String error = ObjectPath.evaluate(responseBody, "error_message"); - assertThat(error, containsString("Trial was already activated.")); - } - } - - static Map getIndexSettingsWithDefaults(String index) throws IOException { - Request request = new Request("GET", "/" + index + "/_settings"); - request.addParameter("flat_settings", "true"); - request.addParameter("include_defaults", "true"); - Response response = client().performRequest(request); - try (InputStream is = response.getEntity().getContent()) { - return XContentHelper.convertToMap( - XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), - is, - true - ); - } - } - - static String formatInstant(Instant instant) { - return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); - } - -} diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java index 09a65f93c9857..42138462f1397 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java @@ -11,25 +11,315 @@ import com.carrotsearch.randomizedtesting.annotations.Name; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.time.FormatNames; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.mapper.MapperFeatures; +import org.elasticsearch.test.rest.ObjectPath; +import org.elasticsearch.xcontent.XContentType; -public class MatchOnlyTextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { +import java.io.IOException; +import java.io.InputStream; +import java.time.Instant; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; +import static org.elasticsearch.upgrades.TextRollingUpgradeIT.randomAlphasDelimitedBySpace; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; + +public class MatchOnlyTextRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { + + private static final String DATA_STREAM = "logs-bwc-test"; + + private static final int IGNORE_ABOVE_MAX = 256; + private static final int NUM_REQUESTS = 4; + private static final int NUM_DOCS_PER_REQUEST = 1024; + + static String BULK_ITEM_TEMPLATE = + """ + { "create": {} } + {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} + """; + + private static final String TEMPLATE = """ + { + "mappings": { + "properties": { + "@timestamp" : { + "type": "date" + }, + "method": { + "type": "keyword" + }, + "message": { + "type": "match_only_text", + "fields": { + "keyword": { + "ignore_above": $IGNORE_ABOVE, + "type": "keyword" + } + } + }, + "ip": { + "type": "ip" + }, + "length": { + "type": "long" + }, + "factor": { + "type": "double" + } + } + } + }"""; + + // when sorted, this message will appear at the top and hence can be used to validate query results + private static String smallestMessage; public MatchOnlyTextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { super(upgradedNodes); } - @Override - public String stringType() { - return "match_only_text"; - } - - @Override public void testIndexing() throws Exception { assumeTrue( "Match only text block loader fix is not present in this cluster", oldClusterHasFeature(MapperFeatures.MATCH_ONLY_TEXT_BLOCK_LOADER_FIX) ); - super.testIndexing(); + + if (isOldCluster()) { + // given - enable logsdb and create a template + startTrial(); + enableLogsdbByDefault(); + String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); + createTemplate(DATA_STREAM, templateId, prepareTemplate()); + + // when - index some documents + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // then - verify that logsdb and synthetic source are both enabled + String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); + var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); + assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); + assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); + + // then continued - verify that the created data stream uses the created template + LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); + + // when/then - run some queries and verify results + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isMixedCluster()) { + // when + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // when/then + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isUpgradedCluster()) { + // when/then + ensureGreen(DATA_STREAM); + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + + // when/then continued - force merge all shard segments into one + var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); + forceMergeRequest.addParameter("max_num_segments", "1"); + assertOK(client().performRequest(forceMergeRequest)); + + // then continued + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + query(DATA_STREAM); + } + } + + private String prepareTemplate() { + boolean shouldSetIgnoreAbove = randomBoolean(); + if (shouldSetIgnoreAbove) { + return TEMPLATE.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); + } + + // removes the entire line that defines ignore_above + return TEMPLATE.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); } + + static void createTemplate(String dataStreamName, String id, String template) throws IOException { + final String INDEX_TEMPLATE = """ + { + "priority": 500, + "index_patterns": ["$DATASTREAM"], + "template": $TEMPLATE, + "data_stream": { + } + }"""; + var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); + putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); + assertOK(client().performRequest(putIndexTemplateRequest)); + } + + private void bulkIndex(int numRequest, int numDocs) throws Exception { + String firstIndex = null; + Instant startTime = Instant.now().minusSeconds(60 * 60); + + for (int i = 0; i < numRequest; i++) { + var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); + bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); + bulkRequest.addParameter("refresh", "true"); + + var response = client().performRequest(bulkRequest); + var responseBody = entityAsMap(response); + + assertOK(response); + assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); + if (firstIndex == null) { + firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); + } + } + } + + private String bulkIndexRequestBody(int numDocs, Instant startTime) { + StringBuilder requestBody = new StringBuilder(); + + for (int j = 0; j < numDocs; j++) { + String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. + String methodName = "method" + j % 5; + String ip = NetworkAddress.format(randomIp(true)); + String message = randomAlphasDelimitedBySpace(10, 1, 15); + recordSmallestMessage(message); + long length = randomLong(); + double factor = randomDouble(); + + requestBody.append( + BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) + .replace("$host", hostName) + .replace("$method", methodName) + .replace("$ip", ip) + .replace("$message", message) + .replace("$length", Long.toString(length)) + .replace("$factor", Double.toString(factor)) + ); + requestBody.append('\n'); + + startTime = startTime.plusMillis(1); + } + + return requestBody.toString(); + } + + private void recordSmallestMessage(final String message) { + if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { + smallestMessage = message; + } + } + + void search(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "size": 500 + } + """); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); + assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); + } + + private void phraseSearch(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "query": { + "match_phrase": { + "message": "$smallestMessage" + } + } + } + """.replace("$smallestMessage", smallestMessage)); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); + } + + private void query(String dataStreamName) throws Exception { + var queryRequest = new Request("POST", "/_query"); + queryRequest.addParameter("pretty", "true"); + queryRequest.setJsonEntity(""" + { + "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" + } + """.replace("$ds", dataStreamName)); + var response = client().performRequest(queryRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); + assertThat(column1, equalTo("max(length)")); + String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); + assertThat(column2, equalTo("max(factor)")); + String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); + assertThat(column3, equalTo("message")); + + Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); + assertThat(maxRx, notNullValue()); + Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); + assertThat(maxTx, notNullValue()); + String key = ObjectPath.evaluate(responseBody, "values.0.2"); + assertThat(key, equalTo(smallestMessage)); + } + + protected static void startTrial() throws IOException { + Request startTrial = new Request("POST", "/_license/start_trial"); + startTrial.addParameter("acknowledge", "true"); + try { + assertOK(client().performRequest(startTrial)); + } catch (ResponseException e) { + var responseBody = entityAsMap(e.getResponse()); + String error = ObjectPath.evaluate(responseBody, "error_message"); + assertThat(error, containsString("Trial was already activated.")); + } + } + + static Map getIndexSettingsWithDefaults(String index) throws IOException { + Request request = new Request("GET", "/" + index + "/_settings"); + request.addParameter("flat_settings", "true"); + request.addParameter("include_defaults", "true"); + Response response = client().performRequest(request); + try (InputStream is = response.getEntity().getContent()) { + return XContentHelper.convertToMap( + XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), + is, + true + ); + } + } + + static String formatInstant(Instant instant) { + return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); + } + } diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java index c6c35625375da..a2bbd4471bab1 100644 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java +++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/TextRollingUpgradeIT.java @@ -1,25 +1,329 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the "Elastic License - * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. */ package org.elasticsearch.upgrades; import com.carrotsearch.randomizedtesting.annotations.Name; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.time.FormatNames; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.test.rest.ObjectPath; +import org.elasticsearch.xcontent.XContentType; -public class TextRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { +import java.io.IOException; +import java.io.InputStream; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.enableLogsdbByDefault; +import static org.elasticsearch.upgrades.StandardToLogsDbIndexModeRollingUpgradeIT.getWriteBackingIndex; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; + +public class TextRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase { + + private static final String DATA_STREAM = "logs-bwc-test"; + + private static final int IGNORE_ABOVE_MAX = 256; + private static final int NUM_REQUESTS = 4; + private static final int NUM_DOCS_PER_REQUEST = 1024; + + static String BULK_ITEM_TEMPLATE = + """ + { "create": {} } + {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor} + """; + + private static final String TEMPLATE = """ + { + "mappings": { + "properties": { + "@timestamp" : { + "type": "date" + }, + "method": { + "type": "keyword" + }, + "message": { + "type": "text", + "fields": { + "keyword": { + "ignore_above": $IGNORE_ABOVE, + "type": "keyword" + } + } + }, + "ip": { + "type": "ip" + }, + "length": { + "type": "long" + }, + "factor": { + "type": "double" + } + } + } + }"""; + + // when sorted, this message will appear at the top and hence can be used to validate query results + private static String smallestMessage; public TextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { super(upgradedNodes); } - @Override - public String stringType() { - return "text"; + public void testIndexing() throws Exception { + if (isOldCluster()) { + // given - enable logsdb and create a template + startTrial(); + enableLogsdbByDefault(); + String templateId = getClass().getSimpleName().toLowerCase(Locale.ROOT); + createTemplate(DATA_STREAM, templateId, prepareTemplate()); + + // when - index some documents + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // then - verify that logsdb and synthetic source are both enabled + String firstBackingIndex = getWriteBackingIndex(client(), DATA_STREAM, 0); + var settings = (Map) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex); + assertThat(((Map) settings.get("settings")).get("index.mode"), equalTo("logsdb")); + assertThat(((Map) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC")); + + // then continued - verify that the created data stream using the created template + LogsdbIndexingRollingUpgradeIT.assertDataStream(DATA_STREAM, templateId); + + // when/then - run some queries and verify results + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isMixedCluster()) { + // when + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + + // when/then + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + } else if (isUpgradedCluster()) { + // when/then + ensureGreen(DATA_STREAM); + bulkIndex(NUM_REQUESTS, NUM_DOCS_PER_REQUEST); + search(DATA_STREAM); + phraseSearch(DATA_STREAM); + query(DATA_STREAM); + + // when/then continued - force merge all shard segments into one + var forceMergeRequest = new Request("POST", "/" + DATA_STREAM + "/_forcemerge"); + forceMergeRequest.addParameter("max_num_segments", "1"); + assertOK(client().performRequest(forceMergeRequest)); + + // then continued + ensureGreen(DATA_STREAM); + search(DATA_STREAM); + query(DATA_STREAM); + } + } + + private String prepareTemplate() { + boolean shouldSetIgnoreAbove = randomBoolean(); + if (shouldSetIgnoreAbove) { + return TEMPLATE.replace("$IGNORE_ABOVE", String.valueOf(randomInt(IGNORE_ABOVE_MAX))); + } + + // removes the entire line that defines ignore_above + return TEMPLATE.replaceAll("(?m)^\\s*\"ignore_above\":\\s*\\$IGNORE_ABOVE\\s*,?\\s*\\n?", ""); + } + + static void createTemplate(String dataStreamName, String id, String template) throws IOException { + final String INDEX_TEMPLATE = """ + { + "priority": 500, + "index_patterns": ["$DATASTREAM"], + "template": $TEMPLATE, + "data_stream": { + } + }"""; + var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id); + putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName)); + assertOK(client().performRequest(putIndexTemplateRequest)); + } + + private void bulkIndex(int numRequest, int numDocs) throws Exception { + String firstIndex = null; + Instant startTime = Instant.now().minusSeconds(60 * 60); + + for (int i = 0; i < numRequest; i++) { + var bulkRequest = new Request("POST", "/" + DATA_STREAM + "/_bulk"); + bulkRequest.setJsonEntity(bulkIndexRequestBody(numDocs, startTime)); + bulkRequest.addParameter("refresh", "true"); + + var response = client().performRequest(bulkRequest); + var responseBody = entityAsMap(response); + + assertOK(response); + assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false)); + if (firstIndex == null) { + firstIndex = (String) ((Map) ((Map) ((List) responseBody.get("items")).get(0)).get("create")).get("_index"); + } + } + } + + private String bulkIndexRequestBody(int numDocs, Instant startTime) { + StringBuilder requestBody = new StringBuilder(); + + for (int j = 0; j < numDocs; j++) { + String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier. + String methodName = "method" + j % 5; + String ip = NetworkAddress.format(randomIp(true)); + String message = randomAlphasDelimitedBySpace(10, 1, 15); + recordSmallestMessage(message); + long length = randomLong(); + double factor = randomDouble(); + + requestBody.append( + BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime)) + .replace("$host", hostName) + .replace("$method", methodName) + .replace("$ip", ip) + .replace("$message", message) + .replace("$length", Long.toString(length)) + .replace("$factor", Double.toString(factor)) + ); + requestBody.append('\n'); + + startTime = startTime.plusMillis(1); + } + + return requestBody.toString(); + } + + /** + * Generates a string containing a random number of random length alphas, all delimited by space. + */ + public static String randomAlphasDelimitedBySpace(int maxAlphas, int minCodeUnits, int maxCodeUnits) { + int numAlphas = randomIntBetween(1, maxAlphas); + List alphas = new ArrayList<>(numAlphas); + for (int i = 0; i < numAlphas; i++) { + alphas.add(randomAlphaOfLengthBetween(minCodeUnits, maxCodeUnits)); + } + return String.join(" ", alphas); } + + private void recordSmallestMessage(final String message) { + if (smallestMessage == null || message.compareTo(smallestMessage) < 0) { + smallestMessage = message; + } + } + + private void search(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "size": 500 + } + """); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value"); + assertThat(totalCount, greaterThanOrEqualTo(NUM_REQUESTS * NUM_DOCS_PER_REQUEST)); + } + + private void phraseSearch(String dataStreamName) throws Exception { + var searchRequest = new Request("POST", "/" + dataStreamName + "/_search"); + searchRequest.addParameter("pretty", "true"); + searchRequest.setJsonEntity(""" + { + "query": { + "match_phrase": { + "message": "$smallestMessage" + } + } + } + """.replace("$smallestMessage", smallestMessage)); + var response = client().performRequest(searchRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + assertThat(ObjectPath.evaluate(responseBody, "hits.total.value"), greaterThanOrEqualTo(1)); + } + + private void query(String dataStreamName) throws Exception { + var queryRequest = new Request("POST", "/_query"); + queryRequest.addParameter("pretty", "true"); + queryRequest.setJsonEntity(""" + { + "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5" + } + """.replace("$ds", dataStreamName)); + var response = client().performRequest(queryRequest); + assertOK(response); + var responseBody = entityAsMap(response); + logger.info("{}", responseBody); + + String column1 = ObjectPath.evaluate(responseBody, "columns.0.name"); + assertThat(column1, equalTo("max(length)")); + String column2 = ObjectPath.evaluate(responseBody, "columns.1.name"); + assertThat(column2, equalTo("max(factor)")); + String column3 = ObjectPath.evaluate(responseBody, "columns.2.name"); + assertThat(column3, equalTo("message")); + + Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0"); + assertThat(maxRx, notNullValue()); + Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1"); + assertThat(maxTx, notNullValue()); + String key = ObjectPath.evaluate(responseBody, "values.0.2"); + assertThat(key, equalTo(smallestMessage)); + } + + protected static void startTrial() throws IOException { + Request startTrial = new Request("POST", "/_license/start_trial"); + startTrial.addParameter("acknowledge", "true"); + try { + assertOK(client().performRequest(startTrial)); + } catch (ResponseException e) { + var responseBody = entityAsMap(e.getResponse()); + String error = ObjectPath.evaluate(responseBody, "error_message"); + assertThat(error, containsString("Trial was already activated.")); + } + } + + static Map getIndexSettingsWithDefaults(String index) throws IOException { + Request request = new Request("GET", "/" + index + "/_settings"); + request.addParameter("flat_settings", "true"); + request.addParameter("include_defaults", "true"); + Response response = client().performRequest(request); + try (InputStream is = response.getEntity().getContent()) { + return XContentHelper.convertToMap( + XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(), + is, + true + ); + } + } + + static String formatInstant(Instant instant) { + return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant); + } + } diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java deleted file mode 100644 index e2ae2bf23e955..0000000000000 --- a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/WildcardRollingUpgradeIT.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the "Elastic License - * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.upgrades; - -import com.carrotsearch.randomizedtesting.annotations.Name; - - -public class WildcardRollingUpgradeIT extends AbstractStringTypeRollingUpgradeIT { - - public WildcardRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) { - super(upgradedNodes); - } - - @Override - public String stringType() { - return "wildcard"; - } -} From 2c1f1430accea5ad06dd030e7b38e32c36cd6f58 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 29 Oct 2025 19:25:59 +0000 Subject: [PATCH 32/64] [CI] Auto commit changes from spotless --- .../index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 2 +- .../index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 5dec4fe36b1f3..c6d7ebb8ba2ea 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -350,7 +350,7 @@ public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) th assert maxLength >= minLength; if (binaryDVCompressionMode == BinaryDVCompressionMode.NO_COMPRESS) { var offsetsAccumulator = maxLength > minLength ? new OffsetsAccumulator(dir, context, data, numDocsWithField) : null; - binaryWriter = new DirectBinaryWriter(offsetsAccumulator, null); + binaryWriter = new DirectBinaryWriter(offsetsAccumulator, null); } else { binaryWriter = new CompressedBinaryBlockWriter(binaryDVCompressionMode); } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index fd5ebde9f2396..4bd9056e683df 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -48,7 +48,6 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; import org.elasticsearch.index.codec.tsdb.TSDBDocValuesEncoder; -import org.elasticsearch.index.codec.zstd.ZstdDecompressor; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.blockloader.docvalues.BlockDocValuesReader; From 636c150582fd5b63970dbb3dbc933be406d3b996 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 14:40:15 -0500 Subject: [PATCH 33/64] Update code lookup to support other compressors --- .../tsdb/TSDBDocValuesMergeBenchmark.java | 2 +- .../codec/tsdb/BinaryDVCompressionMode.java | 20 ++++++++++++------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 2 +- .../es819/ES819TSDBDocValuesProducer.java | 2 +- 4 files changed, 16 insertions(+), 10 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index 671505d43dbac..9c136a72c8101 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -262,7 +262,7 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE 4096, 512, optimizedMergeEnabled, - BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_LEVEL_1 + BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 ); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 9122340fa0755..a08be3e42e701 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -15,21 +15,27 @@ public enum BinaryDVCompressionMode { NO_COMPRESS((byte) 0, null), - COMPRESSED_WITH_ZSTD_LEVEL_1((byte) 1, new ZstdCompressionMode(1)); + COMPRESSED_ZSTD_LEVEL_1((byte) 1, new ZstdCompressionMode(1)); public final byte code; public final CompressionMode compressionMode; + private static final BinaryDVCompressionMode[] values = new BinaryDVCompressionMode[values().length]; + static { + for (BinaryDVCompressionMode mode : values()) { + values[mode.code] = mode; + } + } + BinaryDVCompressionMode(byte code, CompressionMode compressionMode) { this.code = code; this.compressionMode = compressionMode; } - public static BinaryDVCompressionMode fromMode(byte mode) { - return switch (mode) { - case 0 -> NO_COMPRESS; - case 1 -> COMPRESSED_WITH_ZSTD_LEVEL_1; - default -> throw new IllegalStateException("unknown compression mode [" + mode + "]"); - }; + public static BinaryDVCompressionMode fromMode(byte code) { + if (code < 0 || code >= values.length) { + throw new IllegalStateException("unknown compression mode [" + code + "]"); + } + return values[code]; } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 634af5135423b..7ad9c56fd1b25 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -129,7 +129,7 @@ public ES819TSDBDocValuesFormat() { DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - BinaryDVCompressionMode.COMPRESSED_WITH_ZSTD_LEVEL_1 + BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 ); } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 4bd9056e683df..f90877aa7df56 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -198,7 +198,7 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return switch (entry.compression) { case NO_COMPRESS -> getUncompressedBinary(entry); - case COMPRESSED_WITH_ZSTD_LEVEL_1 -> getCompressedBinary(entry); + default -> getCompressedBinary(entry); }; } From 09898ff42b5fdd6a9c2e9e258230840a2c412ff2 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 15:18:20 -0500 Subject: [PATCH 34/64] feedback --- .../codec/tsdb/BinaryDVCompressionMode.java | 9 +++++++- .../es819/ES819TSDBDocValuesConsumer.java | 21 +++++++++--------- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 4 ++++ .../es819/ES819TSDBDocValuesProducer.java | 22 +++++++++---------- 4 files changed, 34 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index a08be3e42e701..78fd68239e814 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -18,7 +18,7 @@ public enum BinaryDVCompressionMode { COMPRESSED_ZSTD_LEVEL_1((byte) 1, new ZstdCompressionMode(1)); public final byte code; - public final CompressionMode compressionMode; + private final CompressionMode compressionMode; private static final BinaryDVCompressionMode[] values = new BinaryDVCompressionMode[values().length]; static { @@ -38,4 +38,11 @@ public static BinaryDVCompressionMode fromMode(byte code) { } return values[code]; } + + public CompressionMode compressionMode() { + if (compressionMode == null) { + throw new UnsupportedOperationException("BinaryDVCompressionMode [" + code + "] does not support compression"); + } + return compressionMode; + } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index c6d7ebb8ba2ea..351c0785dbd6f 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -517,8 +517,8 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { final Compressor compressor; final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); - final long[] docOffsetsCompressBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; - int[] docOffsets = new int[START_BLOCK_DOCS]; + final long[] docRangesBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; + int[] docRanges = new int[START_BLOCK_DOCS]; int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -533,7 +533,7 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { final DelayedOffsetAccumulator blockDocRangeAcc; CompressedBinaryBlockWriter(BinaryDVCompressionMode compressionMode) throws IOException { - this.compressor = compressionMode.compressionMode.newCompressor(); + this.compressor = compressionMode.compressionMode().newCompressor(); long blockAddressesStart = data.getFilePointer(); blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-addresses", blockAddressesStart); @@ -552,10 +552,11 @@ public void addDoc(BytesRef v) throws IOException { uncompressedBlockLength += v.length; numDocsInCurrentBlock++; - docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block - docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; + docRanges = ArrayUtil.grow(docRanges, numDocsInCurrentBlock + 1); // need one extra since writing start for next block + docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; - if (uncompressedBlockLength > MIN_BLOCK_BYTES) { + int totalUncompressedLength = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; + if (totalUncompressedLength > MIN_BLOCK_BYTES) { flushData(); } } @@ -593,12 +594,12 @@ void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOExce while (batchStart < numOffsets) { int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < batchLength; i++) { - docOffsetsCompressBuffer[i] = docOffsets[batchStart + i]; + docRangesBuffer[i] = docRanges[batchStart + i]; } - if (batchLength < docOffsetsCompressBuffer.length) { - Arrays.fill(docOffsetsCompressBuffer, batchLength, docOffsetsCompressBuffer.length, 0); + if (batchLength < docRangesBuffer.length) { + Arrays.fill(docRangesBuffer, batchLength, docRangesBuffer.length, 0); } - encoder.encode(docOffsetsCompressBuffer, output); + encoder.encode(docRangesBuffer, output); batchStart += batchLength; } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 7ad9c56fd1b25..9e24c6689f2c7 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -28,6 +28,10 @@ * view of all values. If index sorting is active merging a doc value field requires a merge sort which can be very cpu intensive. * The previous format always has to merge sort a doc values field multiple times, so doing the merge sort just once saves on * cpu resources. + *
  • Version 1 adds block-wise compression to binary doc values. Each block contains a variable number of values so that each + * block is approximately the same size. To map a given value's index to the block containing the value, there are two parallel + * arrays. These contain the starting address for each block, and the starting value index for each block. Additional compression + * types may be added by creating a new mode in {@link org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode}.
  • * */ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValuesFormat { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index f90877aa7df56..8af97160b5672 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -287,7 +287,7 @@ private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOExceptio final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new DenseBinaryDocValues(maxDoc) { final BinaryDecoder decoder = new BinaryDecoder( - entry.compression.compressionMode.newDecompressor(), + entry.compression.compressionMode().newDecompressor(), addresses, docRanges, data.clone(), @@ -317,7 +317,7 @@ public BytesRef binaryValue() throws IOException { final DirectMonotonicReader docRanges = DirectMonotonicReader.getInstance(entry.docRangeMeta, docRangeData); return new SparseBinaryDocValues(disi) { final BinaryDecoder decoder = new BinaryDecoder( - entry.compression.compressionMode.newDecompressor(), + entry.compression.compressionMode().newDecompressor(), addresses, docRanges, data.clone(), @@ -343,7 +343,7 @@ static final class BinaryDecoder { private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final long[] docOffsetDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; + private final long[] docRangesDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -380,7 +380,7 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - decompressDocOffsets(numDocsInBlock, compressedData); + decompressDocRanges(numDocsInBlock, compressedData); assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; @@ -388,21 +388,21 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); } - void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { + void decompressDocRanges(int numDocsInBlock, DataInput input) throws IOException { int batchStart = 0; int numOffsets = numDocsInBlock + 1; while (batchStart < numOffsets) { - decoder.decode(input, docOffsetDecompBuffer); + decoder.decode(input, docRangesDecompBuffer); int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < lenToCopy; i++) { - uncompressedDocStarts[batchStart + i] = (int) docOffsetDecompBuffer[i]; + uncompressedDocStarts[batchStart + i] = (int) docRangesDecompBuffer[i]; } batchStart += NUMERIC_BLOCK_SIZE; } } - long findAndUpdateBlock(DirectMonotonicReader docOffsets, long lastBlockId, int docNumber, int numBlocks) { - long index = docOffsets.binarySearch(lastBlockId + 1, numBlocks, docNumber); + long findAndUpdateBlock(DirectMonotonicReader docRanges, long lastBlockId, int docNumber, int numBlocks) { + long index = docRanges.binarySearch(lastBlockId + 1, numBlocks, docNumber); // If index is found, index is inclusive lower bound of docNum range, so docNum is in blockId == index if (index < 0) { // If index was not found, insertion point (-index - 1) will be upper bound of docNum range. @@ -411,8 +411,8 @@ long findAndUpdateBlock(DirectMonotonicReader docOffsets, long lastBlockId, int } assert index < numBlocks : "invalid range " + index + " for doc " + docNumber + " in numBlocks " + numBlocks; - startDocNumForBlock = docOffsets.get(index); - limitDocNumForBlock = docOffsets.get(index + 1); + startDocNumForBlock = docRanges.get(index); + limitDocNumForBlock = docRanges.get(index + 1); return index; } From 8b8b50b470177bf02c5f198bbd87938d3dc21868 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 15:56:52 -0500 Subject: [PATCH 35/64] Update bwc tests --- .../index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java | 9 +++++++++ .../index/codec/tsdb/TsdbDocValueBwcTests.java | 8 +++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 9e24c6689f2c7..e4df44420ccfa 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -137,6 +137,15 @@ public ES819TSDBDocValuesFormat() { ); } + public ES819TSDBDocValuesFormat(BinaryDVCompressionMode binaryDVCompressionMode) { + this( + DEFAULT_SKIP_INDEX_INTERVAL_SIZE, + ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, + OPTIMIZED_MERGE_ENABLE_DEFAULT, + binaryDVCompressionMode + ); + } + /** Doc values fields format with specified skipIndexIntervalSize. */ public ES819TSDBDocValuesFormat( int skipIndexIntervalSize, diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index d07175822be81..6537b7d829b7a 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -59,7 +59,8 @@ public class TsdbDocValueBwcTests extends ESTestCase { public void testMixedIndex() throws Exception { var oldCodec = TestUtil.alwaysDocValuesFormat(new TestES87TSDBDocValuesFormat()); - var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat()); + var compressionMode = ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(); + var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(compressionMode)); testMixedIndex(oldCodec, newCodec); } @@ -74,8 +75,9 @@ public DocValuesFormat getDocValuesFormatForField(String field) { } }; var newCodec = new Elasticsearch92Lucene103Codec() { - - final DocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat(); + final DocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat( + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ); @Override public DocValuesFormat getDocValuesFormatForField(String field) { From 8a82c233a93ef61b044d213cd25b20888333256b Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 16:34:09 -0500 Subject: [PATCH 36/64] cleanup --- .../index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 351c0785dbd6f..b3fb79ad0c301 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -526,7 +526,6 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { byte[] block = BytesRef.EMPTY_BYTES; int totalChunks = 0; - long maxPointer = 0; int maxNumDocsInAnyBlock = 0; final DelayedOffsetAccumulator blockAddressAcc; @@ -583,8 +582,7 @@ public void flushData() throws IOException { numDocsInCurrentBlock = 0; uncompressedBlockLength = 0; - maxPointer = data.getFilePointer(); - long blockLenBytes = maxPointer - thisBlockStartPointer; + long blockLenBytes = data.getFilePointer() - thisBlockStartPointer; blockAddressAcc.addDoc(blockLenBytes); } From 718ffc66888f5902c93c15d064939626914ca00d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Wed, 29 Oct 2025 16:48:08 -0500 Subject: [PATCH 37/64] Fix test broken from merge --- .../index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 9378e69ee48b8..d7c1223e52271 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -1176,7 +1176,8 @@ public void testLoadKeywordFieldWithIndexSorts() throws IOException { final ES819TSDBDocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat( ESTestCase.randomIntBetween(2, 4096), 1, // always enable range-encode - random().nextBoolean() + random().nextBoolean(), + randomBinaryCompressionMode() ); @Override From ebda5b02e033f5d0f91221886ab51bb47c5302fc Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 30 Oct 2025 10:07:59 -0500 Subject: [PATCH 38/64] Update docs/changelog/137139.yaml --- docs/changelog/137139.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/137139.yaml diff --git a/docs/changelog/137139.yaml b/docs/changelog/137139.yaml new file mode 100644 index 0000000000000..ac2bfcba50d86 --- /dev/null +++ b/docs/changelog/137139.yaml @@ -0,0 +1,5 @@ +pr: 137139 +summary: Add binary doc value compression with variable doc count blocks +area: Mapping +type: feature +issues: [] From 9fc23f13a59570faae0aebe6e74c2d84fba19b7a Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 30 Oct 2025 14:39:21 -0500 Subject: [PATCH 39/64] Move block address and doc_range accumulators into BlockMetadataAccumulator class --- .../tsdb/es819/BlockMetadataAccumulator.java | 134 ++++++++++++++++++ .../tsdb/es819/DelayedOffsetAccumulator.java | 100 ------------- .../es819/ES819TSDBDocValuesConsumer.java | 32 +---- 3 files changed, 140 insertions(+), 126 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java delete mode 100644 server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java new file mode 100644 index 0000000000000..698aa8890db20 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java @@ -0,0 +1,134 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.codec.tsdb.es819; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.DirectMonotonicWriter; +import org.elasticsearch.core.IOUtils; + +import java.io.Closeable; +import java.io.IOException; + +public final class BlockMetadataAccumulator implements Closeable { + + private final DelayedOffsetAccumulator blockAddressAcc; + private final DelayedOffsetAccumulator blockDocRangeAcc; + + BlockMetadataAccumulator(Directory dir, IOContext context, IndexOutput data, long addressesStart) throws IOException { + blockDocRangeAcc = new DelayedOffsetAccumulator(dir, context, data, "block-doc-ranges", 0); + blockAddressAcc = new DelayedOffsetAccumulator(dir, context, data, "block-addresses", addressesStart); + } + + public void addDoc(long numDocsInBlock, long blockLenInBytes) throws IOException { + blockDocRangeAcc.addDoc(numDocsInBlock); + blockAddressAcc.addDoc(blockLenInBytes); + } + + public void build(IndexOutput meta, IndexOutput data) throws IOException { + long dataAddressesStart = data.getFilePointer(); + blockAddressAcc.build(meta, data); + long dataDocRangeStart = data.getFilePointer(); + long addressesLength = dataDocRangeStart - dataAddressesStart; + meta.writeLong(addressesLength); + + meta.writeLong(dataDocRangeStart); + blockDocRangeAcc.build(meta, data); + long docRangesLen = data.getFilePointer() - dataDocRangeStart; + meta.writeLong(docRangesLen); + } + + @Override + public void close() throws IOException { + IOUtils.closeWhileHandlingException(blockAddressAcc, blockDocRangeAcc); + } + + /** + * Like OffsetsAccumulator builds offsets and stores in a DirectMonotonicWriter. But write to temp file + * rather than directly to a DirectMonotonicWriter because the number of values is unknown. + */ + static final class DelayedOffsetAccumulator implements Closeable { + + private final Directory dir; + private final long startOffset; + + private int numValues = 0; + private final IndexOutput tempOutput; + private final String suffix; + + DelayedOffsetAccumulator(Directory dir, IOContext context, IndexOutput data, String suffix, long startOffset) throws IOException { + this.dir = dir; + this.startOffset = startOffset; + this.suffix = suffix; + + boolean success = false; + try { + tempOutput = dir.createTempOutput(data.getName(), suffix, context); + CodecUtil.writeHeader(tempOutput, ES819TSDBDocValuesFormat.META_CODEC + suffix, ES819TSDBDocValuesFormat.VERSION_CURRENT); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } + } + + void addDoc(long delta) throws IOException { + tempOutput.writeVLong(delta); + numValues++; + } + + void build(IndexOutput meta, IndexOutput data) throws IOException { + CodecUtil.writeFooter(tempOutput); + IOUtils.close(tempOutput); + + // write the offsets info to the meta file by reading from temp file + try (ChecksumIndexInput tempInput = dir.openChecksumInput(tempOutput.getName());) { + CodecUtil.checkHeader( + tempInput, + ES819TSDBDocValuesFormat.META_CODEC + suffix, + ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_CURRENT + ); + Throwable priorE = null; + try { + final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( + meta, + data, + numValues + 1, + ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT + ); + + long offset = startOffset; + writer.add(offset); + for (int i = 0; i < numValues; ++i) { + offset += tempInput.readVLong(); + writer.add(offset); + } + writer.finish(); + } catch (Throwable e) { + priorE = e; + } finally { + CodecUtil.checkFooter(tempInput, priorE); + } + } + } + + @Override + public void close() throws IOException { + if (tempOutput != null) { + IOUtils.close(tempOutput, () -> dir.deleteFile(tempOutput.getName())); + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java deleted file mode 100644 index 1e9d4e46d2b70..0000000000000 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/DelayedOffsetAccumulator.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the "Elastic License - * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.index.codec.tsdb.es819; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.packed.DirectMonotonicWriter; -import org.elasticsearch.core.IOUtils; - -import java.io.Closeable; -import java.io.IOException; - -/** - * Like OffsetsAccumulator builds offsets and stores in a DirectMonotonicWriter. But write to temp file - * rather than directly to a DirectMonotonicWriter because the number of values is unknown. If number of - * values is known prefer OffsetsWriter. - */ -final class DelayedOffsetAccumulator implements Closeable { - private final Directory dir; - private final long startOffset; - - private int numValues = 0; - private final IndexOutput tempOutput; - private final String suffix; - - DelayedOffsetAccumulator(Directory dir, IOContext context, IndexOutput data, String suffix, long startOffset) throws IOException { - this.dir = dir; - this.startOffset = startOffset; - this.suffix = suffix; - - boolean success = false; - try { - tempOutput = dir.createTempOutput(data.getName(), suffix, context); - CodecUtil.writeHeader(tempOutput, ES819TSDBDocValuesFormat.META_CODEC + suffix, ES819TSDBDocValuesFormat.VERSION_CURRENT); - success = true; - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't - } - } - } - - public void addDoc(long delta) throws IOException { - tempOutput.writeVLong(delta); - numValues++; - } - - public void build(IndexOutput meta, IndexOutput data) throws IOException { - CodecUtil.writeFooter(tempOutput); - IOUtils.close(tempOutput); - - // write the offsets info to the meta file by reading from temp file - try (ChecksumIndexInput tempInput = dir.openChecksumInput(tempOutput.getName());) { - CodecUtil.checkHeader( - tempInput, - ES819TSDBDocValuesFormat.META_CODEC + suffix, - ES819TSDBDocValuesFormat.VERSION_CURRENT, - ES819TSDBDocValuesFormat.VERSION_CURRENT - ); - Throwable priorE = null; - try { - final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance( - meta, - data, - numValues + 1, - ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT - ); - - long offset = startOffset; - writer.add(offset); - for (int i = 0; i < numValues; ++i) { - offset += tempInput.readVLong(); - writer.add(offset); - } - writer.finish(); - } catch (Throwable e) { - priorE = e; - } finally { - CodecUtil.checkFooter(tempInput, priorE); - } - } - } - - @Override - public void close() throws IOException { - if (tempOutput != null) { - IOUtils.close(tempOutput, () -> dir.deleteFile(tempOutput.getName())); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index b3fb79ad0c301..ed22bf934afe4 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -528,20 +528,12 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { int totalChunks = 0; int maxNumDocsInAnyBlock = 0; - final DelayedOffsetAccumulator blockAddressAcc; - final DelayedOffsetAccumulator blockDocRangeAcc; + final BlockMetadataAccumulator blockMetaAcc; CompressedBinaryBlockWriter(BinaryDVCompressionMode compressionMode) throws IOException { this.compressor = compressionMode.compressionMode().newCompressor(); long blockAddressesStart = data.getFilePointer(); - blockAddressAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-addresses", blockAddressesStart); - - try { - blockDocRangeAcc = new DelayedOffsetAccumulator(state.directory, state.context, data, "block-doc-ranges", 0); - } catch (IOException e) { - blockAddressAcc.close(); - throw e; - } + this.blockMetaAcc = new BlockMetadataAccumulator(state.directory, state.context, data, blockAddressesStart); } @Override @@ -578,12 +570,9 @@ public void flushData() throws IOException { compressOffsets(data, numDocsInCurrentBlock); compress(block, uncompressedBlockLength, data); - blockDocRangeAcc.addDoc(numDocsInCurrentBlock); - numDocsInCurrentBlock = 0; - - uncompressedBlockLength = 0; long blockLenBytes = data.getFilePointer() - thisBlockStartPointer; - blockAddressAcc.addDoc(blockLenBytes); + blockMetaAcc.addDoc(numDocsInCurrentBlock, blockLenBytes); + numDocsInCurrentBlock = uncompressedBlockLength = 0; } void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { @@ -621,21 +610,12 @@ public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithFi meta.writeVInt(maxNumDocsInAnyBlock); meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT); - blockAddressAcc.build(meta, data); - long dataDocRangeStart = data.getFilePointer(); - long addressesLength = dataDocRangeStart - dataAddressesStart; - meta.writeLong(addressesLength); - - meta.writeLong(dataDocRangeStart); - blockDocRangeAcc.build(meta, data); - long docRangesLen = data.getFilePointer() - dataDocRangeStart; - meta.writeLong(docRangesLen); + blockMetaAcc.build(meta, data); } @Override public void close() throws IOException { - blockDocRangeAcc.close(); - blockAddressAcc.close(); + blockMetaAcc.close(); } } From 80525bfa77590a64377960c42ef73dce0d6003a6 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 31 Oct 2025 12:15:41 -0500 Subject: [PATCH 40/64] Unit tests that require multiple doc value blocks --- .../es819/ES819TSDBDocValuesConsumer.java | 11 +- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 2 + .../es819/ES819TSDBDocValuesFormatTests.java | 119 ++++++++++++++++++ 3 files changed, 129 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index ed22bf934afe4..2481a063bdf73 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -56,6 +56,7 @@ import static org.elasticsearch.index.codec.tsdb.es819.DocValuesConsumerUtil.compatibleWithOptimizedMerge; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; @@ -508,10 +509,14 @@ public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithFi } } } + + @Override + public void close() throws IOException { + IOUtils.close(offsetsAccumulator); + } } private final class CompressedBinaryBlockWriter implements BinaryWriter { - static final int MIN_BLOCK_BYTES = 256 * 1024; static final int START_BLOCK_DOCS = 1024; final Compressor compressor; @@ -547,7 +552,7 @@ public void addDoc(BytesRef v) throws IOException { docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; int totalUncompressedLength = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; - if (totalUncompressedLength > MIN_BLOCK_BYTES) { + if (totalUncompressedLength > MIN_BLOCK_SIZE_BYTES) { flushData(); } } @@ -615,7 +620,7 @@ public void writeAddressMetadata(int minLength, int maxLength, int numDocsWithFi @Override public void close() throws IOException { - blockMetaAcc.close(); + IOUtils.close(blockMetaAcc); } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index e4df44420ccfa..4acb7f47e2da9 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -63,6 +63,8 @@ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValues 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; + public static final int MIN_BLOCK_SIZE_BYTES = 256 * 1024; + // number of documents in an interval private static final int DEFAULT_SKIP_INDEX_INTERVAL_SIZE = 4096; // bytes on an interval: diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index d7c1223e52271..0d8c07041e880 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -63,7 +63,10 @@ import java.util.function.Supplier; import java.util.stream.IntStream; +import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; +import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomFrom; +import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -89,6 +92,122 @@ protected Codec getCodec() { return codec; } + // Test with data large enough to require multiple binary doc value blocks + public void testBlockWiseBinarySparse() throws Exception { + String timestampField = "@timestamp"; + String hostnameField = "host.name"; + long baseTimestamp = 1704067200000L; + String binaryField = "binary_field"; + + var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); + try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { + + int maxBlocks = 4; + int binaryDataSize = randomIntBetween(0, ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES * maxBlocks); + + List binaryValues = new ArrayList<>(); + int totalSize = 0; + while (totalSize < binaryDataSize) { + if (randomBoolean()) { + String value = randomAlphaOfLengthBetween(0, 100); + binaryValues.add(value); + totalSize += value.length(); + } else { + binaryValues.add(null); + } + } + int numDocs = binaryValues.size(); + + for (int i = 0; i < numDocs; i++) { + var d = new Document(); + long timestamp = baseTimestamp + (1000L * i); + d.add(new SortedDocValuesField(hostnameField, new BytesRef("host-1"))); + d.add(new SortedNumericDocValuesField(timestampField, timestamp)); + + String binaryValue = binaryValues.get(i); + if (binaryValue != null) { + d.add(new BinaryDocValuesField(binaryField, new BytesRef(binaryValue))); + } + + iw.addDocument(d); + if (i % 100 == 0) { + iw.commit(); + } + } + iw.commit(); + iw.forceMerge(1); + + try (var reader = DirectoryReader.open(iw)) { + assertEquals(1, reader.leaves().size()); + assertEquals(numDocs, reader.maxDoc()); + var leaf = reader.leaves().get(0).reader(); + var binaryDV = leaf.getBinaryDocValues(binaryField); + assertNotNull(binaryDV); + for (int i = 0; i < numDocs; i++) { + String expected = binaryValues.removeLast(); + if (expected == null) { + assertFalse(binaryDV.advanceExact(i)); + } else { + assertTrue(binaryDV.advanceExact(i)); + assertEquals(expected, binaryDV.binaryValue().utf8ToString()); + } + } + } + } + } + + // Test with data large enough to require multiple binary doc value blocks + public void testBlockWiseBinaryDense() throws Exception { + String timestampField = "@timestamp"; + String hostnameField = "host.name"; + long baseTimestamp = 1704067200000L; + String binaryField = "binary_field"; + + var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); + try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { + + int maxBlocks = 4; + int binaryDataSize = randomIntBetween(0, ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES * maxBlocks); + + List binaryValues = new ArrayList<>(); + int totalSize = 0; + while (totalSize < binaryDataSize) { + String value = randomAlphaOfLengthBetween(0, 100); + binaryValues.add(value); + totalSize += value.length(); + } + int numDocs = binaryValues.size(); + + for (int i = 0; i < numDocs; i++) { + var d = new Document(); + long timestamp = baseTimestamp + (1000L * i); + d.add(new SortedDocValuesField(hostnameField, new BytesRef("host-1"))); + d.add(new SortedNumericDocValuesField(timestampField, timestamp)); + + d.add(new BinaryDocValuesField(binaryField, new BytesRef(binaryValues.get(i)))); + + iw.addDocument(d); + if (i % 100 == 0) { + iw.commit(); + } + } + iw.commit(); + iw.forceMerge(1); + + try (var reader = DirectoryReader.open(iw)) { + assertEquals(1, reader.leaves().size()); + assertEquals(numDocs, reader.maxDoc()); + var leaf = reader.leaves().get(0).reader(); + var binaryDV = leaf.getBinaryDocValues(binaryField); + assertNotNull(binaryDV); + for (int i = 0; i < numDocs; i++) { + assertEquals(i, binaryDV.nextDoc()); + assertEquals(binaryValues.removeLast(), binaryDV.binaryValue().utf8ToString()); + } + } + } + } + public void testForceMergeDenseCase() throws Exception { String timestampField = "@timestamp"; String hostnameField = "host.name"; From b1d4b17d74446e4ce9926c8a69689bf97bfc4232 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 31 Oct 2025 12:32:07 -0500 Subject: [PATCH 41/64] Test values near the size of a block --- .../es819/ES819TSDBDocValuesFormatTests.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 0d8c07041e880..bf7ffcba92782 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -63,8 +63,10 @@ import java.util.function.Supplier; import java.util.stream.IntStream; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; import static org.elasticsearch.test.ESTestCase.randomBoolean; +import static org.elasticsearch.test.ESTestCase.randomFloat; import static org.elasticsearch.test.ESTestCase.randomFrom; import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.hamcrest.Matchers.equalTo; @@ -98,18 +100,21 @@ public void testBlockWiseBinarySparse() throws Exception { String hostnameField = "host.name"; long baseTimestamp = 1704067200000L; String binaryField = "binary_field"; + boolean testVeryLargeValues = randomFloat() < 0.1; var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { int maxBlocks = 4; - int binaryDataSize = randomIntBetween(0, ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES * maxBlocks); + int binaryDataSize = randomIntBetween(0, MIN_BLOCK_SIZE_BYTES * maxBlocks); List binaryValues = new ArrayList<>(); int totalSize = 0; while (totalSize < binaryDataSize) { if (randomBoolean()) { - String value = randomAlphaOfLengthBetween(0, 100); + final String value = testVeryLargeValues + ? randomAlphaOfLengthBetween(MIN_BLOCK_SIZE_BYTES / 2, 2 * MIN_BLOCK_SIZE_BYTES) + : randomAlphaOfLengthBetween(0, 50); binaryValues.add(value); totalSize += value.length(); } else { @@ -162,17 +167,21 @@ public void testBlockWiseBinaryDense() throws Exception { String hostnameField = "host.name"; long baseTimestamp = 1704067200000L; String binaryField = "binary_field"; + boolean testVeryLargeValues = randomFloat() < 0.1; var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { int maxBlocks = 4; - int binaryDataSize = randomIntBetween(0, ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES * maxBlocks); + int binaryDataSize = randomIntBetween(0, MIN_BLOCK_SIZE_BYTES * maxBlocks); List binaryValues = new ArrayList<>(); int totalSize = 0; while (totalSize < binaryDataSize) { - String value = randomAlphaOfLengthBetween(0, 100); + final String value = testVeryLargeValues + ? randomAlphaOfLengthBetween(MIN_BLOCK_SIZE_BYTES / 2, 2 * MIN_BLOCK_SIZE_BYTES) + : randomAlphaOfLengthBetween(0, 50); + binaryValues.add(value); totalSize += value.length(); } From e33261956be5a4a3db818e6164f4304f85ee35ed Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 31 Oct 2025 13:43:20 -0500 Subject: [PATCH 42/64] Self close BlockMetadataAcc if throw during construction --- .../codec/tsdb/es819/BlockMetadataAccumulator.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java index 698aa8890db20..54cc127c8b66e 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java @@ -26,8 +26,16 @@ public final class BlockMetadataAccumulator implements Closeable { private final DelayedOffsetAccumulator blockDocRangeAcc; BlockMetadataAccumulator(Directory dir, IOContext context, IndexOutput data, long addressesStart) throws IOException { - blockDocRangeAcc = new DelayedOffsetAccumulator(dir, context, data, "block-doc-ranges", 0); - blockAddressAcc = new DelayedOffsetAccumulator(dir, context, data, "block-addresses", addressesStart); + boolean success = false; + try { + blockDocRangeAcc = new DelayedOffsetAccumulator(dir, context, data, "block-doc-ranges", 0); + blockAddressAcc = new DelayedOffsetAccumulator(dir, context, data, "block-addresses", addressesStart); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't + } + } } public void addDoc(long numDocsInBlock, long blockLenInBytes) throws IOException { From 1209e78d53103e3ebe5edd054f327739ec840846 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 3 Nov 2025 10:47:26 -0600 Subject: [PATCH 43/64] Update tsdb doc_values bwc test to mention version 1 --- .../index/codec/tsdb/TsdbDocValueBwcTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index f311dfa8d6482..c58f83732e391 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -65,10 +65,10 @@ public void testMixedIndex() throws Exception { testMixedIndex(oldCodec, newCodec); } - // TODO update Current to Version1 once version is incremented - public void testMixedIndexDocValueVersion0ToCurrent() throws Exception { + public void testMixedIndexDocValueVersion0ToVersion1() throws Exception { var oldCodec = TestUtil.alwaysDocValuesFormat(new TestES819TSDBDocValuesFormatVersion0()); - var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat()); + var compressionMode = ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(); + var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(compressionMode)); testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); } From 80c14a3e533eca78d8c937d9279531777b2a213a Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 3 Nov 2025 13:30:30 -0600 Subject: [PATCH 44/64] Update docs/changelog/137139.yaml --- docs/changelog/137139.yaml | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/changelog/137139.yaml b/docs/changelog/137139.yaml index ac2bfcba50d86..e15e9c02f8ce7 100644 --- a/docs/changelog/137139.yaml +++ b/docs/changelog/137139.yaml @@ -3,3 +3,25 @@ summary: Add binary doc value compression with variable doc count blocks area: Mapping type: feature issues: [] +highlight: + title: Add binary doc value compression with variable doc count blocks + body: "Add compression for binary doc values using Zstd and blocks with a\nvariable\ + \ number of values.\n\nBlock-wise LZ4 was previously added to Lucene in\n[LUCENE-9211](https://issues.apache.org/jira/browse/LUCENE-9211).\ + \ This\nwas subsequently removed in\n[LUCENE-9378](https://issues.apache.org/jira/browse/LUCENE-9378)\ + \ due to\nquery performance issues. \n\nWe investigated adding to adding the original\ + \ Lucene implementation to\nES in https://github.com/elastic/elasticsearch/pull/112416\ + \ and\nhttps://github.com/elastic/elasticsearch/pull/105301. This approach\nstores\ + \ a constant number of values per block (specifically 32 values).\nThis is nice\ + \ because it makes it very easy to map a given value index\n(eg docId for dense\ + \ values) to the block containing it with `blockId =\ndocId / 32`. Unfortunately,\ + \ if values are very large we cannot reduce\nthe number of values per block and\ + \ (de)compressing a block could cause\nan OOM. Also, since this is a concern,\ + \ we have to keep the number of\nvalues lower than ideal.\n\nThis PR instead stores\ + \ a variable number of documents per block. It\nstores a minimum of 1 document\ + \ per block and stops adding values when\nthe size of a block exceeds a threshold.\ + \ Like the previous version is\nstores an array of address for the start of each\ + \ block. Additionally, it\nstores are parallel array with the value index at the\ + \ start of each\nblock. When looking up a given value index, if it is not in the\ + \ current\nblock, we binary search the array of value index starts to find the\n\ + blockId containing the value. Then look up the address of the block." + notable: true From 602c203fe39b0bcfd6e29d865eb4c433d56c754d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 3 Nov 2025 19:39:34 -0600 Subject: [PATCH 45/64] Disable compression for geo_shape type --- .../index/codec/PerFieldFormatSupplier.java | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java index 2ed1aa6c9f17f..47358772a076b 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java +++ b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java @@ -41,6 +41,7 @@ public class PerFieldFormatSupplier { private static final Set INCLUDE_META_FIELDS; + private static final Set EXCLUDE_MAPPER_TYPES; static { // TODO: should we just allow all fields to use tsdb doc values codec? @@ -53,6 +54,7 @@ public class PerFieldFormatSupplier { // Don't the include _recovery_source_size and _recovery_source fields, since their values can be trimmed away in // RecoverySourcePruneMergePolicy, which leads to inconsistencies between merge stats and actual values. INCLUDE_META_FIELDS = Collections.unmodifiableSet(includeMetaField); + EXCLUDE_MAPPER_TYPES = Set.of("geo_shape"); } private static final DocValuesFormat docValuesFormat = new Lucene90DocValuesFormat(); @@ -138,6 +140,10 @@ boolean useTSDBDocValuesFormat(final String field) { return false; } + if (excludeMapperTypes(field)) { + return false; + } + return mapperService != null && (isTimeSeriesModeIndex() || isLogsModeIndex()) && mapperService.getIndexSettings().isES87TSDBCodecEnabled(); @@ -147,6 +153,14 @@ private boolean excludeFields(String fieldName) { return fieldName.startsWith("_") && INCLUDE_META_FIELDS.contains(fieldName) == false; } + private boolean excludeMapperTypes(String fieldName) { + var typeName = getMapperType(fieldName); + if (typeName == null) { + return false; + } + return EXCLUDE_MAPPER_TYPES.contains(getMapperType(fieldName)); + } + private boolean isTimeSeriesModeIndex() { return mapperService != null && IndexMode.TIME_SERIES == mapperService.getIndexSettings().getMode(); } @@ -155,4 +169,13 @@ private boolean isLogsModeIndex() { return mapperService != null && IndexMode.LOGSDB == mapperService.getIndexSettings().getMode(); } + String getMapperType(final String field) { + if (mapperService != null) { + Mapper mapper = mapperService.mappingLookup().getMapper(field); + if (mapper != null) { + return mapper.typeName(); + } + } + return null; + } } From d6293d9fffb455d6b89d8e4d57acd4446298c11d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Tue, 4 Nov 2025 16:00:54 -0600 Subject: [PATCH 46/64] Test that wildcard uses ES819 docs encoding and geo_shape does not --- .../mapper/GeoShapeDocValueFormatTests.java | 80 +++++++++++++++++++ .../mapper/WildcardDocValueFormatTests.java | 70 ++++++++++++++++ 2 files changed, 150 insertions(+) create mode 100644 x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java create mode 100644 x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java new file mode 100644 index 0000000000000..89227de152d08 --- /dev/null +++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java @@ -0,0 +1,80 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.spatial.index.mapper; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.spatial.LocalStateSpatialPlugin; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.startsWith; + +public class GeoShapeDocValueFormatTests extends ESSingleNodeTestCase { + + @Override + protected Collection> getPlugins() { + return List.of(LocalStateSpatialPlugin.class); + } + + /** + * geo_shape uses binary doc values internally. As geo_shape does not work well with binary doc_value compression, if logsdb + * is used, geo_shape should use the Lucene doc value format rather than ES819TSDBDocValuesFormat. + */ + public void testGeoShapeDocValueUseLuceneFormat() throws IOException { + String indexName = "test"; + String fieldName = "field_name"; + Settings settings = Settings.builder().put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()).build(); + IndexService indexService = createIndex(indexName, settings, "doc", "@timestamp", "type=date", fieldName, "type=geo_shape"); + + var indexRequest = new IndexRequest(indexName) + .opType(DocWriteRequest.OpType.CREATE) + .source(""" + { + "@timestamp": "2025-10-01T12:34:56.789", + "%field": { + "type" : "Point", + "coordinates" : [-77.03653, 38.897676] + } + } + """.replace("%field", fieldName), XContentType.JSON); + var response = client().bulk(new BulkRequest().add(indexRequest)).actionGet(); + assertFalse(response.hasFailures()); + safeGet(indicesAdmin().refresh(new RefreshRequest(indexName).indicesOptions(IndicesOptions.lenientExpandOpenHidden()))); + + try (var searcher = indexService.getShard(0).acquireSearcher(indexName)) { + try (var indexReader = searcher.getIndexReader()) { + var leaves = indexReader.leaves(); + assertThat(leaves.size(), equalTo(1)); + FieldInfos fieldInfos = leaves.getFirst().reader().getFieldInfos(); + FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldName); + assertNotNull(fieldInfo); + Map attributes = fieldInfo.attributes(); + assertThat(attributes, hasKey("PerFieldDocValuesFormat.format")); + assertThat(attributes.get("PerFieldDocValuesFormat.format"), startsWith("Lucene")); + } + } + } +} diff --git a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java new file mode 100644 index 0000000000000..34cea70192094 --- /dev/null +++ b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java @@ -0,0 +1,70 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.wildcard.mapper; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.wildcard.Wildcard; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; + +public class WildcardDocValueFormatTests extends ESSingleNodeTestCase { + + protected Collection> getPlugins() { + return List.of(Wildcard.class); + } + + public void testWildcardDocValueUseES819Format() throws IOException { + String indexName = "test"; + String fieldName = "field_name"; + Settings settings = Settings.builder().put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()).build(); + IndexService indexService = createIndex(indexName, settings, "doc", "@timestamp", "type=date", fieldName, "type=wildcard"); + + var indexRequest = new IndexRequest(indexName) + .opType(DocWriteRequest.OpType.CREATE) + .source(""" + { + "@timestamp": "2025-10-01T12:34:56.789", + "%field": "baz" + } + """.replace("%field", fieldName), XContentType.JSON); + var response = client().bulk(new BulkRequest().add(indexRequest)).actionGet(); + assertFalse(response.hasFailures()); + safeGet(indicesAdmin().refresh(new RefreshRequest(indexName).indicesOptions(IndicesOptions.lenientExpandOpenHidden()))); + + try (var searcher = indexService.getShard(0).acquireSearcher(indexName)) { + try (var indexReader = searcher.getIndexReader()) { + var leaves = indexReader.leaves(); + assertThat(leaves.size(), equalTo(1)); + FieldInfos fieldInfos = leaves.getFirst().reader().getFieldInfos(); + FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldName); + assertNotNull(fieldInfo); + Map attributes = fieldInfo.attributes(); + assertThat(attributes, hasEntry("PerFieldDocValuesFormat.format", "ES819TSDB")); + } + } + } +} From 982386ef12b9f4866a8528e3148bc0a0a4defb1d Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 4 Nov 2025 22:07:23 +0000 Subject: [PATCH 47/64] [CI] Auto commit changes from spotless --- .../spatial/index/mapper/GeoShapeDocValueFormatTests.java | 4 +--- .../xpack/wildcard/mapper/WildcardDocValueFormatTests.java | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java index 89227de152d08..5498105425c4a 100644 --- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java +++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeDocValueFormatTests.java @@ -49,9 +49,7 @@ public void testGeoShapeDocValueUseLuceneFormat() throws IOException { Settings settings = Settings.builder().put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()).build(); IndexService indexService = createIndex(indexName, settings, "doc", "@timestamp", "type=date", fieldName, "type=geo_shape"); - var indexRequest = new IndexRequest(indexName) - .opType(DocWriteRequest.OpType.CREATE) - .source(""" + var indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE).source(""" { "@timestamp": "2025-10-01T12:34:56.789", "%field": { diff --git a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java index 34cea70192094..01ea60dcf2abf 100644 --- a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java +++ b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardDocValueFormatTests.java @@ -43,9 +43,7 @@ public void testWildcardDocValueUseES819Format() throws IOException { Settings settings = Settings.builder().put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()).build(); IndexService indexService = createIndex(indexName, settings, "doc", "@timestamp", "type=date", fieldName, "type=wildcard"); - var indexRequest = new IndexRequest(indexName) - .opType(DocWriteRequest.OpType.CREATE) - .source(""" + var indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE).source(""" { "@timestamp": "2025-10-01T12:34:56.789", "%field": "baz" From e61b8c2125e486d3da819fd579a0e47410afa5c9 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 6 Nov 2025 16:28:16 -0600 Subject: [PATCH 48/64] Add feature flag for binary dv compression --- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 5 +++- .../codec/tsdb/TsdbDocValueBwcTests.java | 27 ++++++++++++++++++- .../ES819TSDBDocValuesConsumerVersion0.java | 4 +-- .../es819/ES819TSDBDocValuesFormatTests.java | 9 +++++++ .../test/cluster/FeatureFlag.java | 3 ++- .../xpack/logsdb/LogsdbTestSuiteIT.java | 1 + 6 files changed, 44 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index c9a1acb30e913..643e76cf0ba70 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -13,6 +13,7 @@ import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; +import org.elasticsearch.common.util.FeatureFlag; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.index.codec.tsdb.BinaryDVCompressionMode; @@ -36,6 +37,8 @@ */ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValuesFormat { + public static final boolean BINARY_DV_COMPRESSION_FEATURE_FLAG = new FeatureFlag("binary_dv_compression").isEnabled(); + static final int NUMERIC_BLOCK_SHIFT = 7; public static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT; static final int NUMERIC_BLOCK_MASK = NUMERIC_BLOCK_SIZE - 1; @@ -135,7 +138,7 @@ public ES819TSDBDocValuesFormat() { DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 + BINARY_DV_COMPRESSION_FEATURE_FLAG ? BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 : BinaryDVCompressionMode.NO_COMPRESS ); } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index c58f83732e391..21a12c414a290 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -12,6 +12,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.Document; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; @@ -72,6 +73,13 @@ public void testMixedIndexDocValueVersion0ToVersion1() throws Exception { testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); } + public void testMixedIndexDocValueBinaryCompressionFeatureDisabledOldCodec() throws Exception { + // Mimic the behavior of BINARY_DV_COMPRESSION_FEATURE_FLAG being disabled in the oldCodec, but enabled in the newCodec. + var oldCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.NO_COMPRESS)); + var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1)); + testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); + } + public void testMixedIndex816To900Lucene101() throws Exception { var oldCodec = new Elasticsearch816Codec() { @@ -151,8 +159,9 @@ void testMixedIndex(Codec oldCodec, Codec newCodec, VersionAssert assertOldVersi d.add(new SortedNumericDocValuesField(timestampField, timestamp++)); if (r % 10 < 8) { - // Most of the time store counter: + // Most of the time store counter and binary value: d.add(new NumericDocValuesField("counter_1", counter1++)); + d.add(new BinaryDocValuesField("binary_tag", new BytesRef(tags[j % tags.length]))); } if (r % 10 == 5) { @@ -194,6 +203,10 @@ void testMixedIndex(Codec oldCodec, Codec newCodec, VersionAssert assertOldVersi if (tagsDV == null) { tagsDV = DocValues.emptySortedSet(); } + var binaryDV = MultiDocValues.getBinaryValues(reader, "binary_tag"); + if (binaryDV == null) { + binaryDV = DocValues.emptyBinary(); + } for (int i = 0; i < numDocs; i++) { assertEquals(i, hostNameDV.nextDoc()); String actualHostName = hostNameDV.lookupOrd(hostNameDV.ordValue()).utf8ToString(); @@ -224,6 +237,10 @@ void testMixedIndex(Codec oldCodec, Codec newCodec, VersionAssert assertOldVersi assertTrue("unexpected tag [" + actualTag + "]", Arrays.binarySearch(tags, actualTag) >= 0); } } + if (binaryDV.advanceExact(i)) { + String actualBinary = binaryDV.binaryValue().utf8ToString(); + assertTrue("unexpected binary [" + actualBinary + "]", Arrays.binarySearch(tags, actualBinary) >= 0); + } } } @@ -254,6 +271,10 @@ void testMixedIndex(Codec oldCodec, Codec newCodec, VersionAssert assertOldVersi if (tagsDV == null) { tagsDV = DocValues.emptySortedSet(); } + var binaryDV = MultiDocValues.getBinaryValues(reader, "binary_tag"); + if (binaryDV == null) { + binaryDV = DocValues.emptyBinary(); + } for (int i = 0; i < numDocs; i++) { assertEquals(i, hostNameDV.nextDoc()); String actualHostName = hostNameDV.lookupOrd(hostNameDV.ordValue()).utf8ToString(); @@ -284,6 +305,10 @@ void testMixedIndex(Codec oldCodec, Codec newCodec, VersionAssert assertOldVersi assertTrue("unexpected tag [" + actualTag + "]", Arrays.binarySearch(tags, actualTag) >= 0); } } + if (binaryDV.advanceExact(i)) { + String actualBinary = binaryDV.binaryValue().utf8ToString(); + assertTrue("unexpected binary [" + actualBinary + "]", Arrays.binarySearch(tags, actualBinary) >= 0); + } } } } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumerVersion0.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumerVersion0.java index df669f007db2b..9115842533453 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumerVersion0.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumerVersion0.java @@ -88,7 +88,7 @@ final class ES819TSDBDocValuesConsumerVersion0 extends XDocValuesConsumer { CodecUtil.writeIndexHeader( data, dataCodec, - ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_START, // Test with version 0 rather than current state.segmentInfo.getId(), state.segmentSuffix ); @@ -97,7 +97,7 @@ final class ES819TSDBDocValuesConsumerVersion0 extends XDocValuesConsumer { CodecUtil.writeIndexHeader( meta, metaCodec, - ES819TSDBDocValuesFormat.VERSION_CURRENT, + ES819TSDBDocValuesFormat.VERSION_START, // Test with version 0 rather than current state.segmentInfo.getId(), state.segmentSuffix ); diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 2dd762869da34..36ae6facbc8af 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -117,6 +117,15 @@ protected Codec getCodec() { return codec; } + public void testBinaryCompressionFeatureFlag() { + ES819TSDBDocValuesFormat docValueFormat = new ES819TSDBDocValuesFormat(); + if (ES819TSDBDocValuesFormat.BINARY_DV_COMPRESSION_FEATURE_FLAG) { + assertThat(docValueFormat.binaryDVCompressionMode, equalTo(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1)); + } else { + assertThat(docValueFormat.binaryDVCompressionMode, equalTo(BinaryDVCompressionMode.NO_COMPRESS)); + } + } + // Test with data large enough to require multiple binary doc value blocks public void testBlockWiseBinarySparse() throws Exception { String timestampField = "@timestamp"; diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java index 168b5a15743c1..429a1eaec0837 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java @@ -26,7 +26,8 @@ public enum FeatureFlag { Version.fromString("9.2.0"), null ), - RANDOM_SAMPLING("es.random_sampling_feature_flag_enabled=true", Version.fromString("9.2.0"), null); + RANDOM_SAMPLING("es.random_sampling_feature_flag_enabled=true", Version.fromString("9.2.0"), null), + BINARY_DOC_VALUE_COMPRESSION("es.binary_dv_compression_feature_flag_enabled=true", Version.fromString("9.3.0"), null); public final String systemProperty; public final Version from; diff --git a/x-pack/plugin/logsdb/src/yamlRestTest/java/org/elasticsearch/xpack/logsdb/LogsdbTestSuiteIT.java b/x-pack/plugin/logsdb/src/yamlRestTest/java/org/elasticsearch/xpack/logsdb/LogsdbTestSuiteIT.java index 5a1ccb4e08140..bfa059d428bc0 100644 --- a/x-pack/plugin/logsdb/src/yamlRestTest/java/org/elasticsearch/xpack/logsdb/LogsdbTestSuiteIT.java +++ b/x-pack/plugin/logsdb/src/yamlRestTest/java/org/elasticsearch/xpack/logsdb/LogsdbTestSuiteIT.java @@ -37,6 +37,7 @@ public class LogsdbTestSuiteIT extends ESClientYamlSuiteTestCase { .setting("xpack.security.autoconfiguration.enabled", "false") .setting("xpack.license.self_generated.type", "trial") .feature(FeatureFlag.DOC_VALUES_SKIPPER) + .feature(FeatureFlag.BINARY_DOC_VALUE_COMPRESSION) .build(); public LogsdbTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { From 5fe2c809d77b7f008eccb88a05870426cce0b4d7 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 7 Nov 2025 11:46:53 -0600 Subject: [PATCH 49/64] Add block count threshold in addition to size threshold --- .../tsdb/es819/ES819TSDBDocValuesConsumer.java | 6 +++--- .../codec/tsdb/es819/ES819TSDBDocValuesFormat.java | 8 +++++++- .../index/store/FsDirectoryFactory.java | 2 +- .../tsdb/es819/ES819TSDBDocValuesFormatTests.java | 14 +++++--------- 4 files changed, 16 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 2481a063bdf73..7c0cc7c61b9c7 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -55,8 +55,9 @@ import java.util.List; import static org.elasticsearch.index.codec.tsdb.es819.DocValuesConsumerUtil.compatibleWithOptimizedMerge; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; @@ -551,8 +552,7 @@ public void addDoc(BytesRef v) throws IOException { docRanges = ArrayUtil.grow(docRanges, numDocsInCurrentBlock + 1); // need one extra since writing start for next block docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; - int totalUncompressedLength = uncompressedBlockLength + numDocsInCurrentBlock * Integer.BYTES; - if (totalUncompressedLength > MIN_BLOCK_SIZE_BYTES) { + if (uncompressedBlockLength >= BLOCK_BYTES_THRESHOLD || numDocsInCurrentBlock >= BLOCK_COUNT_THRESHOLD) { flushData(); } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index 643e76cf0ba70..bbf59198658be 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -66,7 +66,13 @@ public class ES819TSDBDocValuesFormat extends org.apache.lucene.codecs.DocValues 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; - public static final int MIN_BLOCK_SIZE_BYTES = 256 * 1024; + /** + * These thresholds determine the size of a compressed binary block. We build a new block if the uncompressed data in the block + * is 128k, or if the number of values is 1024. These values are a tradeoff between the high compression ratio and decompression + * speed of large blocks, and the ability to avoid decompressing unneeded values provided by small blocks. + */ + public static final int BLOCK_BYTES_THRESHOLD = 128 * 1024; + public static final int BLOCK_COUNT_THRESHOLD = 1024; // number of documents in an interval private static final int DEFAULT_SKIP_INDEX_INTERVAL_SIZE = 4096; diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java index cd1deec68a342..dc33b28e68530 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java @@ -302,7 +302,7 @@ static boolean avoidDelegateForFdtTempFiles(String name, LuceneFilesExtensions e return extension == LuceneFilesExtensions.TMP && NO_MMAP_FILE_SUFFIXES.stream().anyMatch(name::contains); } - static final List NO_MMAP_FILE_SUFFIXES = List.of("fdt", "disi", "address-data", "block-addresses", "block-doc-ranges"); + static final Set NO_MMAP_FILE_SUFFIXES = Set.of("fdt", "disi", "address-data", "block-addresses", "block-doc-ranges"); MMapDirectory getDelegate() { return delegate; diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 36ae6facbc8af..7cf586323f6e3 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -65,7 +65,7 @@ import java.util.function.Supplier; import java.util.stream.IntStream; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.MIN_BLOCK_SIZE_BYTES; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomFloat; @@ -137,15 +137,13 @@ public void testBlockWiseBinarySparse() throws Exception { var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { - int maxBlocks = 4; - int binaryDataSize = randomIntBetween(0, MIN_BLOCK_SIZE_BYTES * maxBlocks); - + int binaryDataSize = randomIntBetween(0, 4 * BLOCK_BYTES_THRESHOLD); List binaryValues = new ArrayList<>(); int totalSize = 0; while (totalSize < binaryDataSize) { if (randomBoolean()) { final String value = testVeryLargeValues - ? randomAlphaOfLengthBetween(MIN_BLOCK_SIZE_BYTES / 2, 2 * MIN_BLOCK_SIZE_BYTES) + ? randomAlphaOfLengthBetween(BLOCK_BYTES_THRESHOLD / 2, 2 * BLOCK_BYTES_THRESHOLD) : randomAlphaOfLengthBetween(0, 50); binaryValues.add(value); totalSize += value.length(); @@ -204,14 +202,12 @@ public void testBlockWiseBinaryDense() throws Exception { var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { - int maxBlocks = 4; - int binaryDataSize = randomIntBetween(0, MIN_BLOCK_SIZE_BYTES * maxBlocks); - + int binaryDataSize = randomIntBetween(0, 4 * BLOCK_BYTES_THRESHOLD); List binaryValues = new ArrayList<>(); int totalSize = 0; while (totalSize < binaryDataSize) { final String value = testVeryLargeValues - ? randomAlphaOfLengthBetween(MIN_BLOCK_SIZE_BYTES / 2, 2 * MIN_BLOCK_SIZE_BYTES) + ? randomAlphaOfLengthBetween(BLOCK_BYTES_THRESHOLD / 2, 2 * BLOCK_BYTES_THRESHOLD) : randomAlphaOfLengthBetween(0, 50); binaryValues.add(value); From 51b21ae88ce05b8a82041af6d10ee7fe52b437a4 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 7 Nov 2025 17:53:33 +0000 Subject: [PATCH 50/64] [CI] Auto commit changes from spotless --- .../java/org/elasticsearch/index/store/FsDirectoryFactory.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java index dc33b28e68530..1a9670382dc15 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryFactory.java @@ -41,7 +41,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.HashSet; -import java.util.List; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; From 07eeb5adc9bbe8102b4108e45a3a71bfb17c6586 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 7 Nov 2025 12:38:05 -0600 Subject: [PATCH 51/64] Add test for very small binary values --- .../es819/ES819TSDBDocValuesFormatTests.java | 137 ++++++++---------- 1 file changed, 63 insertions(+), 74 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 7cf586323f6e3..479926f62b74a 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -66,9 +66,9 @@ import java.util.stream.IntStream; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; import static org.elasticsearch.test.ESTestCase.randomBoolean; -import static org.elasticsearch.test.ESTestCase.randomFloat; import static org.elasticsearch.test.ESTestCase.randomFrom; import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.hamcrest.Matchers.equalTo; @@ -126,33 +126,76 @@ public void testBinaryCompressionFeatureFlag() { } } - // Test with data large enough to require multiple binary doc value blocks - public void testBlockWiseBinarySparse() throws Exception { + public void testBlockWiseBinary() throws Exception { + boolean sparse = randomBoolean(); + int numBlocksBound = 10; + // Since average size is 25b will hit count threshold rather than size threshold, so use count threshold compute needed docs. + int numNonNullValues = randomIntBetween(0, numBlocksBound * BLOCK_COUNT_THRESHOLD); + + List binaryValues = new ArrayList<>(); + int numNonNull = 0; + while (numNonNull < numNonNullValues) { + if (sparse && randomBoolean()) { + binaryValues.add(null); + } else { + // Average + final String value = randomAlphaOfLengthBetween(0, 50); + binaryValues.add(value); + numNonNull++; + } + } + + assertBinaryValues(binaryValues); + } + + public void testBlockWiseBinarySmallValues() throws Exception { + boolean sparse = randomBoolean(); + int numBlocksBound = 5; + int numNonNullValues = randomIntBetween(0, numBlocksBound * BLOCK_COUNT_THRESHOLD); + + List binaryValues = new ArrayList<>(); + int numNonNull = 0; + while (numNonNull < numNonNullValues) { + if (sparse && randomBoolean()) { + binaryValues.add(null); + } else { + final String value = randomAlphaOfLengthBetween(0, 2); + binaryValues.add(value); + numNonNull++; + } + } + + assertBinaryValues(binaryValues); + } + + public void testBlockWiseBinaryLargeValues() throws Exception { + boolean sparse = randomBoolean(); + int numBlocksBound = 5; + int binaryDataSize = randomIntBetween(0, numBlocksBound * BLOCK_BYTES_THRESHOLD); + List binaryValues = new ArrayList<>(); + int totalSize = 0; + while (totalSize < binaryDataSize) { + if (sparse && randomBoolean()) { + binaryValues.add(null); + } else { + final String value = randomAlphaOfLengthBetween(BLOCK_BYTES_THRESHOLD / 2, 2 * BLOCK_BYTES_THRESHOLD); + binaryValues.add(value); + totalSize += value.length(); + } + } + + assertBinaryValues(binaryValues); + } + + public void assertBinaryValues(List binaryValues) throws Exception { String timestampField = "@timestamp"; String hostnameField = "host.name"; long baseTimestamp = 1704067200000L; String binaryField = "binary_field"; - boolean testVeryLargeValues = randomFloat() < 0.1; - var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { - int binaryDataSize = randomIntBetween(0, 4 * BLOCK_BYTES_THRESHOLD); - List binaryValues = new ArrayList<>(); - int totalSize = 0; - while (totalSize < binaryDataSize) { - if (randomBoolean()) { - final String value = testVeryLargeValues - ? randomAlphaOfLengthBetween(BLOCK_BYTES_THRESHOLD / 2, 2 * BLOCK_BYTES_THRESHOLD) - : randomAlphaOfLengthBetween(0, 50); - binaryValues.add(value); - totalSize += value.length(); - } else { - binaryValues.add(null); - } - } int numDocs = binaryValues.size(); - for (int i = 0; i < numDocs; i++) { var d = new Document(); long timestamp = baseTimestamp + (1000L * i); @@ -191,60 +234,6 @@ public void testBlockWiseBinarySparse() throws Exception { } } - // Test with data large enough to require multiple binary doc value blocks - public void testBlockWiseBinaryDense() throws Exception { - String timestampField = "@timestamp"; - String hostnameField = "host.name"; - long baseTimestamp = 1704067200000L; - String binaryField = "binary_field"; - boolean testVeryLargeValues = randomFloat() < 0.1; - - var config = getTimeSeriesIndexWriterConfig(hostnameField, timestampField); - try (var dir = newDirectory(); var iw = new IndexWriter(dir, config)) { - - int binaryDataSize = randomIntBetween(0, 4 * BLOCK_BYTES_THRESHOLD); - List binaryValues = new ArrayList<>(); - int totalSize = 0; - while (totalSize < binaryDataSize) { - final String value = testVeryLargeValues - ? randomAlphaOfLengthBetween(BLOCK_BYTES_THRESHOLD / 2, 2 * BLOCK_BYTES_THRESHOLD) - : randomAlphaOfLengthBetween(0, 50); - - binaryValues.add(value); - totalSize += value.length(); - } - int numDocs = binaryValues.size(); - - for (int i = 0; i < numDocs; i++) { - var d = new Document(); - long timestamp = baseTimestamp + (1000L * i); - d.add(new SortedDocValuesField(hostnameField, new BytesRef("host-1"))); - d.add(new SortedNumericDocValuesField(timestampField, timestamp)); - - d.add(new BinaryDocValuesField(binaryField, new BytesRef(binaryValues.get(i)))); - - iw.addDocument(d); - if (i % 100 == 0) { - iw.commit(); - } - } - iw.commit(); - iw.forceMerge(1); - - try (var reader = DirectoryReader.open(iw)) { - assertEquals(1, reader.leaves().size()); - assertEquals(numDocs, reader.maxDoc()); - var leaf = reader.leaves().get(0).reader(); - var binaryDV = leaf.getBinaryDocValues(binaryField); - assertNotNull(binaryDV); - for (int i = 0; i < numDocs; i++) { - assertEquals(i, binaryDV.nextDoc()); - assertEquals(binaryValues.removeLast(), binaryDV.binaryValue().utf8ToString()); - } - } - } - } - public void testForceMergeDenseCase() throws Exception { String timestampField = "@timestamp"; String hostnameField = "host.name"; From 980df97bcfc91569b684f3e0db976a84664bbd0c Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 13 Nov 2025 18:30:02 -0600 Subject: [PATCH 52/64] Use groupVarInt instead of TSDB encoder --- .../es819/ES819TSDBDocValuesConsumer.java | 23 ++++++------------- .../es819/ES819TSDBDocValuesProducer.java | 21 +++++++---------- 2 files changed, 15 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 7c0cc7c61b9c7..2afea7c019a9b 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -522,9 +522,7 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { final Compressor compressor; - final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); - final long[] docRangesBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; - int[] docRanges = new int[START_BLOCK_DOCS]; + int[] docOffsets = new int[START_BLOCK_DOCS]; int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -549,8 +547,8 @@ public void addDoc(BytesRef v) throws IOException { uncompressedBlockLength += v.length; numDocsInCurrentBlock++; - docRanges = ArrayUtil.grow(docRanges, numDocsInCurrentBlock + 1); // need one extra since writing start for next block - docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; + docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block + docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; if (uncompressedBlockLength >= BLOCK_BYTES_THRESHOLD || numDocsInCurrentBlock >= BLOCK_COUNT_THRESHOLD) { flushData(); @@ -581,19 +579,12 @@ public void flushData() throws IOException { } void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { - int batchStart = 0; int numOffsets = numDocsInCurrentBlock + 1; - while (batchStart < numOffsets) { - int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); - for (int i = 0; i < batchLength; i++) { - docRangesBuffer[i] = docRanges[batchStart + i]; - } - if (batchLength < docRangesBuffer.length) { - Arrays.fill(docRangesBuffer, batchLength, docRangesBuffer.length, 0); - } - encoder.encode(docRangesBuffer, output); - batchStart += batchLength; + // delta encode + for (int i = numOffsets - 1; i > 0; i--) { + docOffsets[i] -= docOffsets[i - 1]; } + output.writeGroupVInts(docOffsets, numOffsets); } void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index fd1a2abcc581d..b5536fae89eab 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -40,6 +40,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.GroupVIntUtil; import org.apache.lucene.util.LongValues; import org.apache.lucene.util.compress.LZ4; import org.apache.lucene.util.packed.DirectMonotonicReader; @@ -54,7 +55,6 @@ import java.io.IOException; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -380,13 +380,11 @@ public BytesRef binaryValue() throws IOException { // Decompresses blocks of binary values to retrieve content static final class BinaryDecoder { - private final TSDBDocValuesEncoder decoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); private final LongValues addresses; private final DirectMonotonicReader docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final long[] docRangesDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -423,7 +421,7 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - decompressDocRanges(numDocsInBlock, compressedData); + decompressDocOffsets(numDocsInBlock, compressedData); assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; @@ -431,16 +429,13 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); } - void decompressDocRanges(int numDocsInBlock, DataInput input) throws IOException { - int batchStart = 0; + void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { int numOffsets = numDocsInBlock + 1; - while (batchStart < numOffsets) { - decoder.decode(input, docRangesDecompBuffer); - int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); - for (int i = 0; i < lenToCopy; i++) { - uncompressedDocStarts[batchStart + i] = (int) docRangesDecompBuffer[i]; - } - batchStart += NUMERIC_BLOCK_SIZE; + GroupVIntUtil.readGroupVInts(input, uncompressedDocStarts, numOffsets); + + // decode deltas + for (int i = 1; i < numOffsets; ++i) { + uncompressedDocStarts[i] += uncompressedDocStarts[i - 1]; } } From 21a98ac778a471636c67f036b78d9ba2a17f9e82 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 13 Nov 2025 18:54:30 -0600 Subject: [PATCH 53/64] Dont test bulk loading if compressed, as not implemented --- .../es819/ES819TSDBDocValuesFormatTests.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 1dc945b3e84e9..370d73b0e209e 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -81,6 +81,8 @@ public class ES819TSDBDocValuesFormatTests extends ES87TSDBDocValuesFormatTests private final Codec codec = new Elasticsearch92Lucene103Codec() { + + final ES819TSDBDocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat( ESTestCase.randomIntBetween(2, 4096), ESTestCase.randomIntBetween(1, 512), @@ -981,7 +983,8 @@ public void testOptionalColumnAtATimeReader() throws Exception { } } - { + // TODO add bulk loading to compressed values so this is not necessary + if (isCompressed(config, binaryFixedField) == false){ // bulk loading binary fixed length field: var block = (TestBlock) binaryFixedDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); @@ -993,7 +996,8 @@ public void testOptionalColumnAtATimeReader() throws Exception { } } - { + // TODO add bulk loading to compressed values so this is not necessary + if (isCompressed(config, binaryVariableField) == false){ // bulk loading binary variable length field: var block = (TestBlock) binaryVariableDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); @@ -1739,4 +1743,13 @@ public static BinaryDVCompressionMode randomBinaryCompressionMode() { return modes[random().nextInt(modes.length)]; } + private boolean isCompressed(IndexWriterConfig config, String field) { + if (config.getCodec() instanceof Elasticsearch92Lucene103Codec codec) { + if (codec.getDocValuesFormatForField(field) instanceof ES819TSDBDocValuesFormat format) { + return format.binaryDVCompressionMode != BinaryDVCompressionMode.NO_COMPRESS; + } + } + return false; + } + } From 22397321f6b6e574547f74c3bc1c9963d35d062f Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 14 Nov 2025 01:50:07 +0000 Subject: [PATCH 54/64] [CI] Auto commit changes from spotless --- .../codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 1 - .../tsdb/es819/ES819TSDBDocValuesFormatTests.java | 10 ++++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 2afea7c019a9b..01926af17d543 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -58,7 +58,6 @@ import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SORTED_SET; diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 370d73b0e209e..613422a06e2ae 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -68,10 +68,10 @@ import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; -import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; -import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.between; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; +import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; +import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomFrom; import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.hamcrest.Matchers.equalTo; @@ -81,8 +81,6 @@ public class ES819TSDBDocValuesFormatTests extends ES87TSDBDocValuesFormatTests private final Codec codec = new Elasticsearch92Lucene103Codec() { - - final ES819TSDBDocValuesFormat docValuesFormat = new ES819TSDBDocValuesFormat( ESTestCase.randomIntBetween(2, 4096), ESTestCase.randomIntBetween(1, 512), @@ -984,7 +982,7 @@ public void testOptionalColumnAtATimeReader() throws Exception { } // TODO add bulk loading to compressed values so this is not necessary - if (isCompressed(config, binaryFixedField) == false){ + if (isCompressed(config, binaryFixedField) == false) { // bulk loading binary fixed length field: var block = (TestBlock) binaryFixedDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); @@ -997,7 +995,7 @@ public void testOptionalColumnAtATimeReader() throws Exception { } // TODO add bulk loading to compressed values so this is not necessary - if (isCompressed(config, binaryVariableField) == false){ + if (isCompressed(config, binaryVariableField) == false) { // bulk loading binary variable length field: var block = (TestBlock) binaryVariableDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); From 15823e857375755ebeac6ff508cec8b6dcff80dc Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 13 Nov 2025 19:51:23 -0600 Subject: [PATCH 55/64] Fix broken merge --- .../main/java/org/elasticsearch/test/cluster/FeatureFlag.java | 1 - 1 file changed, 1 deletion(-) diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java index 71a75699ec93d..21684da097e66 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/FeatureFlag.java @@ -29,7 +29,6 @@ public enum FeatureFlag { RANDOM_SAMPLING("es.random_sampling_feature_flag_enabled=true", Version.fromString("9.2.0"), null), INFERENCE_API_CCM("es.inference_api_ccm_feature_flag_enabled=true", Version.fromString("9.3.0"), null), GENERIC_VECTOR_FORMAT("es.generic_vector_format_feature_flag_enabled=true", Version.fromString("9.3.0"), null), - INFERENCE_API_CCM("es.inference_api_ccm_feature_flag_enabled=true", Version.fromString("9.3.0"), null), BINARY_DOC_VALUE_COMPRESSION("es.binary_dv_compression_feature_flag_enabled=true", Version.fromString("9.3.0"), null); public final String systemProperty; From 200e14c9cb2f5ebcbfec3c5a290e7dc64120157c Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 14 Nov 2025 18:20:37 -0600 Subject: [PATCH 56/64] Revert to using TSDBDocValueEncoder for offsets --- .../es819/ES819TSDBDocValuesConsumer.java | 24 +++++++++++++------ .../es819/ES819TSDBDocValuesProducer.java | 21 +++++++++------- 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 01926af17d543..7c0cc7c61b9c7 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -58,6 +58,7 @@ import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SORTED_SET; @@ -521,7 +522,9 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { final Compressor compressor; - int[] docOffsets = new int[START_BLOCK_DOCS]; + final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); + final long[] docRangesBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; + int[] docRanges = new int[START_BLOCK_DOCS]; int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -546,8 +549,8 @@ public void addDoc(BytesRef v) throws IOException { uncompressedBlockLength += v.length; numDocsInCurrentBlock++; - docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block - docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; + docRanges = ArrayUtil.grow(docRanges, numDocsInCurrentBlock + 1); // need one extra since writing start for next block + docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; if (uncompressedBlockLength >= BLOCK_BYTES_THRESHOLD || numDocsInCurrentBlock >= BLOCK_COUNT_THRESHOLD) { flushData(); @@ -578,12 +581,19 @@ public void flushData() throws IOException { } void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { + int batchStart = 0; int numOffsets = numDocsInCurrentBlock + 1; - // delta encode - for (int i = numOffsets - 1; i > 0; i--) { - docOffsets[i] -= docOffsets[i - 1]; + while (batchStart < numOffsets) { + int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); + for (int i = 0; i < batchLength; i++) { + docRangesBuffer[i] = docRanges[batchStart + i]; + } + if (batchLength < docRangesBuffer.length) { + Arrays.fill(docRangesBuffer, batchLength, docRangesBuffer.length, 0); + } + encoder.encode(docRangesBuffer, output); + batchStart += batchLength; } - output.writeGroupVInts(docOffsets, numOffsets); } void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index b5536fae89eab..fd1a2abcc581d 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -40,7 +40,6 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.GroupVIntUtil; import org.apache.lucene.util.LongValues; import org.apache.lucene.util.compress.LZ4; import org.apache.lucene.util.packed.DirectMonotonicReader; @@ -55,6 +54,7 @@ import java.io.IOException; +import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; @@ -380,11 +380,13 @@ public BytesRef binaryValue() throws IOException { // Decompresses blocks of binary values to retrieve content static final class BinaryDecoder { + private final TSDBDocValuesEncoder decoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); private final LongValues addresses; private final DirectMonotonicReader docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; + private final long[] docRangesDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -421,7 +423,7 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - decompressDocOffsets(numDocsInBlock, compressedData); + decompressDocRanges(numDocsInBlock, compressedData); assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; @@ -429,13 +431,16 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); } - void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { + void decompressDocRanges(int numDocsInBlock, DataInput input) throws IOException { + int batchStart = 0; int numOffsets = numDocsInBlock + 1; - GroupVIntUtil.readGroupVInts(input, uncompressedDocStarts, numOffsets); - - // decode deltas - for (int i = 1; i < numOffsets; ++i) { - uncompressedDocStarts[i] += uncompressedDocStarts[i - 1]; + while (batchStart < numOffsets) { + decoder.decode(input, docRangesDecompBuffer); + int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); + for (int i = 0; i < lenToCopy; i++) { + uncompressedDocStarts[batchStart + i] = (int) docRangesDecompBuffer[i]; + } + batchStart += NUMERIC_BLOCK_SIZE; } } From 5ca24b4826721e84c214b860f48bf62fd2dd412f Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 14 Nov 2025 18:31:18 -0600 Subject: [PATCH 57/64] Better naming and minor optmization --- .../tsdb/es819/ES819TSDBDocValuesConsumer.java | 17 +++++++++-------- .../tsdb/es819/ES819TSDBDocValuesProducer.java | 6 +++--- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 7c0cc7c61b9c7..0ae0f0de351d2 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -523,8 +523,8 @@ private final class CompressedBinaryBlockWriter implements BinaryWriter { final Compressor compressor; final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); - final long[] docRangesBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; - int[] docRanges = new int[START_BLOCK_DOCS]; + final long[] docOffsetsBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; + int[] docOffsets = new int[START_BLOCK_DOCS]; int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -549,8 +549,8 @@ public void addDoc(BytesRef v) throws IOException { uncompressedBlockLength += v.length; numDocsInCurrentBlock++; - docRanges = ArrayUtil.grow(docRanges, numDocsInCurrentBlock + 1); // need one extra since writing start for next block - docRanges[numDocsInCurrentBlock] = uncompressedBlockLength; + docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block + docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; if (uncompressedBlockLength >= BLOCK_BYTES_THRESHOLD || numDocsInCurrentBlock >= BLOCK_COUNT_THRESHOLD) { flushData(); @@ -586,12 +586,13 @@ void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOExce while (batchStart < numOffsets) { int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < batchLength; i++) { - docRangesBuffer[i] = docRanges[batchStart + i]; + docOffsetsBuffer[i] = docOffsets[batchStart + i]; } - if (batchLength < docRangesBuffer.length) { - Arrays.fill(docRangesBuffer, batchLength, docRangesBuffer.length, 0); + if (batchLength < docOffsetsBuffer.length) { + // fill with last offset so a negative delta doesn't reduce compression ratio + Arrays.fill(docOffsetsBuffer, batchLength, docOffsetsBuffer.length, docOffsetsBuffer[batchLength - 1]); } - encoder.encode(docRangesBuffer, output); + encoder.encode(docOffsetsBuffer, output); batchStart += batchLength; } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index fd1a2abcc581d..a962684ed00cb 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -386,7 +386,7 @@ static final class BinaryDecoder { private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final long[] docRangesDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; + private final long[] docOffsetsDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -435,10 +435,10 @@ void decompressDocRanges(int numDocsInBlock, DataInput input) throws IOException int batchStart = 0; int numOffsets = numDocsInBlock + 1; while (batchStart < numOffsets) { - decoder.decode(input, docRangesDecompBuffer); + decoder.decode(input, docOffsetsDecompBuffer); int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); for (int i = 0; i < lenToCopy; i++) { - uncompressedDocStarts[batchStart + i] = (int) docRangesDecompBuffer[i]; + uncompressedDocStarts[batchStart + i] = (int) docOffsetsDecompBuffer[i]; } batchStart += NUMERIC_BLOCK_SIZE; } From 7f8fa16d60d3acea8d9969851a3b2082eb69a4b8 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Fri, 14 Nov 2025 18:35:14 -0600 Subject: [PATCH 58/64] Dont need to grow offsets array --- .../index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 0ae0f0de351d2..d16155c58dfb8 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -518,13 +518,11 @@ public void close() throws IOException { } private final class CompressedBinaryBlockWriter implements BinaryWriter { - static final int START_BLOCK_DOCS = 1024; - final Compressor compressor; final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); final long[] docOffsetsBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; - int[] docOffsets = new int[START_BLOCK_DOCS]; + final int[] docOffsets = new int[BLOCK_COUNT_THRESHOLD + 1]; // start for each doc plus start of doc that would be after last int uncompressedBlockLength = 0; int maxUncompressedBlockLength = 0; @@ -549,7 +547,6 @@ public void addDoc(BytesRef v) throws IOException { uncompressedBlockLength += v.length; numDocsInCurrentBlock++; - docOffsets = ArrayUtil.grow(docOffsets, numDocsInCurrentBlock + 1); // need one extra since writing start for next block docOffsets[numDocsInCurrentBlock] = uncompressedBlockLength; if (uncompressedBlockLength >= BLOCK_BYTES_THRESHOLD || numDocsInCurrentBlock >= BLOCK_COUNT_THRESHOLD) { From 91c23ee3e6a1b72a26434cde2a0e9027795c3798 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 17 Nov 2025 11:20:32 -0600 Subject: [PATCH 59/64] And back to GroupedVarInt, this time with better delta decoding --- .../es819/ES819TSDBDocValuesConsumer.java | 18 +++---------- .../es819/ES819TSDBDocValuesProducer.java | 26 ++++++++++--------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index d16155c58dfb8..92beeab8b5d49 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -520,8 +520,6 @@ public void close() throws IOException { private final class CompressedBinaryBlockWriter implements BinaryWriter { final Compressor compressor; - final TSDBDocValuesEncoder encoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); - final long[] docOffsetsBuffer = new long[ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE]; final int[] docOffsets = new int[BLOCK_COUNT_THRESHOLD + 1]; // start for each doc plus start of doc that would be after last int uncompressedBlockLength = 0; @@ -578,20 +576,12 @@ public void flushData() throws IOException { } void compressOffsets(DataOutput output, int numDocsInCurrentBlock) throws IOException { - int batchStart = 0; int numOffsets = numDocsInCurrentBlock + 1; - while (batchStart < numOffsets) { - int batchLength = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); - for (int i = 0; i < batchLength; i++) { - docOffsetsBuffer[i] = docOffsets[batchStart + i]; - } - if (batchLength < docOffsetsBuffer.length) { - // fill with last offset so a negative delta doesn't reduce compression ratio - Arrays.fill(docOffsetsBuffer, batchLength, docOffsetsBuffer.length, docOffsetsBuffer[batchLength - 1]); - } - encoder.encode(docOffsetsBuffer, output); - batchStart += batchLength; + // delta encode + for (int i = numOffsets - 1; i > 0; i--) { + docOffsets[i] -= docOffsets[i - 1]; } + output.writeGroupVInts(docOffsets, numOffsets); } void compress(byte[] data, int uncompressedLength, DataOutput output) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index a962684ed00cb..2df92c9b65bd4 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -40,6 +40,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.GroupVIntUtil; import org.apache.lucene.util.LongValues; import org.apache.lucene.util.compress.LZ4; import org.apache.lucene.util.packed.DirectMonotonicReader; @@ -380,13 +381,11 @@ public BytesRef binaryValue() throws IOException { // Decompresses blocks of binary values to retrieve content static final class BinaryDecoder { - private final TSDBDocValuesEncoder decoder = new TSDBDocValuesEncoder(ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE); private final LongValues addresses; private final DirectMonotonicReader docRanges; private final IndexInput compressedData; // Cache of last uncompressed block private long lastBlockId = -1; - private final long[] docOffsetsDecompBuffer = new long[NUMERIC_BLOCK_SIZE]; private final int[] uncompressedDocStarts; private final byte[] uncompressedBlock; private final BytesRef uncompressedBytesRef; @@ -423,7 +422,7 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException return; } - decompressDocRanges(numDocsInBlock, compressedData); + decompressDocOffsets(numDocsInBlock, compressedData); assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; @@ -431,16 +430,19 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); } - void decompressDocRanges(int numDocsInBlock, DataInput input) throws IOException { - int batchStart = 0; + void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { int numOffsets = numDocsInBlock + 1; - while (batchStart < numOffsets) { - decoder.decode(input, docOffsetsDecompBuffer); - int lenToCopy = Math.min(numOffsets - batchStart, NUMERIC_BLOCK_SIZE); - for (int i = 0; i < lenToCopy; i++) { - uncompressedDocStarts[batchStart + i] = (int) docOffsetsDecompBuffer[i]; - } - batchStart += NUMERIC_BLOCK_SIZE; + GroupVIntUtil.readGroupVInts(input, uncompressedDocStarts, numOffsets); + deltaDecode(uncompressedDocStarts, numOffsets); + } + + // Borrowed from to TSDBDocValuesEncoder.decodeDelta + // The `sum` variable helps compiler optimize method, should not be removed. + void deltaDecode(int[] arr, int length) { + int sum = 0; + for (int i = 0; i < length; ++i) { + sum += arr[i]; + arr[i] = sum; } } From 92c8050c17d7937a2594ed366f7bbdbe8a38aa8f Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 17 Nov 2025 11:41:05 -0600 Subject: [PATCH 60/64] Add header to control whether block is compressed or uncompressed --- .../codec/tsdb/BinaryDVCompressionMode.java | 17 +++++++++++++++++ .../tsdb/es819/ES819TSDBDocValuesConsumer.java | 15 +++++++++++++-- .../tsdb/es819/ES819TSDBDocValuesProducer.java | 10 ++++++++-- 3 files changed, 38 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 78fd68239e814..928a9d0badcf4 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -45,4 +45,21 @@ public CompressionMode compressionMode() { } return compressionMode; } + + public record BlockHeader(boolean isCompressed) { + static final int IS_COMPRESSED = 0x1; + + public static BlockHeader fromInt(int header) { + boolean isCompressed = (header & IS_COMPRESSED) != 0; + return new BlockHeader(isCompressed); + } + + public int toInt() { + int header = 0; + if (isCompressed) { + header |= IS_COMPRESSED; + } + return header; + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 92beeab8b5d49..9a2ae29016e48 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -561,14 +561,25 @@ public void flushData() throws IOException { totalChunks++; long thisBlockStartPointer = data.getFilePointer(); + // Data can be compressed or uncompressed on a per-block granularity, though is currently always compressed. + // In the future will leave data uncompressed if compression does not reduce storage. + boolean shouldCompress = true; + var header = new BinaryDVCompressionMode.BlockHeader(shouldCompress); + data.writeVInt(header.toInt()); + // write length of string data - data.writeInt(uncompressedBlockLength); + data.writeVInt(uncompressedBlockLength); maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength); maxNumDocsInAnyBlock = Math.max(maxNumDocsInAnyBlock, numDocsInCurrentBlock); compressOffsets(data, numDocsInCurrentBlock); - compress(block, uncompressedBlockLength, data); + + if (shouldCompress) { + compress(block, uncompressedBlockLength, data); + } else { + data.writeBytes(block, 0, uncompressedBlockLength); + } long blockLenBytes = data.getFilePointer() - thisBlockStartPointer; blockMetaAcc.addDoc(numDocsInCurrentBlock, blockLenBytes); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 2df92c9b65bd4..817b6c18a841b 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -416,7 +416,8 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException long blockStartOffset = addresses.get(blockId); compressedData.seek(blockStartOffset); - int uncompressedBlockLength = compressedData.readInt(); + var header = BinaryDVCompressionMode.BlockHeader.fromInt(compressedData.readVInt()); + int uncompressedBlockLength = compressedData.readVInt(); if (uncompressedBlockLength == 0) { return; @@ -427,7 +428,12 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException assert uncompressedBlockLength <= uncompressedBlock.length; uncompressedBytesRef.offset = 0; uncompressedBytesRef.length = uncompressedBlock.length; - decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); + + if (header.isCompressed()) { + decompressor.decompress(compressedData, uncompressedBlockLength, 0, uncompressedBlockLength, uncompressedBytesRef); + } else { + compressedData.readBytes(uncompressedBlock, 0, uncompressedBlockLength); + } } void decompressDocOffsets(int numDocsInBlock, DataInput input) throws IOException { From 016352aa25470948b93b59300b4a919e54a9d313 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 17 Nov 2025 12:56:21 -0600 Subject: [PATCH 61/64] Handle isCompressed in ES819DocValuesProducer, add bwc tests --- .../tsdb/TSDBDocValuesMergeBenchmark.java | 3 ++- .../codec/tsdb/BinaryDVCompressionMode.java | 8 +++---- .../es819/ES819TSDBDocValuesConsumer.java | 10 +++++---- .../tsdb/es819/ES819TSDBDocValuesFormat.java | 22 ++++++++++++++++--- .../es819/ES819TSDBDocValuesProducer.java | 2 +- .../codec/tsdb/DocValuesCodecDuelTests.java | 3 ++- .../codec/tsdb/TsdbDocValueBwcTests.java | 9 +++++++- .../es819/ES819TSDBDocValuesFormatTests.java | 6 +++-- ...ValuesFormatVariableSkipIntervalTests.java | 6 +++-- 9 files changed, 50 insertions(+), 19 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java index 9c136a72c8101..9c94dcde21bb2 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/index/codec/tsdb/TSDBDocValuesMergeBenchmark.java @@ -262,7 +262,8 @@ private static IndexWriterConfig createIndexWriterConfig(boolean optimizedMergeE 4096, 512, optimizedMergeEnabled, - BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 + BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, + true ); config.setCodec(new Elasticsearch92Lucene103Codec() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java index 928a9d0badcf4..ad5f3b9ef1f98 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/BinaryDVCompressionMode.java @@ -47,15 +47,15 @@ public CompressionMode compressionMode() { } public record BlockHeader(boolean isCompressed) { - static final int IS_COMPRESSED = 0x1; + static final byte IS_COMPRESSED = 0x1; - public static BlockHeader fromInt(int header) { + public static BlockHeader fromByte(byte header) { boolean isCompressed = (header & IS_COMPRESSED) != 0; return new BlockHeader(isCompressed); } - public int toInt() { - int header = 0; + public byte toByte() { + byte header = 0; if (isCompressed) { header |= IS_COMPRESSED; } diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index 9a2ae29016e48..cb26668c35578 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -58,7 +58,6 @@ import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_BYTES_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SORTED_SET; @@ -76,10 +75,12 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { private final int primarySortFieldNumber; final SegmentWriteState state; final BinaryDVCompressionMode binaryDVCompressionMode; + private final boolean enablePerBlockCompression; // only false for testing ES819TSDBDocValuesConsumer( BinaryDVCompressionMode binaryDVCompressionMode, - SegmentWriteState state, + final boolean enablePerBlockCompression, + SegmentWriteState state, int skipIndexIntervalSize, int minDocsPerOrdinalForOrdinalRangeEncoding, boolean enableOptimizedMerge, @@ -89,6 +90,7 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { String metaExtension ) throws IOException { this.binaryDVCompressionMode = binaryDVCompressionMode; + this.enablePerBlockCompression = enablePerBlockCompression; this.state = state; this.termsDictBuffer = new byte[1 << 14]; this.dir = state.directory; @@ -563,9 +565,9 @@ public void flushData() throws IOException { // Data can be compressed or uncompressed on a per-block granularity, though is currently always compressed. // In the future will leave data uncompressed if compression does not reduce storage. - boolean shouldCompress = true; + final boolean shouldCompress = enablePerBlockCompression; var header = new BinaryDVCompressionMode.BlockHeader(shouldCompress); - data.writeVInt(header.toInt()); + data.writeByte(header.toByte()); // write length of string data data.writeVInt(uncompressedBlockLength); diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java index bbf59198658be..dbfbed346bc07 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormat.java @@ -137,6 +137,7 @@ private static boolean getOptimizedMergeEnabledDefault() { final int minDocsPerOrdinalForRangeEncoding; final boolean enableOptimizedMerge; final BinaryDVCompressionMode binaryDVCompressionMode; + final boolean enablePerBlockCompression; /** Default constructor. */ public ES819TSDBDocValuesFormat() { @@ -144,7 +145,8 @@ public ES819TSDBDocValuesFormat() { DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - BINARY_DV_COMPRESSION_FEATURE_FLAG ? BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 : BinaryDVCompressionMode.NO_COMPRESS + BINARY_DV_COMPRESSION_FEATURE_FLAG ? BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1 : BinaryDVCompressionMode.NO_COMPRESS, + true ); } @@ -153,7 +155,18 @@ public ES819TSDBDocValuesFormat(BinaryDVCompressionMode binaryDVCompressionMode) DEFAULT_SKIP_INDEX_INTERVAL_SIZE, ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, OPTIMIZED_MERGE_ENABLE_DEFAULT, - binaryDVCompressionMode + binaryDVCompressionMode, + true + ); + } + + public ES819TSDBDocValuesFormat(BinaryDVCompressionMode binaryDVCompressionMode, boolean enablePerBlockCompression) { + this( + DEFAULT_SKIP_INDEX_INTERVAL_SIZE, + ORDINAL_RANGE_ENCODING_MIN_DOC_PER_ORDINAL, + OPTIMIZED_MERGE_ENABLE_DEFAULT, + binaryDVCompressionMode, + enablePerBlockCompression ); } @@ -162,7 +175,8 @@ public ES819TSDBDocValuesFormat( int skipIndexIntervalSize, int minDocsPerOrdinalForRangeEncoding, boolean enableOptimizedMerge, - BinaryDVCompressionMode binaryDVCompressionMode + BinaryDVCompressionMode binaryDVCompressionMode, + final boolean enablePerBlockCompression ) { super(CODEC_NAME); if (skipIndexIntervalSize < 2) { @@ -172,12 +186,14 @@ public ES819TSDBDocValuesFormat( this.minDocsPerOrdinalForRangeEncoding = minDocsPerOrdinalForRangeEncoding; this.enableOptimizedMerge = enableOptimizedMerge; this.binaryDVCompressionMode = binaryDVCompressionMode; + this.enablePerBlockCompression = enablePerBlockCompression; } @Override public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException { return new ES819TSDBDocValuesConsumer( binaryDVCompressionMode, + enablePerBlockCompression, state, skipIndexIntervalSize, minDocsPerOrdinalForRangeEncoding, diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index 817b6c18a841b..cce9e0f0f890b 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -416,7 +416,7 @@ private void decompressBlock(int blockId, int numDocsInBlock) throws IOException long blockStartOffset = addresses.get(blockId); compressedData.seek(blockStartOffset); - var header = BinaryDVCompressionMode.BlockHeader.fromInt(compressedData.readVInt()); + var header = BinaryDVCompressionMode.BlockHeader.fromByte(compressedData.readByte()); int uncompressedBlockLength = compressedData.readVInt(); if (uncompressedBlockLength == 0) { diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java index 2eee88c414733..ab7e65959d686 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/DocValuesCodecDuelTests.java @@ -63,7 +63,8 @@ public void testDuel() throws IOException { ESTestCase.randomIntBetween(1, 4096), ESTestCase.randomIntBetween(1, 512), random().nextBoolean(), - ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(), + random().nextBoolean() ) : new TestES87TSDBDocValuesFormat(); diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index 21a12c414a290..73b614b97747b 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -80,6 +80,12 @@ public void testMixedIndexDocValueBinaryCompressionFeatureDisabledOldCodec() thr testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); } + public void testMixedIndexDocValueBinaryPerBlockCompression() throws Exception { + var oldCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean())); + var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean())); + testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); + } + public void testMixedIndex816To900Lucene101() throws Exception { var oldCodec = new Elasticsearch816Codec() { @@ -339,7 +345,8 @@ public void testEncodeOrdinalRange() throws IOException { random().nextInt(16, 128), nextOrdinalRangeThreshold.getAsInt(), random().nextBoolean(), - ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(), + randomBoolean() ) ) ); diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index 613422a06e2ae..07dc33a8960c8 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -85,7 +85,8 @@ public class ES819TSDBDocValuesFormatTests extends ES87TSDBDocValuesFormatTests ESTestCase.randomIntBetween(2, 4096), ESTestCase.randomIntBetween(1, 512), random().nextBoolean(), - randomBinaryCompressionMode() + randomBinaryCompressionMode(), + true ); @Override @@ -1362,7 +1363,8 @@ public void testLoadKeywordFieldWithIndexSorts() throws IOException { ESTestCase.randomIntBetween(2, 4096), 1, // always enable range-encode random().nextBoolean(), - randomBinaryCompressionMode() + randomBinaryCompressionMode(), + randomBoolean() ); @Override diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java index cca5764aa5982..d43e253729a64 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatVariableSkipIntervalTests.java @@ -23,7 +23,8 @@ protected Codec getCodec() { random().nextInt(4, 16), random().nextInt(1, 32), random().nextBoolean(), - ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(), + random().nextBoolean() ) ); } @@ -35,7 +36,8 @@ public void testSkipIndexIntervalSize() { random().nextInt(Integer.MIN_VALUE, 2), random().nextInt(1, 32), random().nextBoolean(), - ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode() + ES819TSDBDocValuesFormatTests.randomBinaryCompressionMode(), + random().nextBoolean() ) ); assertTrue(ex.getMessage().contains("skipIndexIntervalSize must be > 1")); From 026406b86caf6fc996673022ca3ff86bae093745 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 17 Nov 2025 19:05:43 +0000 Subject: [PATCH 62/64] [CI] Auto commit changes from spotless --- .../codec/tsdb/es819/ES819TSDBDocValuesConsumer.java | 2 +- .../codec/tsdb/es819/ES819TSDBDocValuesProducer.java | 1 - .../index/codec/tsdb/TsdbDocValueBwcTests.java | 8 ++++++-- .../codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java | 1 - 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java index cb26668c35578..c75fd6a470add 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesConsumer.java @@ -80,7 +80,7 @@ final class ES819TSDBDocValuesConsumer extends XDocValuesConsumer { ES819TSDBDocValuesConsumer( BinaryDVCompressionMode binaryDVCompressionMode, final boolean enablePerBlockCompression, - SegmentWriteState state, + SegmentWriteState state, int skipIndexIntervalSize, int minDocsPerOrdinalForOrdinalRangeEncoding, boolean enableOptimizedMerge, diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java index a290536e00e40..8870e0220ad25 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesProducer.java @@ -55,7 +55,6 @@ import java.io.IOException; -import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.NUMERIC_BLOCK_SIZE; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java index 73b614b97747b..f9dc86e181638 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/TsdbDocValueBwcTests.java @@ -81,8 +81,12 @@ public void testMixedIndexDocValueBinaryCompressionFeatureDisabledOldCodec() thr } public void testMixedIndexDocValueBinaryPerBlockCompression() throws Exception { - var oldCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean())); - var newCodec = TestUtil.alwaysDocValuesFormat(new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean())); + var oldCodec = TestUtil.alwaysDocValuesFormat( + new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean()) + ); + var newCodec = TestUtil.alwaysDocValuesFormat( + new ES819TSDBDocValuesFormat(BinaryDVCompressionMode.COMPRESSED_ZSTD_LEVEL_1, randomBoolean()) + ); testMixedIndex(oldCodec, newCodec, this::assertVersion819, this::assertVersion819); } diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index fc11bed619753..f7ebda1855b01 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -70,7 +70,6 @@ import static org.elasticsearch.index.codec.tsdb.es819.ES819TSDBDocValuesFormat.BLOCK_COUNT_THRESHOLD; import static org.elasticsearch.test.ESTestCase.between; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; -import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLengthBetween; import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomFrom; From d27bb8b31bfa4ce9cd596f1b6155d34e101734ce Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Mon, 17 Nov 2025 13:56:35 -0600 Subject: [PATCH 63/64] Skip bulk loading tests if compressed --- .../es819/ES819TSDBDocValuesFormatTests.java | 39 ++++++++++++------- 1 file changed, 25 insertions(+), 14 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java index f7ebda1855b01..7f65cb9af3ad5 100644 --- a/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java +++ b/server/src/test/java/org/elasticsearch/index/codec/tsdb/es819/ES819TSDBDocValuesFormatTests.java @@ -984,9 +984,11 @@ public void testOptionalColumnAtATimeReader() throws Exception { } // TODO add bulk loading to compressed values so this is not necessary - if (isCompressed(config, binaryFixedField) == false) { + var block = (TestBlock) binaryFixedDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); + if (isCompressed(config, binaryFixedField)) { + assertNull(block); + } else { // bulk loading binary fixed length field: - var block = (TestBlock) binaryFixedDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); assertEquals(size, block.size()); for (int j = 0; j < block.size(); j++) { @@ -997,9 +999,11 @@ public void testOptionalColumnAtATimeReader() throws Exception { } // TODO add bulk loading to compressed values so this is not necessary - if (isCompressed(config, binaryVariableField) == false) { + block = (TestBlock) binaryVariableDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); + if (isCompressed(config, binaryVariableField)) { + assertNull(block); + } else { // bulk loading binary variable length field: - var block = (TestBlock) binaryVariableDV.tryRead(factory, docs, 0, random().nextBoolean(), null, false); assertNotNull(block); assertEquals(size, block.size()); for (int j = 0; j < block.size(); j++) { @@ -1367,25 +1371,32 @@ public void testOptionalColumnAtATimeReaderWithSparseDocs() throws Exception { { // Bulk binary loader can only handle sparse queries over dense documents List testDocs = IntStream.range(0, numDocs - 1).filter(i -> randomBoolean()).boxed().toList(); + docs = TestBlock.docs(testDocs.stream().mapToInt(n -> n).toArray()); if (testDocs.isEmpty() == false) { { - // fixed length - docs = TestBlock.docs(testDocs.stream().mapToInt(n -> n).toArray()); var dv = getDenseBinaryValues(leafReader, binaryFixedField); var block = (TestBlock) dv.tryRead(factory, docs, 0, random().nextBoolean(), null, false); - assertNotNull(block); - for (int i = 0; i < testDocs.size(); i++) { - assertThat(block.get(i), equalTo(binaryFixed[testDocs.get(i)])); + // TODO add bulk loading to compressed values so this is not necessary + if (isCompressed(config, binaryFixedField)) { + assertNull(block); + } else { + assertNotNull(block); + for (int i = 0; i < testDocs.size(); i++) { + assertThat(block.get(i), equalTo(binaryFixed[testDocs.get(i)])); + } } } { - // variable length - docs = TestBlock.docs(testDocs.stream().mapToInt(n -> n).toArray()); var dv = getDenseBinaryValues(leafReader, binaryVariableField); var block = (TestBlock) dv.tryRead(factory, docs, 0, random().nextBoolean(), null, false); - assertNotNull(block); - for (int i = 0; i < testDocs.size(); i++) { - assertThat(block.get(i), equalTo(binaryVariable[testDocs.get(i)])); + // TODO add bulk loading to compressed values so this is not necessary + if (isCompressed(config, binaryVariableField)) { + assertNull(block); + } else { + assertNotNull(block); + for (int i = 0; i < testDocs.size(); i++) { + assertThat(block.get(i), equalTo(binaryVariable[testDocs.get(i)])); + } } } } From db68af65ba1352f044d0b6a417d441285b4b6892 Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Tue, 18 Nov 2025 17:01:53 -0600 Subject: [PATCH 64/64] review feedback --- docs/changelog/137139.yaml | 27 ------------------- .../tsdb/es819/BlockMetadataAccumulator.java | 2 +- 2 files changed, 1 insertion(+), 28 deletions(-) delete mode 100644 docs/changelog/137139.yaml diff --git a/docs/changelog/137139.yaml b/docs/changelog/137139.yaml deleted file mode 100644 index e15e9c02f8ce7..0000000000000 --- a/docs/changelog/137139.yaml +++ /dev/null @@ -1,27 +0,0 @@ -pr: 137139 -summary: Add binary doc value compression with variable doc count blocks -area: Mapping -type: feature -issues: [] -highlight: - title: Add binary doc value compression with variable doc count blocks - body: "Add compression for binary doc values using Zstd and blocks with a\nvariable\ - \ number of values.\n\nBlock-wise LZ4 was previously added to Lucene in\n[LUCENE-9211](https://issues.apache.org/jira/browse/LUCENE-9211).\ - \ This\nwas subsequently removed in\n[LUCENE-9378](https://issues.apache.org/jira/browse/LUCENE-9378)\ - \ due to\nquery performance issues. \n\nWe investigated adding to adding the original\ - \ Lucene implementation to\nES in https://github.com/elastic/elasticsearch/pull/112416\ - \ and\nhttps://github.com/elastic/elasticsearch/pull/105301. This approach\nstores\ - \ a constant number of values per block (specifically 32 values).\nThis is nice\ - \ because it makes it very easy to map a given value index\n(eg docId for dense\ - \ values) to the block containing it with `blockId =\ndocId / 32`. Unfortunately,\ - \ if values are very large we cannot reduce\nthe number of values per block and\ - \ (de)compressing a block could cause\nan OOM. Also, since this is a concern,\ - \ we have to keep the number of\nvalues lower than ideal.\n\nThis PR instead stores\ - \ a variable number of documents per block. It\nstores a minimum of 1 document\ - \ per block and stops adding values when\nthe size of a block exceeds a threshold.\ - \ Like the previous version is\nstores an array of address for the start of each\ - \ block. Additionally, it\nstores are parallel array with the value index at the\ - \ start of each\nblock. When looking up a given value index, if it is not in the\ - \ current\nblock, we binary search the array of value index starts to find the\n\ - blockId containing the value. Then look up the address of the block." - notable: true diff --git a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java index 54cc127c8b66e..3f9a5a5574864 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java +++ b/server/src/main/java/org/elasticsearch/index/codec/tsdb/es819/BlockMetadataAccumulator.java @@ -20,7 +20,7 @@ import java.io.Closeable; import java.io.IOException; -public final class BlockMetadataAccumulator implements Closeable { +final class BlockMetadataAccumulator implements Closeable { private final DelayedOffsetAccumulator blockAddressAcc; private final DelayedOffsetAccumulator blockDocRangeAcc;