From acee2aef1fbcd5c71c3df93f6f044659bdc26600 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 28 Aug 2024 17:01:01 -0400 Subject: [PATCH 01/10] Fix Netty's ByteBuf leak (#15475) Signed-off-by: Andriy Redko --- .../netty4/ReactorNetty4HttpChunk.java | 26 +++++++------------ .../ReactorNetty4StreamingHttpChannel.java | 1 + ...ReactorNetty4StreamingRequestConsumer.java | 2 +- 3 files changed, 11 insertions(+), 18 deletions(-) diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpChunk.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpChunk.java index 3b4a308691e7b..1c4c1fd3e49e3 100644 --- a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpChunk.java +++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpChunk.java @@ -8,39 +8,31 @@ package org.opensearch.http.reactor.netty4; +import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.http.HttpChunk; -import org.opensearch.transport.reactor.netty4.Netty4Utils; - -import java.util.concurrent.atomic.AtomicBoolean; import io.netty.buffer.ByteBuf; class ReactorNetty4HttpChunk implements HttpChunk { - private final AtomicBoolean released; - private final boolean pooled; - private final ByteBuf content; + private final BytesArray content; private final boolean last; - ReactorNetty4HttpChunk(ByteBuf content, boolean last) { - this.content = content; - this.pooled = true; - this.released = new AtomicBoolean(false); + ReactorNetty4HttpChunk(ByteBuf buf, boolean last) { + // Since the chunks could be batched and processing could be delayed, we are copying the content here + final byte[] content = new byte[buf.readableBytes()]; + buf.readBytes(content); + this.content = new BytesArray(content); this.last = last; } @Override public BytesReference content() { - assert released.get() == false; - return Netty4Utils.toBytesReference(content); + return content; } @Override - public void close() { - if (pooled && released.compareAndSet(false, true)) { - content.release(); - } - } + public void close() {} @Override public boolean isLast() { diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingHttpChannel.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingHttpChannel.java index 12ed847c0c0de..1aa03aa9967e2 100644 --- a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingHttpChannel.java +++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingHttpChannel.java @@ -103,6 +103,7 @@ public void receiveChunk(HttpChunk message) { } } catch (final Exception ex) { producer.error(ex); + } finally { message.close(); } } diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingRequestConsumer.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingRequestConsumer.java index 282a82dc39fda..0559f89478838 100644 --- a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingRequestConsumer.java +++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4StreamingRequestConsumer.java @@ -44,7 +44,7 @@ public void subscribe(Subscriber s) { } HttpChunk createChunk(HttpContent chunk, boolean last) { - return new ReactorNetty4HttpChunk(chunk.copy().content().retain(), last); + return new ReactorNetty4HttpChunk(chunk.content(), last); } StreamingHttpChannel httpChannel() { From 5663b4ae5c5553b358226f15f17b89aa843f9479 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Wed, 28 Aug 2024 18:57:14 -0500 Subject: [PATCH 02/10] Change abstraction point for transport protocol (#15432) * Revert "Replacing InboundMessage with NativeInboundMessage for deprecation (#13126)" This reverts commit f5c3ef9fa329df83083dc607ccdb74f5c65b3198. Signed-off-by: Andrew Ross * Change abstraction point for transport protocol The previous implementation had a transport switch point in InboundPipeline when the bytes were initially pulled off the wire. There was no implementation for any other protocol as the `canHandleBytes` method was hardcoded to return true. I believe this is the wrong point to switch on the protocol. This change makes NativeInboundBytesHandler protocol agnostic beyond the header. With this change, a complete message is parsed from the stream of bytes, with the header schema being unchanged from what exists today. The protocol switch point will now be at `InboundHandler::inboundMessage`. The header will indicate what protocol was used to serialize the the non-header bytes of the message and then invoke the appropriate handler based on that field. Signed-off-by: Andrew Ross --------- Signed-off-by: Andrew Ross --- .../java/org/opensearch/transport/Header.java | 10 +- .../transport/InboundAggregator.java | 7 +- .../transport/InboundBytesHandler.java | 137 +++++++++++++-- .../opensearch/transport/InboundDecoder.java | 5 +- .../opensearch/transport/InboundHandler.java | 13 +- .../opensearch/transport/InboundMessage.java | 149 ++++++++++++++++ .../opensearch/transport/InboundPipeline.java | 36 +--- .../transport/NativeMessageHandler.java | 7 +- .../opensearch/transport/TcpTransport.java | 10 +- .../opensearch/transport/TransportLogger.java | 5 +- .../transport/TransportProtocol.java | 29 +++ .../NativeInboundBytesHandler.java | 166 ------------------ .../nativeprotocol/NativeInboundMessage.java | 100 +---------- .../transport/InboundAggregatorTests.java | 45 +++-- .../transport/InboundHandlerTests.java | 114 ++++++------ .../transport/InboundPipelineTests.java | 8 +- .../transport/NativeOutboundHandlerTests.java | 6 +- .../transport/TransportProtocolTests.java | 22 +++ 18 files changed, 467 insertions(+), 402 deletions(-) create mode 100644 server/src/main/java/org/opensearch/transport/InboundMessage.java create mode 100644 server/src/main/java/org/opensearch/transport/TransportProtocol.java delete mode 100644 server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundBytesHandler.java create mode 100644 server/src/test/java/org/opensearch/transport/TransportProtocolTests.java diff --git a/server/src/main/java/org/opensearch/transport/Header.java b/server/src/main/java/org/opensearch/transport/Header.java index ac30df8dda02c..fcfeb9c632075 100644 --- a/server/src/main/java/org/opensearch/transport/Header.java +++ b/server/src/main/java/org/opensearch/transport/Header.java @@ -55,6 +55,7 @@ public class Header { private static final String RESPONSE_NAME = "NO_ACTION_NAME_FOR_RESPONSES"; + private final TransportProtocol protocol; private final int networkMessageSize; private final Version version; private final long requestId; @@ -64,13 +65,18 @@ public class Header { Tuple, Map>> headers; Set features; - Header(int networkMessageSize, long requestId, byte status, Version version) { + Header(TransportProtocol protocol, int networkMessageSize, long requestId, byte status, Version version) { + this.protocol = protocol; this.networkMessageSize = networkMessageSize; this.version = version; this.requestId = requestId; this.status = status; } + TransportProtocol getTransportProtocol() { + return protocol; + } + public int getNetworkMessageSize() { return networkMessageSize; } @@ -142,6 +148,8 @@ void finishParsingHeader(StreamInput input) throws IOException { @Override public String toString() { return "Header{" + + protocol + + "}{" + networkMessageSize + "}{" + version diff --git a/server/src/main/java/org/opensearch/transport/InboundAggregator.java b/server/src/main/java/org/opensearch/transport/InboundAggregator.java index f52875d880b4f..e894331f3b64e 100644 --- a/server/src/main/java/org/opensearch/transport/InboundAggregator.java +++ b/server/src/main/java/org/opensearch/transport/InboundAggregator.java @@ -40,7 +40,6 @@ import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.bytes.CompositeBytesReference; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import java.io.IOException; import java.util.ArrayList; @@ -114,7 +113,7 @@ public void aggregate(ReleasableBytesReference content) { } } - public NativeInboundMessage finishAggregation() throws IOException { + public InboundMessage finishAggregation() throws IOException { ensureOpen(); final ReleasableBytesReference releasableContent; if (isFirstContent()) { @@ -128,7 +127,7 @@ public NativeInboundMessage finishAggregation() throws IOException { } final BreakerControl breakerControl = new BreakerControl(circuitBreaker); - final NativeInboundMessage aggregated = new NativeInboundMessage(currentHeader, releasableContent, breakerControl); + final InboundMessage aggregated = new InboundMessage(currentHeader, releasableContent, breakerControl); boolean success = false; try { if (aggregated.getHeader().needsToReadVariableHeader()) { @@ -143,7 +142,7 @@ public NativeInboundMessage finishAggregation() throws IOException { if (isShortCircuited()) { aggregated.close(); success = true; - return new NativeInboundMessage(aggregated.getHeader(), aggregationException); + return new InboundMessage(aggregated.getHeader(), aggregationException); } else { success = true; return aggregated; diff --git a/server/src/main/java/org/opensearch/transport/InboundBytesHandler.java b/server/src/main/java/org/opensearch/transport/InboundBytesHandler.java index 276891212e43f..ad839bc990018 100644 --- a/server/src/main/java/org/opensearch/transport/InboundBytesHandler.java +++ b/server/src/main/java/org/opensearch/transport/InboundBytesHandler.java @@ -9,24 +9,139 @@ package org.opensearch.transport; import org.opensearch.common.bytes.ReleasableBytesReference; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; +import org.opensearch.core.common.bytes.CompositeBytesReference; -import java.io.Closeable; import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.function.BiConsumer; /** - * Interface for handling inbound bytes. Can be implemented by different transport protocols. + * Handler for inbound bytes, using {@link InboundDecoder} to decode headers + * and {@link InboundAggregator} to assemble complete messages to forward to + * the given message handler to parse the message payload. */ -public interface InboundBytesHandler extends Closeable { +class InboundBytesHandler { - public void doHandleBytes( - TcpChannel channel, - ReleasableBytesReference reference, - BiConsumer messageHandler - ) throws IOException; + private static final ThreadLocal> fragmentList = ThreadLocal.withInitial(ArrayList::new); - public boolean canHandleBytes(ReleasableBytesReference reference); + private final ArrayDeque pending; + private final InboundDecoder decoder; + private final InboundAggregator aggregator; + private final StatsTracker statsTracker; + private boolean isClosed = false; + + InboundBytesHandler( + ArrayDeque pending, + InboundDecoder decoder, + InboundAggregator aggregator, + StatsTracker statsTracker + ) { + this.pending = pending; + this.decoder = decoder; + this.aggregator = aggregator; + this.statsTracker = statsTracker; + } + + public void close() { + isClosed = true; + } + + public void doHandleBytes(TcpChannel channel, ReleasableBytesReference reference, BiConsumer messageHandler) + throws IOException { + final ArrayList fragments = fragmentList.get(); + boolean continueHandling = true; + + while (continueHandling && isClosed == false) { + boolean continueDecoding = true; + while (continueDecoding && pending.isEmpty() == false) { + try (ReleasableBytesReference toDecode = getPendingBytes()) { + final int bytesDecoded = decoder.decode(toDecode, fragments::add); + if (bytesDecoded != 0) { + releasePendingBytes(bytesDecoded); + if (fragments.isEmpty() == false && endOfMessage(fragments.get(fragments.size() - 1))) { + continueDecoding = false; + } + } else { + continueDecoding = false; + } + } + } + + if (fragments.isEmpty()) { + continueHandling = false; + } else { + try { + forwardFragments(channel, fragments, messageHandler); + } finally { + for (Object fragment : fragments) { + if (fragment instanceof ReleasableBytesReference) { + ((ReleasableBytesReference) fragment).close(); + } + } + fragments.clear(); + } + } + } + } + + private ReleasableBytesReference getPendingBytes() { + if (pending.size() == 1) { + return pending.peekFirst().retain(); + } else { + final ReleasableBytesReference[] bytesReferences = new ReleasableBytesReference[pending.size()]; + int index = 0; + for (ReleasableBytesReference pendingReference : pending) { + bytesReferences[index] = pendingReference.retain(); + ++index; + } + final Releasable releasable = () -> Releasables.closeWhileHandlingException(bytesReferences); + return new ReleasableBytesReference(CompositeBytesReference.of(bytesReferences), releasable); + } + } + + private void releasePendingBytes(int bytesConsumed) { + int bytesToRelease = bytesConsumed; + while (bytesToRelease != 0) { + try (ReleasableBytesReference reference = pending.pollFirst()) { + assert reference != null; + if (bytesToRelease < reference.length()) { + pending.addFirst(reference.retainedSlice(bytesToRelease, reference.length() - bytesToRelease)); + bytesToRelease -= bytesToRelease; + } else { + bytesToRelease -= reference.length(); + } + } + } + } + + private boolean endOfMessage(Object fragment) { + return fragment == InboundDecoder.PING || fragment == InboundDecoder.END_CONTENT || fragment instanceof Exception; + } + + private void forwardFragments(TcpChannel channel, ArrayList fragments, BiConsumer messageHandler) + throws IOException { + for (Object fragment : fragments) { + if (fragment instanceof Header) { + assert aggregator.isAggregating() == false; + aggregator.headerReceived((Header) fragment); + } else if (fragment == InboundDecoder.PING) { + assert aggregator.isAggregating() == false; + messageHandler.accept(channel, InboundMessage.PING); + } else if (fragment == InboundDecoder.END_CONTENT) { + assert aggregator.isAggregating(); + try (InboundMessage aggregated = aggregator.finishAggregation()) { + statsTracker.markMessageReceived(); + messageHandler.accept(channel, aggregated); + } + } else { + assert aggregator.isAggregating(); + assert fragment instanceof ReleasableBytesReference; + aggregator.aggregate((ReleasableBytesReference) fragment); + } + } + } - @Override - void close(); } diff --git a/server/src/main/java/org/opensearch/transport/InboundDecoder.java b/server/src/main/java/org/opensearch/transport/InboundDecoder.java index d6b7a98e876b3..3e735d4be2420 100644 --- a/server/src/main/java/org/opensearch/transport/InboundDecoder.java +++ b/server/src/main/java/org/opensearch/transport/InboundDecoder.java @@ -187,11 +187,12 @@ private int headerBytesToRead(BytesReference reference) { // exposed for use in tests static Header readHeader(Version version, int networkMessageSize, BytesReference bytesReference) throws IOException { try (StreamInput streamInput = bytesReference.streamInput()) { - streamInput.skip(TcpHeader.BYTES_REQUIRED_FOR_MESSAGE_SIZE); + TransportProtocol protocol = TransportProtocol.fromBytes(streamInput.readByte(), streamInput.readByte()); + streamInput.skip(TcpHeader.MESSAGE_LENGTH_SIZE); long requestId = streamInput.readLong(); byte status = streamInput.readByte(); Version remoteVersion = Version.fromId(streamInput.readInt()); - Header header = new Header(networkMessageSize, requestId, status, remoteVersion); + Header header = new Header(protocol, networkMessageSize, requestId, status, remoteVersion); final IllegalStateException invalidVersion = ensureVersionCompatibility(remoteVersion, version, header.isHandshake()); if (invalidVersion != null) { throw invalidVersion; diff --git a/server/src/main/java/org/opensearch/transport/InboundHandler.java b/server/src/main/java/org/opensearch/transport/InboundHandler.java index f77c44ea362cf..76a44832b08dc 100644 --- a/server/src/main/java/org/opensearch/transport/InboundHandler.java +++ b/server/src/main/java/org/opensearch/transport/InboundHandler.java @@ -38,7 +38,6 @@ import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import java.io.IOException; import java.util.Map; @@ -56,7 +55,7 @@ public class InboundHandler { private volatile long slowLogThresholdMs = Long.MAX_VALUE; - private final Map protocolMessageHandlers; + private final Map protocolMessageHandlers; InboundHandler( String nodeName, @@ -75,7 +74,7 @@ public class InboundHandler { ) { this.threadPool = threadPool; this.protocolMessageHandlers = Map.of( - NativeInboundMessage.NATIVE_PROTOCOL, + TransportProtocol.NATIVE, new NativeMessageHandler( nodeName, version, @@ -107,16 +106,16 @@ void setSlowLogThreshold(TimeValue slowLogThreshold) { this.slowLogThresholdMs = slowLogThreshold.getMillis(); } - void inboundMessage(TcpChannel channel, ProtocolInboundMessage message) throws Exception { + void inboundMessage(TcpChannel channel, InboundMessage message) throws Exception { final long startTime = threadPool.relativeTimeInMillis(); channel.getChannelStats().markAccessed(startTime); messageReceivedFromPipeline(channel, message, startTime); } - private void messageReceivedFromPipeline(TcpChannel channel, ProtocolInboundMessage message, long startTime) throws IOException { - ProtocolMessageHandler protocolMessageHandler = protocolMessageHandlers.get(message.getProtocol()); + private void messageReceivedFromPipeline(TcpChannel channel, InboundMessage message, long startTime) throws IOException { + ProtocolMessageHandler protocolMessageHandler = protocolMessageHandlers.get(message.getTransportProtocol()); if (protocolMessageHandler == null) { - throw new IllegalStateException("No protocol message handler found for protocol: " + message.getProtocol()); + throw new IllegalStateException("No protocol message handler found for protocol: " + message.getTransportProtocol()); } protocolMessageHandler.messageReceived(channel, message, startTime, slowLogThresholdMs, messageListener); } diff --git a/server/src/main/java/org/opensearch/transport/InboundMessage.java b/server/src/main/java/org/opensearch/transport/InboundMessage.java new file mode 100644 index 0000000000000..576ab73ce9c98 --- /dev/null +++ b/server/src/main/java/org/opensearch/transport/InboundMessage.java @@ -0,0 +1,149 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.transport; + +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.bytes.ReleasableBytesReference; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * Inbound data as a message + */ +@PublicApi(since = "1.0.0") +public class InboundMessage implements Releasable, ProtocolInboundMessage { + + static final InboundMessage PING = new InboundMessage(null, null, null, true, null); + + protected final Header header; + protected final ReleasableBytesReference content; + protected final Exception exception; + protected final boolean isPing; + private Releasable breakerRelease; + private StreamInput streamInput; + + public InboundMessage(Header header, ReleasableBytesReference content, Releasable breakerRelease) { + this(header, content, null, false, breakerRelease); + } + + public InboundMessage(Header header, Exception exception) { + this(header, null, exception, false, null); + } + + public InboundMessage(Header header, boolean isPing) { + this(header, null, null, isPing, null); + } + + private InboundMessage( + Header header, + ReleasableBytesReference content, + Exception exception, + boolean isPing, + Releasable breakerRelease + ) { + this.header = header; + this.content = content; + this.exception = exception; + this.isPing = isPing; + this.breakerRelease = breakerRelease; + } + + TransportProtocol getTransportProtocol() { + if (isPing) { + return TransportProtocol.NATIVE; + } + return header.getTransportProtocol(); + } + + public String getProtocol() { + return header.getTransportProtocol().toString(); + } + + public Header getHeader() { + return header; + } + + public int getContentLength() { + if (content == null) { + return 0; + } else { + return content.length(); + } + } + + public Exception getException() { + return exception; + } + + public boolean isPing() { + return isPing; + } + + public boolean isShortCircuit() { + return exception != null; + } + + public Releasable takeBreakerReleaseControl() { + final Releasable toReturn = breakerRelease; + breakerRelease = null; + if (toReturn != null) { + return toReturn; + } else { + return () -> {}; + } + } + + public StreamInput openOrGetStreamInput() throws IOException { + assert isPing == false && content != null; + if (streamInput == null) { + streamInput = content.streamInput(); + streamInput.setVersion(header.getVersion()); + } + return streamInput; + } + + @Override + public void close() { + IOUtils.closeWhileHandlingException(streamInput); + Releasables.closeWhileHandlingException(content, breakerRelease); + } + + @Override + public String toString() { + return "InboundMessage{" + header + "}"; + } +} diff --git a/server/src/main/java/org/opensearch/transport/InboundPipeline.java b/server/src/main/java/org/opensearch/transport/InboundPipeline.java index 5cee3bb975223..3acb43f58b443 100644 --- a/server/src/main/java/org/opensearch/transport/InboundPipeline.java +++ b/server/src/main/java/org/opensearch/transport/InboundPipeline.java @@ -38,11 +38,9 @@ import org.opensearch.common.lease.Releasables; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.core.common.breaker.CircuitBreaker; -import org.opensearch.transport.nativeprotocol.NativeInboundBytesHandler; import java.io.IOException; import java.util.ArrayDeque; -import java.util.List; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.LongSupplier; @@ -62,9 +60,8 @@ public class InboundPipeline implements Releasable { private Exception uncaughtException; private final ArrayDeque pending = new ArrayDeque<>(2); private boolean isClosed = false; - private final BiConsumer messageHandler; - private final List protocolBytesHandlers; - private InboundBytesHandler currentHandler; + private final BiConsumer messageHandler; + private final InboundBytesHandler bytesHandler; public InboundPipeline( Version version, @@ -73,7 +70,7 @@ public InboundPipeline( LongSupplier relativeTimeInMillis, Supplier circuitBreaker, Function> registryFunction, - BiConsumer messageHandler + BiConsumer messageHandler ) { this( statsTracker, @@ -89,23 +86,20 @@ public InboundPipeline( LongSupplier relativeTimeInMillis, InboundDecoder decoder, InboundAggregator aggregator, - BiConsumer messageHandler + BiConsumer messageHandler ) { this.relativeTimeInMillis = relativeTimeInMillis; this.statsTracker = statsTracker; this.decoder = decoder; this.aggregator = aggregator; - this.protocolBytesHandlers = List.of(new NativeInboundBytesHandler(pending, decoder, aggregator, statsTracker)); + this.bytesHandler = new InboundBytesHandler(pending, decoder, aggregator, statsTracker); this.messageHandler = messageHandler; } @Override public void close() { isClosed = true; - if (currentHandler != null) { - currentHandler.close(); - currentHandler = null; - } + bytesHandler.close(); Releasables.closeWhileHandlingException(decoder, aggregator); Releasables.closeWhileHandlingException(pending); pending.clear(); @@ -127,22 +121,6 @@ public void doHandleBytes(TcpChannel channel, ReleasableBytesReference reference channel.getChannelStats().markAccessed(relativeTimeInMillis.getAsLong()); statsTracker.markBytesRead(reference.length()); pending.add(reference.retain()); - - // If we don't have a current handler, we should try to find one based on the protocol of the incoming bytes. - if (currentHandler == null) { - for (InboundBytesHandler handler : protocolBytesHandlers) { - if (handler.canHandleBytes(reference)) { - currentHandler = handler; - break; - } - } - } - - // If we have a current handler determined based on protocol, we should continue to use it for the fragmented bytes. - if (currentHandler != null) { - currentHandler.doHandleBytes(channel, reference, messageHandler); - } else { - throw new IllegalStateException("No bytes handler found for the incoming transport protocol"); - } + bytesHandler.doHandleBytes(channel, reference, messageHandler); } } diff --git a/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java b/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java index 4c972fdc14fa5..58adc2d3d68a5 100644 --- a/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java +++ b/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java @@ -52,7 +52,6 @@ import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.telemetry.tracing.channels.TraceableTcpTransportChannel; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import org.opensearch.transport.nativeprotocol.NativeOutboundHandler; import java.io.EOFException; @@ -119,7 +118,7 @@ public void messageReceived( long slowLogThresholdMs, TransportMessageListener messageListener ) throws IOException { - NativeInboundMessage inboundMessage = (NativeInboundMessage) message; + InboundMessage inboundMessage = (InboundMessage) message; TransportLogger.logInboundMessage(channel, inboundMessage); if (inboundMessage.isPing()) { keepAlive.receiveKeepAlive(channel); @@ -130,7 +129,7 @@ public void messageReceived( private void handleMessage( TcpChannel channel, - NativeInboundMessage message, + InboundMessage message, long startTime, long slowLogThresholdMs, TransportMessageListener messageListener @@ -202,7 +201,7 @@ private Map> extractHeaders(Map heade private void handleRequest( TcpChannel channel, Header header, - NativeInboundMessage message, + InboundMessage message, TransportMessageListener messageListener ) throws IOException { final String action = header.getActionName(); diff --git a/server/src/main/java/org/opensearch/transport/TcpTransport.java b/server/src/main/java/org/opensearch/transport/TcpTransport.java index ffa3168da0b3e..f56cd146ce953 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransport.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransport.java @@ -777,13 +777,21 @@ protected void serverAcceptedChannel(TcpChannel channel) { */ protected abstract void stopInternal(); + /** + * @deprecated Use {{@link #inboundMessage(TcpChannel, InboundMessage)}} instead + */ + @Deprecated + public void inboundMessage(TcpChannel channel, ProtocolInboundMessage message) { + inboundMessage(channel, (InboundMessage) message); + } + /** * Handles inbound message that has been decoded. * * @param channel the channel the message is from * @param message the message */ - public void inboundMessage(TcpChannel channel, ProtocolInboundMessage message) { + public void inboundMessage(TcpChannel channel, InboundMessage message) { try { inboundHandler.inboundMessage(channel, message); } catch (Exception e) { diff --git a/server/src/main/java/org/opensearch/transport/TransportLogger.java b/server/src/main/java/org/opensearch/transport/TransportLogger.java index e780f643aafd7..997b3bb5ba18e 100644 --- a/server/src/main/java/org/opensearch/transport/TransportLogger.java +++ b/server/src/main/java/org/opensearch/transport/TransportLogger.java @@ -40,7 +40,6 @@ import org.opensearch.core.common.io.stream.InputStreamStreamInput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.compress.CompressorRegistry; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import java.io.IOException; @@ -65,7 +64,7 @@ static void logInboundMessage(TcpChannel channel, BytesReference message) { } } - static void logInboundMessage(TcpChannel channel, NativeInboundMessage message) { + static void logInboundMessage(TcpChannel channel, InboundMessage message) { if (logger.isTraceEnabled()) { try { String logMessage = format(channel, message, "READ"); @@ -137,7 +136,7 @@ private static String format(TcpChannel channel, BytesReference message, String return sb.toString(); } - private static String format(TcpChannel channel, NativeInboundMessage message, String event) throws IOException { + private static String format(TcpChannel channel, InboundMessage message, String event) throws IOException { final StringBuilder sb = new StringBuilder(); sb.append(channel); diff --git a/server/src/main/java/org/opensearch/transport/TransportProtocol.java b/server/src/main/java/org/opensearch/transport/TransportProtocol.java new file mode 100644 index 0000000000000..4a11520d38d56 --- /dev/null +++ b/server/src/main/java/org/opensearch/transport/TransportProtocol.java @@ -0,0 +1,29 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport; + +/** + * Enumeration of transport protocols. + */ +enum TransportProtocol { + /** + * The original, hand-rolled binary protocol used for node-to-node + * communication. Message schemas are defined implicitly in code using the + * StreamInput and StreamOutput classes to parse and generate binary data. + */ + NATIVE; + + public static TransportProtocol fromBytes(byte b1, byte b2) { + if (b1 == 'E' && b2 == 'S') { + return NATIVE; + } + + throw new IllegalArgumentException("Unknown transport protocol: [" + b1 + ", " + b2 + "]"); + } +} diff --git a/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundBytesHandler.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundBytesHandler.java deleted file mode 100644 index 97981aeb6736e..0000000000000 --- a/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundBytesHandler.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.transport.nativeprotocol; - -import org.opensearch.common.bytes.ReleasableBytesReference; -import org.opensearch.common.lease.Releasable; -import org.opensearch.common.lease.Releasables; -import org.opensearch.core.common.bytes.CompositeBytesReference; -import org.opensearch.transport.Header; -import org.opensearch.transport.InboundAggregator; -import org.opensearch.transport.InboundBytesHandler; -import org.opensearch.transport.InboundDecoder; -import org.opensearch.transport.ProtocolInboundMessage; -import org.opensearch.transport.StatsTracker; -import org.opensearch.transport.TcpChannel; - -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.function.BiConsumer; - -/** - * Handler for inbound bytes for the native protocol. - */ -public class NativeInboundBytesHandler implements InboundBytesHandler { - - private static final ThreadLocal> fragmentList = ThreadLocal.withInitial(ArrayList::new); - private static final NativeInboundMessage PING_MESSAGE = new NativeInboundMessage(null, true); - - private final ArrayDeque pending; - private final InboundDecoder decoder; - private final InboundAggregator aggregator; - private final StatsTracker statsTracker; - private boolean isClosed = false; - - public NativeInboundBytesHandler( - ArrayDeque pending, - InboundDecoder decoder, - InboundAggregator aggregator, - StatsTracker statsTracker - ) { - this.pending = pending; - this.decoder = decoder; - this.aggregator = aggregator; - this.statsTracker = statsTracker; - } - - @Override - public void close() { - isClosed = true; - } - - @Override - public boolean canHandleBytes(ReleasableBytesReference reference) { - return true; - } - - @Override - public void doHandleBytes( - TcpChannel channel, - ReleasableBytesReference reference, - BiConsumer messageHandler - ) throws IOException { - final ArrayList fragments = fragmentList.get(); - boolean continueHandling = true; - - while (continueHandling && isClosed == false) { - boolean continueDecoding = true; - while (continueDecoding && pending.isEmpty() == false) { - try (ReleasableBytesReference toDecode = getPendingBytes()) { - final int bytesDecoded = decoder.decode(toDecode, fragments::add); - if (bytesDecoded != 0) { - releasePendingBytes(bytesDecoded); - if (fragments.isEmpty() == false && endOfMessage(fragments.get(fragments.size() - 1))) { - continueDecoding = false; - } - } else { - continueDecoding = false; - } - } - } - - if (fragments.isEmpty()) { - continueHandling = false; - } else { - try { - forwardFragments(channel, fragments, messageHandler); - } finally { - for (Object fragment : fragments) { - if (fragment instanceof ReleasableBytesReference) { - ((ReleasableBytesReference) fragment).close(); - } - } - fragments.clear(); - } - } - } - } - - private ReleasableBytesReference getPendingBytes() { - if (pending.size() == 1) { - return pending.peekFirst().retain(); - } else { - final ReleasableBytesReference[] bytesReferences = new ReleasableBytesReference[pending.size()]; - int index = 0; - for (ReleasableBytesReference pendingReference : pending) { - bytesReferences[index] = pendingReference.retain(); - ++index; - } - final Releasable releasable = () -> Releasables.closeWhileHandlingException(bytesReferences); - return new ReleasableBytesReference(CompositeBytesReference.of(bytesReferences), releasable); - } - } - - private void releasePendingBytes(int bytesConsumed) { - int bytesToRelease = bytesConsumed; - while (bytesToRelease != 0) { - try (ReleasableBytesReference reference = pending.pollFirst()) { - assert reference != null; - if (bytesToRelease < reference.length()) { - pending.addFirst(reference.retainedSlice(bytesToRelease, reference.length() - bytesToRelease)); - bytesToRelease -= bytesToRelease; - } else { - bytesToRelease -= reference.length(); - } - } - } - } - - private boolean endOfMessage(Object fragment) { - return fragment == InboundDecoder.PING || fragment == InboundDecoder.END_CONTENT || fragment instanceof Exception; - } - - private void forwardFragments( - TcpChannel channel, - ArrayList fragments, - BiConsumer messageHandler - ) throws IOException { - for (Object fragment : fragments) { - if (fragment instanceof Header) { - assert aggregator.isAggregating() == false; - aggregator.headerReceived((Header) fragment); - } else if (fragment == InboundDecoder.PING) { - assert aggregator.isAggregating() == false; - messageHandler.accept(channel, PING_MESSAGE); - } else if (fragment == InboundDecoder.END_CONTENT) { - assert aggregator.isAggregating(); - try (NativeInboundMessage aggregated = aggregator.finishAggregation()) { - statsTracker.markMessageReceived(); - messageHandler.accept(channel, aggregated); - } - } else { - assert aggregator.isAggregating(); - assert fragment instanceof ReleasableBytesReference; - aggregator.aggregate((ReleasableBytesReference) fragment); - } - } - } - -} diff --git a/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundMessage.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundMessage.java index 1143f129b6319..47dcb87e5a386 100644 --- a/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundMessage.java +++ b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeInboundMessage.java @@ -32,118 +32,34 @@ package org.opensearch.transport.nativeprotocol; -import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.annotation.DeprecatedApi; import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.lease.Releasable; -import org.opensearch.common.lease.Releasables; -import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.transport.Header; -import org.opensearch.transport.ProtocolInboundMessage; - -import java.io.IOException; +import org.opensearch.transport.InboundMessage; /** * Inbound data as a message * - * @opensearch.api + * This class is deprecated in favor of {@link InboundMessage}. */ -@PublicApi(since = "2.14.0") -public class NativeInboundMessage implements Releasable, ProtocolInboundMessage { +@DeprecatedApi(since = "2.17.0") +public class NativeInboundMessage extends InboundMessage { /** * The protocol used to encode this message */ public static String NATIVE_PROTOCOL = "native"; - private final Header header; - private final ReleasableBytesReference content; - private final Exception exception; - private final boolean isPing; - private Releasable breakerRelease; - private StreamInput streamInput; - public NativeInboundMessage(Header header, ReleasableBytesReference content, Releasable breakerRelease) { - this.header = header; - this.content = content; - this.breakerRelease = breakerRelease; - this.exception = null; - this.isPing = false; + super(header, content, breakerRelease); } public NativeInboundMessage(Header header, Exception exception) { - this.header = header; - this.content = null; - this.breakerRelease = null; - this.exception = exception; - this.isPing = false; + super(header, exception); } public NativeInboundMessage(Header header, boolean isPing) { - this.header = header; - this.content = null; - this.breakerRelease = null; - this.exception = null; - this.isPing = isPing; - } - - @Override - public String getProtocol() { - return NATIVE_PROTOCOL; - } - - public Header getHeader() { - return header; - } - - public int getContentLength() { - if (content == null) { - return 0; - } else { - return content.length(); - } - } - - public Exception getException() { - return exception; - } - - public boolean isPing() { - return isPing; + super(header, isPing); } - - public boolean isShortCircuit() { - return exception != null; - } - - public Releasable takeBreakerReleaseControl() { - final Releasable toReturn = breakerRelease; - breakerRelease = null; - if (toReturn != null) { - return toReturn; - } else { - return () -> {}; - } - } - - public StreamInput openOrGetStreamInput() throws IOException { - assert isPing == false && content != null; - if (streamInput == null) { - streamInput = content.streamInput(); - streamInput.setVersion(header.getVersion()); - } - return streamInput; - } - - @Override - public void close() { - IOUtils.closeWhileHandlingException(streamInput); - Releasables.closeWhileHandlingException(content, breakerRelease); - } - - @Override - public String toString() { - return "InboundMessage{" + header + "}"; - } - } diff --git a/server/src/test/java/org/opensearch/transport/InboundAggregatorTests.java b/server/src/test/java/org/opensearch/transport/InboundAggregatorTests.java index 4ac78366360d7..6168fd1c6a307 100644 --- a/server/src/test/java/org/opensearch/transport/InboundAggregatorTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundAggregatorTests.java @@ -42,7 +42,6 @@ import org.opensearch.core.common.breaker.CircuitBreakingException; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import org.junit.Before; import java.io.IOException; @@ -79,7 +78,7 @@ public void setUp() throws Exception { public void testInboundAggregation() throws IOException { long requestId = randomNonNegativeLong(); - Header header = new Header(randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header header = new Header(TransportProtocol.NATIVE, randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); header.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); header.actionName = "action_name"; // Initiate Message @@ -108,7 +107,7 @@ public void testInboundAggregation() throws IOException { } // Signal EOS - NativeInboundMessage aggregated = aggregator.finishAggregation(); + InboundMessage aggregated = aggregator.finishAggregation(); assertThat(aggregated, notNullValue()); assertFalse(aggregated.isPing()); @@ -126,7 +125,7 @@ public void testInboundAggregation() throws IOException { public void testInboundUnknownAction() throws IOException { long requestId = randomNonNegativeLong(); - Header header = new Header(randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header header = new Header(TransportProtocol.NATIVE, randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); header.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); header.actionName = unknownAction; // Initiate Message @@ -139,7 +138,7 @@ public void testInboundUnknownAction() throws IOException { assertEquals(0, content.refCount()); // Signal EOS - NativeInboundMessage aggregated = aggregator.finishAggregation(); + InboundMessage aggregated = aggregator.finishAggregation(); assertThat(aggregated, notNullValue()); assertTrue(aggregated.isShortCircuit()); @@ -150,7 +149,13 @@ public void testInboundUnknownAction() throws IOException { public void testCircuitBreak() throws IOException { circuitBreaker.startBreaking(); // Actions are breakable - Header breakableHeader = new Header(randomInt(), randomNonNegativeLong(), TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header breakableHeader = new Header( + TransportProtocol.NATIVE, + randomInt(), + randomNonNegativeLong(), + TransportStatus.setRequest((byte) 0), + Version.CURRENT + ); breakableHeader.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); breakableHeader.actionName = "action_name"; // Initiate Message @@ -162,7 +167,7 @@ public void testCircuitBreak() throws IOException { content1.close(); // Signal EOS - NativeInboundMessage aggregated1 = aggregator.finishAggregation(); + InboundMessage aggregated1 = aggregator.finishAggregation(); assertEquals(0, content1.refCount()); assertThat(aggregated1, notNullValue()); @@ -170,7 +175,13 @@ public void testCircuitBreak() throws IOException { assertThat(aggregated1.getException(), instanceOf(CircuitBreakingException.class)); // Actions marked as unbreakable are not broken - Header unbreakableHeader = new Header(randomInt(), randomNonNegativeLong(), TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header unbreakableHeader = new Header( + TransportProtocol.NATIVE, + randomInt(), + randomNonNegativeLong(), + TransportStatus.setRequest((byte) 0), + Version.CURRENT + ); unbreakableHeader.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); unbreakableHeader.actionName = unBreakableAction; // Initiate Message @@ -181,7 +192,7 @@ public void testCircuitBreak() throws IOException { content2.close(); // Signal EOS - NativeInboundMessage aggregated2 = aggregator.finishAggregation(); + InboundMessage aggregated2 = aggregator.finishAggregation(); assertEquals(1, content2.refCount()); assertThat(aggregated2, notNullValue()); @@ -189,7 +200,13 @@ public void testCircuitBreak() throws IOException { // Handshakes are not broken final byte handshakeStatus = TransportStatus.setHandshake(TransportStatus.setRequest((byte) 0)); - Header handshakeHeader = new Header(randomInt(), randomNonNegativeLong(), handshakeStatus, Version.CURRENT); + Header handshakeHeader = new Header( + TransportProtocol.NATIVE, + randomInt(), + randomNonNegativeLong(), + handshakeStatus, + Version.CURRENT + ); handshakeHeader.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); handshakeHeader.actionName = "handshake"; // Initiate Message @@ -200,7 +217,7 @@ public void testCircuitBreak() throws IOException { content3.close(); // Signal EOS - NativeInboundMessage aggregated3 = aggregator.finishAggregation(); + InboundMessage aggregated3 = aggregator.finishAggregation(); assertEquals(1, content3.refCount()); assertThat(aggregated3, notNullValue()); @@ -209,7 +226,7 @@ public void testCircuitBreak() throws IOException { public void testCloseWillCloseContent() { long requestId = randomNonNegativeLong(); - Header header = new Header(randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header header = new Header(TransportProtocol.NATIVE, randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); header.headers = new Tuple<>(Collections.emptyMap(), Collections.emptyMap()); header.actionName = "action_name"; // Initiate Message @@ -249,7 +266,7 @@ public void testFinishAggregationWillFinishHeader() throws IOException { } else { actionName = "action_name"; } - Header header = new Header(randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); + Header header = new Header(TransportProtocol.NATIVE, randomInt(), requestId, TransportStatus.setRequest((byte) 0), Version.CURRENT); // Initiate Message aggregator.headerReceived(header); @@ -264,7 +281,7 @@ public void testFinishAggregationWillFinishHeader() throws IOException { content.close(); // Signal EOS - NativeInboundMessage aggregated = aggregator.finishAggregation(); + InboundMessage aggregated = aggregator.finishAggregation(); assertThat(aggregated, notNullValue()); assertFalse(header.needsToReadVariableHeader()); diff --git a/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java index 2553e7740990b..7779db9dacc3c 100644 --- a/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java @@ -57,7 +57,6 @@ import org.opensearch.test.VersionUtils; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import org.junit.After; import org.junit.Before; @@ -152,7 +151,7 @@ public void testPing() throws Exception { ); requestHandlers.registerHandler(registry); - handler.inboundMessage(channel, new NativeInboundMessage(null, true)); + handler.inboundMessage(channel, InboundMessage.PING); if (channel.isServerChannel()) { BytesReference ping = channel.getMessageCaptor().get(); assertEquals('E', ping.get(0)); @@ -215,12 +214,14 @@ public TestResponse read(StreamInput in) throws IOException { false ); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); - Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); - NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(requestContent), - () -> {} + Header requestHeader = new Header( + TransportProtocol.NATIVE, + fullRequestBytes.length() - 6, + requestId, + TransportStatus.setRequest((byte) 0), + version ); + InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(requestContent), () -> {}); requestHeader.finishParsingHeader(requestMessage.openOrGetStreamInput()); handler.inboundMessage(channel, requestMessage); @@ -240,12 +241,8 @@ public TestResponse read(StreamInput in) throws IOException { BytesReference fullResponseBytes = channel.getMessageCaptor().get(); BytesReference responseContent = fullResponseBytes.slice(headerSize, fullResponseBytes.length() - headerSize); - Header responseHeader = new Header(fullResponseBytes.length() - 6, requestId, responseStatus, version); - NativeInboundMessage responseMessage = new NativeInboundMessage( - responseHeader, - ReleasableBytesReference.wrap(responseContent), - () -> {} - ); + Header responseHeader = new Header(TransportProtocol.NATIVE, fullResponseBytes.length() - 6, requestId, responseStatus, version); + InboundMessage responseMessage = new InboundMessage(responseHeader, ReleasableBytesReference.wrap(responseContent), () -> {}); responseHeader.finishParsingHeader(responseMessage.openOrGetStreamInput()); handler.inboundMessage(channel, responseMessage); @@ -267,12 +264,13 @@ public void testSendsErrorResponseToHandshakeFromCompatibleVersion() throws Exce final Version remoteVersion = VersionUtils.randomCompatibleVersion(random(), version); final long requestId = randomNonNegativeLong(); final Header requestHeader = new Header( + TransportProtocol.NATIVE, between(0, 100), requestId, TransportStatus.setRequest(TransportStatus.setHandshake((byte) 0)), remoteVersion ); - final NativeInboundMessage requestMessage = unreadableInboundHandshake(remoteVersion, requestHeader); + final InboundMessage requestMessage = unreadableInboundHandshake(remoteVersion, requestHeader); requestHeader.actionName = TransportHandshaker.HANDSHAKE_ACTION_NAME; requestHeader.headers = Tuple.tuple(Map.of(), Map.of()); requestHeader.features = Set.of(); @@ -307,12 +305,13 @@ public void testClosesChannelOnErrorInHandshakeWithIncompatibleVersion() throws final Version remoteVersion = Version.fromId(randomIntBetween(0, version.minimumCompatibilityVersion().id - 1)); final long requestId = randomNonNegativeLong(); final Header requestHeader = new Header( + TransportProtocol.NATIVE, between(0, 100), requestId, TransportStatus.setRequest(TransportStatus.setHandshake((byte) 0)), remoteVersion ); - final NativeInboundMessage requestMessage = unreadableInboundHandshake(remoteVersion, requestHeader); + final InboundMessage requestMessage = unreadableInboundHandshake(remoteVersion, requestHeader); requestHeader.actionName = TransportHandshaker.HANDSHAKE_ACTION_NAME; requestHeader.headers = Tuple.tuple(Map.of(), Map.of()); requestHeader.features = Set.of(); @@ -338,22 +337,19 @@ public void testLogsSlowInboundProcessing() throws Exception { final Version remoteVersion = Version.CURRENT; final long requestId = randomNonNegativeLong(); final Header requestHeader = new Header( + TransportProtocol.NATIVE, between(0, 100), requestId, TransportStatus.setRequest(TransportStatus.setHandshake((byte) 0)), remoteVersion ); - final NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(BytesArray.EMPTY), - () -> { - try { - TimeUnit.SECONDS.sleep(1L); - } catch (InterruptedException e) { - throw new AssertionError(e); - } + final InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(BytesArray.EMPTY), () -> { + try { + TimeUnit.SECONDS.sleep(1L); + } catch (InterruptedException e) { + throw new AssertionError(e); } - ); + }); requestHeader.actionName = TransportHandshaker.HANDSHAKE_ACTION_NAME; requestHeader.headers = Tuple.tuple(Collections.emptyMap(), Collections.emptyMap()); requestHeader.features = Set.of(); @@ -424,12 +420,14 @@ public void onResponseSent(long requestId, String action, Exception error) { BytesReference fullRequestBytes = BytesReference.fromByteBuffer((ByteBuffer) buffer.flip()); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); - Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); - NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(requestContent), - () -> {} + Header requestHeader = new Header( + TransportProtocol.NATIVE, + fullRequestBytes.length() - 6, + requestId, + TransportStatus.setRequest((byte) 0), + version ); + InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(requestContent), () -> {}); requestHeader.finishParsingHeader(requestMessage.openOrGetStreamInput()); handler.inboundMessage(channel, requestMessage); @@ -493,12 +491,14 @@ public void onResponseSent(long requestId, String action, Exception error) { ); // Create the request payload by intentionally stripping 1 byte away BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize - 1); - Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); - NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(requestContent), - () -> {} + Header requestHeader = new Header( + TransportProtocol.NATIVE, + fullRequestBytes.length() - 6, + requestId, + TransportStatus.setRequest((byte) 0), + version ); + InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(requestContent), () -> {}); requestHeader.finishParsingHeader(requestMessage.openOrGetStreamInput()); handler.inboundMessage(channel, requestMessage); @@ -561,12 +561,14 @@ public TestResponse read(StreamInput in) throws IOException { false ); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); - Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); - NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(requestContent), - () -> {} + Header requestHeader = new Header( + TransportProtocol.NATIVE, + fullRequestBytes.length() - 6, + requestId, + TransportStatus.setRequest((byte) 0), + version ); + InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(requestContent), () -> {}); requestHeader.finishParsingHeader(requestMessage.openOrGetStreamInput()); handler.inboundMessage(channel, requestMessage); @@ -587,12 +589,8 @@ public TestResponse read(StreamInput in) throws IOException { BytesReference fullResponseBytes = BytesReference.fromByteBuffer((ByteBuffer) buffer.flip()); BytesReference responseContent = fullResponseBytes.slice(headerSize, fullResponseBytes.length() - headerSize); - Header responseHeader = new Header(fullResponseBytes.length() - 6, requestId, responseStatus, version); - NativeInboundMessage responseMessage = new NativeInboundMessage( - responseHeader, - ReleasableBytesReference.wrap(responseContent), - () -> {} - ); + Header responseHeader = new Header(TransportProtocol.NATIVE, fullResponseBytes.length() - 6, requestId, responseStatus, version); + InboundMessage responseMessage = new InboundMessage(responseHeader, ReleasableBytesReference.wrap(responseContent), () -> {}); responseHeader.finishParsingHeader(responseMessage.openOrGetStreamInput()); handler.inboundMessage(channel, responseMessage); @@ -655,12 +653,14 @@ public TestResponse read(StreamInput in) throws IOException { false ); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); - Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); - NativeInboundMessage requestMessage = new NativeInboundMessage( - requestHeader, - ReleasableBytesReference.wrap(requestContent), - () -> {} + Header requestHeader = new Header( + TransportProtocol.NATIVE, + fullRequestBytes.length() - 6, + requestId, + TransportStatus.setRequest((byte) 0), + version ); + InboundMessage requestMessage = new InboundMessage(requestHeader, ReleasableBytesReference.wrap(requestContent), () -> {}); requestHeader.finishParsingHeader(requestMessage.openOrGetStreamInput()); handler.inboundMessage(channel, requestMessage); @@ -676,12 +676,8 @@ public TestResponse read(StreamInput in) throws IOException { BytesReference fullResponseBytes = channel.getMessageCaptor().get(); // Create the response payload by intentionally stripping 1 byte away BytesReference responseContent = fullResponseBytes.slice(headerSize, fullResponseBytes.length() - headerSize - 1); - Header responseHeader = new Header(fullResponseBytes.length() - 6, requestId, responseStatus, version); - NativeInboundMessage responseMessage = new NativeInboundMessage( - responseHeader, - ReleasableBytesReference.wrap(responseContent), - () -> {} - ); + Header responseHeader = new Header(TransportProtocol.NATIVE, fullResponseBytes.length() - 6, requestId, responseStatus, version); + InboundMessage responseMessage = new InboundMessage(responseHeader, ReleasableBytesReference.wrap(responseContent), () -> {}); responseHeader.finishParsingHeader(responseMessage.openOrGetStreamInput()); handler.inboundMessage(channel, responseMessage); @@ -690,8 +686,8 @@ public TestResponse read(StreamInput in) throws IOException { assertThat(exceptionCaptor.get().getMessage(), containsString("Failed to deserialize response from handler")); } - private static NativeInboundMessage unreadableInboundHandshake(Version remoteVersion, Header requestHeader) { - return new NativeInboundMessage(requestHeader, ReleasableBytesReference.wrap(BytesArray.EMPTY), () -> {}) { + private static InboundMessage unreadableInboundHandshake(Version remoteVersion, Header requestHeader) { + return new InboundMessage(requestHeader, ReleasableBytesReference.wrap(BytesArray.EMPTY), () -> {}) { @Override public StreamInput openOrGetStreamInput() { final StreamInput streamInput = new InputStreamStreamInput(new InputStream() { diff --git a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java index 5a89bf1e0ead3..cd6c4cf260176 100644 --- a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java @@ -49,7 +49,6 @@ import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import java.io.IOException; import java.util.ArrayList; @@ -82,9 +81,8 @@ public void testPipelineHandlingForNativeProtocol() throws IOException { final List> expected = new ArrayList<>(); final List> actual = new ArrayList<>(); final List toRelease = new ArrayList<>(); - final BiConsumer messageHandler = (c, m) -> { + final BiConsumer messageHandler = (c, message) -> { try { - NativeInboundMessage message = (NativeInboundMessage) m; final Header header = message.getHeader(); final MessageData actualData; final Version version = header.getVersion(); @@ -199,7 +197,7 @@ public void testPipelineHandlingForNativeProtocol() throws IOException { } public void testDecodeExceptionIsPropagated() throws IOException { - BiConsumer messageHandler = (c, m) -> {}; + BiConsumer messageHandler = (c, m) -> {}; final StatsTracker statsTracker = new StatsTracker(); final LongSupplier millisSupplier = () -> TimeValue.nsecToMSec(System.nanoTime()); final InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); @@ -229,7 +227,7 @@ public void testDecodeExceptionIsPropagated() throws IOException { } public void testEnsureBodyIsNotPrematurelyReleased() throws IOException { - BiConsumer messageHandler = (c, m) -> {}; + BiConsumer messageHandler = (c, m) -> {}; final StatsTracker statsTracker = new StatsTracker(); final LongSupplier millisSupplier = () -> TimeValue.nsecToMSec(System.nanoTime()); final InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); diff --git a/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java index 01f19bea7a37f..11ca683c306bf 100644 --- a/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java +++ b/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java @@ -52,7 +52,6 @@ import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import org.opensearch.transport.nativeprotocol.NativeOutboundHandler; import org.junit.After; import org.junit.Before; @@ -106,9 +105,8 @@ public void setUp() throws Exception { final InboundAggregator aggregator = new InboundAggregator(breaker, (Predicate) action -> true); pipeline = new InboundPipeline(statsTracker, millisSupplier, decoder, aggregator, (c, m) -> { try (BytesStreamOutput streamOutput = new BytesStreamOutput()) { - NativeInboundMessage m1 = (NativeInboundMessage) m; - Streams.copy(m1.openOrGetStreamInput(), streamOutput); - message.set(new Tuple<>(m1.getHeader(), streamOutput.bytes())); + Streams.copy(m.openOrGetStreamInput(), streamOutput); + message.set(new Tuple<>(m.getHeader(), streamOutput.bytes())); } catch (IOException e) { throw new AssertionError(e); } diff --git a/server/src/test/java/org/opensearch/transport/TransportProtocolTests.java b/server/src/test/java/org/opensearch/transport/TransportProtocolTests.java new file mode 100644 index 0000000000000..024d3281fb76e --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/TransportProtocolTests.java @@ -0,0 +1,22 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport; + +import org.opensearch.test.OpenSearchTestCase; + +public class TransportProtocolTests extends OpenSearchTestCase { + + public void testNativeProtocol() { + assertEquals(TransportProtocol.NATIVE, TransportProtocol.fromBytes((byte) 'E', (byte) 'S')); + } + + public void testInvalidProtocol() { + assertThrows(IllegalArgumentException.class, () -> TransportProtocol.fromBytes((byte) 'e', (byte) 'S')); + } +} From 8629279533ff1a2f2c7bfcd772382d8b50d5cfba Mon Sep 17 00:00:00 2001 From: Sarthak Aggarwal Date: Thu, 29 Aug 2024 10:17:43 +0530 Subject: [PATCH 03/10] Star Tree Meta and Data Writers (#15295) --------- Signed-off-by: Sarthak Aggarwal --- .../LuceneDocValuesConsumerFactory.java | 20 +- .../LuceneDocValuesProducerFactory.java | 19 +- .../CompositeIndexConstants.java | 26 ++ .../CompositeIndexMetadata.java | 53 +++ .../compositeindex/datacube/MetricStat.java | 37 +- .../datacube/ReadDimension.java | 55 +++ .../startree/StarTreeFieldConfiguration.java | 22 +- .../startree/builder/BaseStarTreeBuilder.java | 45 +-- .../startree/builder/StarTreeBuilder.java | 2 +- .../startree/fileformats/StarTreeWriter.java | 81 ++++ .../fileformats/data/StarTreeDataWriter.java | 110 ++++++ .../fileformats/data/package-info.java | 14 + .../fileformats/meta/StarTreeMetaWriter.java | 161 ++++++++ .../fileformats/meta/StarTreeMetadata.java | 363 ++++++++++++++++++ .../fileformats/meta/package-info.java | 14 + .../node/FixedLengthStarTreeNode.java | 302 +++++++++++++++ .../fileformats/node/package-info.java | 12 + .../startree/fileformats/package-info.java | 14 + .../InMemoryTreeNode.java} | 19 +- .../startree/node/StarTreeFactory.java | 42 ++ .../datacube/startree/node/StarTreeNode.java | 29 +- .../startree/node/StarTreeNodeType.java | 103 +++++ .../startree/utils/StarTreeUtils.java | 111 ++++++ .../builder/AbstractStarTreeBuilderTests.java | 45 ++- .../data/StarTreeFileFormatsTests.java | 209 ++++++++++ .../meta/StarTreeMetadataTests.java | 224 +++++++++++ .../node/FixedLengthStarTreeNodeTests.java | 233 +++++++++++ .../startree/node/StarTreeNodeTypeTests.java | 58 +++ .../startree/utils/StarTreeUtilsTests.java | 78 ++++ .../index/mapper/StarTreeMapperTests.java | 9 + 30 files changed, 2418 insertions(+), 92 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/ReadDimension.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/package-info.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/package-info.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/package-info.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/package-info.java rename server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/{utils/TreeNode.java => node/InMemoryTreeNode.java} (76%) create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeFactory.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java diff --git a/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesConsumerFactory.java b/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesConsumerFactory.java index 1ed672870337e..4b3f62b6171da 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesConsumerFactory.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesConsumerFactory.java @@ -34,17 +34,15 @@ public static DocValuesConsumer getDocValuesConsumerForCompositeCodec( String metaCodec, String metaExtension ) throws IOException { - try ( - Lucene90DocValuesConsumerWrapper lucene90DocValuesConsumerWrapper = new Lucene90DocValuesConsumerWrapper( - state, - dataCodec, - dataExtension, - metaCodec, - metaExtension - ) - ) { - return lucene90DocValuesConsumerWrapper.getLucene90DocValuesConsumer(); - } + Lucene90DocValuesConsumerWrapper lucene90DocValuesConsumerWrapper = new Lucene90DocValuesConsumerWrapper( + state, + dataCodec, + dataExtension, + metaCodec, + metaExtension + ); + return lucene90DocValuesConsumerWrapper.getLucene90DocValuesConsumer(); + } } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesProducerFactory.java b/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesProducerFactory.java index 611a97ffeb834..d85205d239648 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesProducerFactory.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesProducerFactory.java @@ -40,17 +40,14 @@ public static DocValuesProducer getDocValuesProducerForCompositeCodec( switch (compositeCodec) { case Composite99Codec.COMPOSITE_INDEX_CODEC_NAME: - try ( - Lucene90DocValuesProducerWrapper lucene90DocValuesProducerWrapper = new Lucene90DocValuesProducerWrapper( - state, - dataCodec, - dataExtension, - metaCodec, - metaExtension - ) - ) { - return lucene90DocValuesProducerWrapper.getLucene90DocValuesProducer(); - } + Lucene90DocValuesProducerWrapper lucene90DocValuesProducerWrapper = new Lucene90DocValuesProducerWrapper( + state, + dataCodec, + dataExtension, + metaCodec, + metaExtension + ); + return lucene90DocValuesProducerWrapper.getLucene90DocValuesProducer(); default: throw new IllegalStateException("Invalid composite codec " + "[" + compositeCodec + "]"); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java new file mode 100644 index 0000000000000..9402675ff39d9 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java @@ -0,0 +1,26 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex; + +/** + * This class contains constants used in the Composite Index implementation. + */ +public class CompositeIndexConstants { + + /** + * The magic marker value used for sanity checks in the Composite Index implementation. + */ + public static final long COMPOSITE_FIELD_MARKER = 0xC0950513F1E1DL; // Composite Field + + /** + * Represents the key to fetch number of non-star aggregated segment documents. + */ + public static final String SEGMENT_DOCS_COUNT = "segmentDocsCount"; + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java new file mode 100644 index 0000000000000..4972c877d4ab8 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java @@ -0,0 +1,53 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex; + +import org.opensearch.index.mapper.CompositeMappedFieldType; + +/** + * This class represents the metadata of a Composite Index, which includes information about + * the composite field name, type, and the specific metadata for the type of composite field + * (e.g., Star Tree metadata). + * + * @opensearch.experimental + */ +public class CompositeIndexMetadata { + + private final String compositeFieldName; + private final CompositeMappedFieldType.CompositeFieldType compositeFieldType; + + /** + * Constructs a CompositeIndexMetadata object with the provided composite field name and type. + * + * @param compositeFieldName the name of the composite field + * @param compositeFieldType the type of the composite field + */ + public CompositeIndexMetadata(String compositeFieldName, CompositeMappedFieldType.CompositeFieldType compositeFieldType) { + this.compositeFieldName = compositeFieldName; + this.compositeFieldType = compositeFieldType; + } + + /** + * Returns the name of the composite field. + * + * @return the composite field name + */ + public String getCompositeFieldName() { + return compositeFieldName; + } + + /** + * Returns the type of the composite field. + * + * @return the composite field type + */ + public CompositeMappedFieldType.CompositeFieldType getCompositeFieldType() { + return compositeFieldType; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/MetricStat.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/MetricStat.java index 1522078024b64..a7b4c96c372d8 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/MetricStat.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/MetricStat.java @@ -20,37 +20,43 @@ */ @ExperimentalApi public enum MetricStat { - VALUE_COUNT("value_count"), - SUM("sum"), - MIN("min"), - MAX("max"), - AVG("avg", VALUE_COUNT, SUM), - DOC_COUNT("doc_count", true); + VALUE_COUNT("value_count", 0), + SUM("sum", 1), + MIN("min", 2), + MAX("max", 3), + AVG("avg", 4, VALUE_COUNT, SUM), + DOC_COUNT("doc_count", true, 5); private final String typeName; private final MetricStat[] baseMetrics; + private final int metricOrdinal; // System field stats cannot be used as input for user metric types private final boolean isSystemFieldStat; - MetricStat(String typeName) { - this(typeName, false); + MetricStat(String typeName, int metricOrdinal) { + this(typeName, false, metricOrdinal); } - MetricStat(String typeName, MetricStat... baseMetrics) { - this(typeName, false, baseMetrics); + MetricStat(String typeName, int metricOrdinal, MetricStat... baseMetrics) { + this(typeName, false, metricOrdinal, baseMetrics); } - MetricStat(String typeName, boolean isSystemFieldStat, MetricStat... baseMetrics) { + MetricStat(String typeName, boolean isSystemFieldStat, int metricOrdinal, MetricStat... baseMetrics) { this.typeName = typeName; this.isSystemFieldStat = isSystemFieldStat; this.baseMetrics = baseMetrics; + this.metricOrdinal = metricOrdinal; } public String getTypeName() { return typeName; } + public int getMetricOrdinal() { + return metricOrdinal; + } + /** * Return the list of metrics that this metric is derived from * For example, AVG is derived from COUNT and SUM @@ -76,4 +82,13 @@ public static MetricStat fromTypeName(String typeName) { } throw new IllegalArgumentException("Invalid metric stat: " + typeName); } + + public static MetricStat fromMetricOrdinal(int metricOrdinal) { + for (MetricStat metric : MetricStat.values()) { + if (metric.getMetricOrdinal() == metricOrdinal) { + return metric; + } + } + throw new IllegalArgumentException("Invalid metric stat: " + metricOrdinal); + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/ReadDimension.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/ReadDimension.java new file mode 100644 index 0000000000000..4264ec87d2c74 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/ReadDimension.java @@ -0,0 +1,55 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube; + +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; + +import java.io.IOException; +import java.util.Objects; + +/** + * Represents a dimension for reconstructing StarTreeField from file formats during searches and merges. + * + * @opensearch.experimental + */ +public class ReadDimension implements Dimension { + public static final String READ = "read"; + private final String field; + + public ReadDimension(String field) { + this.field = field; + } + + public String getField() { + return field; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(CompositeDataCubeFieldType.NAME, field); + builder.field(CompositeDataCubeFieldType.TYPE, READ); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReadDimension dimension = (ReadDimension) o; + return Objects.equals(field, dimension.getField()); + } + + @Override + public int hashCode() { + return Objects.hash(field); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeFieldConfiguration.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeFieldConfiguration.java index 755c064c2c60a..d732a8598d711 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeFieldConfiguration.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeFieldConfiguration.java @@ -56,19 +56,25 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @ExperimentalApi public enum StarTreeBuildMode { // TODO : remove onheap support unless this proves useful - ON_HEAP("onheap"), - OFF_HEAP("offheap"); + ON_HEAP("onheap", (byte) 0), + OFF_HEAP("offheap", (byte) 1); private final String typeName; + private final byte buildModeOrdinal; - StarTreeBuildMode(String typeName) { + StarTreeBuildMode(String typeName, byte buildModeOrdinal) { this.typeName = typeName; + this.buildModeOrdinal = buildModeOrdinal; } public String getTypeName() { return typeName; } + public byte getBuildModeOrdinal() { + return buildModeOrdinal; + } + public static StarTreeBuildMode fromTypeName(String typeName) { for (StarTreeBuildMode starTreeBuildMode : StarTreeBuildMode.values()) { if (starTreeBuildMode.getTypeName().equalsIgnoreCase(typeName)) { @@ -77,6 +83,16 @@ public static StarTreeBuildMode fromTypeName(String typeName) { } throw new IllegalArgumentException(String.format(Locale.ROOT, "Invalid star tree build mode: [%s] ", typeName)); } + + public static StarTreeBuildMode fromBuildModeOrdinal(byte buildModeOrdinal) { + for (StarTreeBuildMode starTreeBuildMode : StarTreeBuildMode.values()) { + if (starTreeBuildMode.getBuildModeOrdinal() == buildModeOrdinal) { + return starTreeBuildMode; + } + } + throw new IllegalArgumentException(String.format(Locale.ROOT, "Invalid star tree build mode: [%s] ", buildModeOrdinal)); + } + } public int maxLeafDocs() { diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index ddcf02cc6291a..d3105b4ae23c7 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -26,8 +26,9 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; import org.opensearch.index.compositeindex.datacube.startree.aggregators.ValueAggregator; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; -import org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode; import org.opensearch.index.fielddata.IndexNumericFieldData; import org.opensearch.index.mapper.DocCountFieldMapper; import org.opensearch.index.mapper.Mapper; @@ -46,7 +47,7 @@ import java.util.Objects; import java.util.Set; -import static org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode.ALL; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; /** * Builder for star tree. Defines the algorithm to construct star-tree @@ -72,7 +73,7 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { protected int numStarTreeNodes; protected final int maxLeafDocuments; - protected final TreeNode rootNode = getNewNode(); + protected final InMemoryTreeNode rootNode = getNewNode(); protected final StarTreeField starTreeField; private final SegmentWriteState state; @@ -556,7 +557,7 @@ void build(Iterator starTreeDocumentIterator) throws IOExcepti int numAggregatedStarTreeDocument = numStarTreeDocs - numStarTreeDocument - numStarTreeDocumentUnderStarNode; logger.debug("Finished creating aggregated documents : {}", numAggregatedStarTreeDocument); - // TODO: When StarTree Codec is ready + // TODO: When StarTreeFactory Codec is ready // Create doc values indices in disk // Serialize and save in disk // Write star tree metadata for off heap implementation @@ -578,9 +579,9 @@ private void appendToStarTree(StarTreeDocument starTreeDocument) throws IOExcept * * @return return new star-tree node */ - private TreeNode getNewNode() { + private InMemoryTreeNode getNewNode() { numStarTreeNodes++; - return new TreeNode(); + return new InMemoryTreeNode(); } /** @@ -591,7 +592,7 @@ private TreeNode getNewNode() { * @param endDocId end document id * @throws IOException throws an exception if we are unable to construct the tree */ - private void constructStarTree(TreeNode node, int startDocId, int endDocId) throws IOException { + private void constructStarTree(InMemoryTreeNode node, int startDocId, int endDocId) throws IOException { int childDimensionId = node.dimensionId + 1; if (childDimensionId == numDimensions) { @@ -600,7 +601,7 @@ private void constructStarTree(TreeNode node, int startDocId, int endDocId) thro // Construct all non-star children nodes node.childDimensionId = childDimensionId; - Map children = constructNonStarNodes(startDocId, endDocId, childDimensionId); + Map children = constructNonStarNodes(startDocId, endDocId, childDimensionId); node.children = children; // Construct star-node if required @@ -609,7 +610,7 @@ private void constructStarTree(TreeNode node, int startDocId, int endDocId) thro } // Further split on child nodes if required - for (TreeNode child : children.values()) { + for (InMemoryTreeNode child : children.values()) { if (child.endDocId - child.startDocId > maxLeafDocuments) { constructStarTree(child, child.startDocId, child.endDocId); } @@ -625,14 +626,14 @@ private void constructStarTree(TreeNode node, int startDocId, int endDocId) thro * @return root node with non-star nodes constructed * @throws IOException throws an exception if we are unable to construct non-star nodes */ - private Map constructNonStarNodes(int startDocId, int endDocId, int dimensionId) throws IOException { - Map nodes = new HashMap<>(); + private Map constructNonStarNodes(int startDocId, int endDocId, int dimensionId) throws IOException { + Map nodes = new HashMap<>(); int nodeStartDocId = startDocId; Long nodeDimensionValue = getDimensionValue(startDocId, dimensionId); for (int i = startDocId + 1; i < endDocId; i++) { Long dimensionValue = getDimensionValue(i, dimensionId); if (Objects.equals(dimensionValue, nodeDimensionValue) == false) { - TreeNode child = getNewNode(); + InMemoryTreeNode child = getNewNode(); child.dimensionId = dimensionId; child.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; child.startDocId = nodeStartDocId; @@ -643,7 +644,7 @@ private Map constructNonStarNodes(int startDocId, int endDocId, nodeDimensionValue = dimensionValue; } } - TreeNode lastNode = getNewNode(); + InMemoryTreeNode lastNode = getNewNode(); lastNode.dimensionId = dimensionId; lastNode.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; lastNode.startDocId = nodeStartDocId; @@ -661,11 +662,11 @@ private Map constructNonStarNodes(int startDocId, int endDocId, * @return root node with star nodes constructed * @throws IOException throws an exception if we are unable to construct non-star nodes */ - private TreeNode constructStarNode(int startDocId, int endDocId, int dimensionId) throws IOException { - TreeNode starNode = getNewNode(); + private InMemoryTreeNode constructStarNode(int startDocId, int endDocId, int dimensionId) throws IOException { + InMemoryTreeNode starNode = getNewNode(); starNode.dimensionId = dimensionId; starNode.dimensionValue = ALL; - starNode.isStarNode = true; + starNode.nodeType = StarTreeNodeType.STAR.getValue(); starNode.startDocId = numStarTreeDocs; Iterator starTreeDocumentIterator = generateStarTreeDocumentsForStarNode(startDocId, endDocId, dimensionId); while (starTreeDocumentIterator.hasNext()) { @@ -682,7 +683,7 @@ private TreeNode constructStarNode(int startDocId, int endDocId, int dimensionId * @return aggregated star-tree documents * @throws IOException throws an exception upon failing to create new aggregated docs based on star tree */ - private StarTreeDocument createAggregatedDocs(TreeNode node) throws IOException { + private StarTreeDocument createAggregatedDocs(InMemoryTreeNode node) throws IOException { StarTreeDocument aggregatedStarTreeDocument = null; if (node.children == null) { @@ -709,8 +710,8 @@ private StarTreeDocument createAggregatedDocs(TreeNode node) throws IOException // For non-leaf node if (node.children.containsKey((long) ALL)) { // If it has star child, use the star child aggregated document directly - for (TreeNode child : node.children.values()) { - if (child.isStarNode) { + for (InMemoryTreeNode child : node.children.values()) { + if (child.nodeType == StarTreeNodeType.STAR.getValue()) { aggregatedStarTreeDocument = createAggregatedDocs(child); node.aggregatedDocId = child.aggregatedDocId; } else { @@ -720,12 +721,12 @@ private StarTreeDocument createAggregatedDocs(TreeNode node) throws IOException } else { // If no star child exists, aggregate all aggregated documents from non-star children if (node.children.values().size() == 1) { - for (TreeNode child : node.children.values()) { + for (InMemoryTreeNode child : node.children.values()) { aggregatedStarTreeDocument = reduceStarTreeDocuments(aggregatedStarTreeDocument, createAggregatedDocs(child)); node.aggregatedDocId = child.aggregatedDocId; } } else { - for (TreeNode child : node.children.values()) { + for (InMemoryTreeNode child : node.children.values()) { aggregatedStarTreeDocument = reduceStarTreeDocuments(aggregatedStarTreeDocument, createAggregatedDocs(child)); } if (null == aggregatedStarTreeDocument) { @@ -760,7 +761,7 @@ public void close() throws IOException { abstract Iterator mergeStarTrees(List starTreeValues) throws IOException; - public TreeNode getRootNode() { + public InMemoryTreeNode getRootNode() { return rootNode; } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java index 94c9c9f2efb18..357f48c0cc726 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java @@ -34,7 +34,7 @@ public interface StarTreeBuilder extends Closeable { void build(Map fieldProducerMap) throws IOException; /** - * Builds the star tree using StarTree values from multiple segments + * Builds the star tree using StarTreeFactory values from multiple segments * * @param starTreeValuesSubs contains the star tree values from multiple segments * @throws IOException when we are unable to build star-tree diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java new file mode 100644 index 0000000000000..7f1839024eea7 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java @@ -0,0 +1,81 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats; + +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.data.StarTreeDataWriter; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetaWriter; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; + +import java.io.IOException; +import java.util.List; + +/** + * Util class for building star tree + * + * @opensearch.experimental + */ +public class StarTreeWriter { + + /** Initial version for the star tree writer */ + public static final int VERSION_START = 0; + + /** Current version for the star tree writer */ + public static final int VERSION_CURRENT = VERSION_START; + + public StarTreeWriter() {} + + /** + * Write star tree to index output stream + * + * @param dataOut data index output + * @param rootNode root star-tree node + * @param numNodes number of nodes in the star tree + * @param name name of the star-tree field + * @return total size of the three + * @throws IOException when star-tree data serialization fails + */ + public long writeStarTree(IndexOutput dataOut, InMemoryTreeNode rootNode, int numNodes, String name) throws IOException { + return StarTreeDataWriter.writeStarTree(dataOut, rootNode, numNodes, name); + } + + /** + * Write star tree metadata to index output stream + * + * @param metaOut meta index output + * @param starTreeField star tree field + * @param metricAggregatorInfos metric aggregator infos + * @param numNodes number of nodes in the star tree + * @param segmentAggregatedCount segment aggregated count + * @param dataFilePointer data file pointer + * @param dataFileLength data file length + * @throws IOException when star-tree data serialization fails + */ + public void writeStarTreeMetadata( + IndexOutput metaOut, + StarTreeField starTreeField, + List metricAggregatorInfos, + Integer numNodes, + Integer segmentAggregatedCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + StarTreeMetaWriter.writeStarTreeMetadata( + metaOut, + starTreeField, + metricAggregatorInfos, + numNodes, + segmentAggregatedCount, + dataFilePointer, + dataFileLength + ); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java new file mode 100644 index 0000000000000..32feb78a4db3d --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java @@ -0,0 +1,110 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.data; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.node.FixedLengthStarTreeNode.SERIALIZABLE_DATA_SIZE_IN_BYTES; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; + +/** + * Utility class for serializing a star-tree data structure. + * + * @opensearch.experimental + */ +public class StarTreeDataWriter { + + private static final Logger logger = LogManager.getLogger(StarTreeDataWriter.class); + + /** + * Writes the star-tree data structure. + * + * @param indexOutput the IndexOutput to write the star-tree data + * @param rootNode the root node of the star-tree + * @param numNodes the total number of nodes in the star-tree + * @param name the name of the star-tree field + * @return the total size in bytes of the serialized star-tree data + * @throws IOException if an I/O error occurs while writing the star-tree data + */ + public static long writeStarTree(IndexOutput indexOutput, InMemoryTreeNode rootNode, int numNodes, String name) throws IOException { + long totalSizeInBytes = (long) numNodes * SERIALIZABLE_DATA_SIZE_IN_BYTES; + + logger.debug("Star tree data size in bytes : {} for star-tree field {}", totalSizeInBytes, name); + + writeStarTreeNodes(indexOutput, rootNode); + return totalSizeInBytes; + } + + /** + * Writes the star-tree nodes in a breadth-first order. + * + * @param output the IndexOutput to write the nodes + * @param rootNode the root node of the star-tree + * @throws IOException if an I/O error occurs while writing the nodes + */ + private static void writeStarTreeNodes(IndexOutput output, InMemoryTreeNode rootNode) throws IOException { + Queue queue = new LinkedList<>(); + queue.add(rootNode); + + int currentNodeId = 0; + while (!queue.isEmpty()) { + InMemoryTreeNode node = queue.remove(); + + if (node.children == null || node.children.isEmpty()) { + writeStarTreeNode(output, node, ALL, ALL); + } else { + + // Sort all children nodes based on dimension value + List sortedChildren = new ArrayList<>(node.children.values()); + sortedChildren.sort( + Comparator.comparingInt(InMemoryTreeNode::getNodeType).thenComparingLong(InMemoryTreeNode::getDimensionValue) + ); + + int firstChildId = currentNodeId + queue.size() + 1; + int lastChildId = firstChildId + sortedChildren.size() - 1; + writeStarTreeNode(output, node, firstChildId, lastChildId); + + queue.addAll(sortedChildren); + } + + currentNodeId++; + } + } + + /** + * Writes a single star-tree node + * + * @param output the IndexOutput to write the node + * @param node the star tree node to write + * @param firstChildId the ID of the first child node + * @param lastChildId the ID of the last child node + * @throws IOException if an I/O error occurs while writing the node + */ + private static void writeStarTreeNode(IndexOutput output, InMemoryTreeNode node, int firstChildId, int lastChildId) throws IOException { + output.writeInt(node.dimensionId); + output.writeLong(node.dimensionValue); + output.writeInt(node.startDocId); + output.writeInt(node.endDocId); + output.writeInt(node.aggregatedDocId); + output.writeByte(node.nodeType); + output.writeInt(firstChildId); + output.writeInt(lastChildId); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/package-info.java new file mode 100644 index 0000000000000..1c6df3886e08d --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/package-info.java @@ -0,0 +1,14 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Writer package for star tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats.data; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java new file mode 100644 index 0000000000000..2515c1efc3aed --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java @@ -0,0 +1,161 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.meta; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; +import java.util.List; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.COMPOSITE_FIELD_MARKER; +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter.VERSION_CURRENT; + +/** + * The utility class for serializing the metadata of a star-tree data structure. + * The metadata includes information about the dimensions, metrics, and other relevant details + * related to the star tree. + * + * @opensearch.experimental + */ +public class StarTreeMetaWriter { + + private static final Logger logger = LogManager.getLogger(StarTreeMetaWriter.class); + + /** + * Writes the star-tree metadata. + * + * @param metaOut the IndexOutput to write the metadata + * @param starTreeField the star-tree field + * @param metricAggregatorInfos the list of metric aggregator information + * @param segmentAggregatedCount the aggregated document count for the segment + * @param numNodes number of nodes in the star tree + * @param dataFilePointer the file pointer to the start of the star tree data + * @param dataFileLength the length of the star tree data file + * @throws IOException if an I/O error occurs while serializing the metadata + */ + public static void writeStarTreeMetadata( + IndexOutput metaOut, + StarTreeField starTreeField, + List metricAggregatorInfos, + Integer numNodes, + Integer segmentAggregatedCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + + long initialMetaFilePointer = metaOut.getFilePointer(); + + writeMetaHeader(metaOut); + + // TODO: Replace the parameters with StarTreeMetadata class object + writeMeta(metaOut, metricAggregatorInfos, starTreeField, numNodes, segmentAggregatedCount, dataFilePointer, dataFileLength); + + logger.debug( + "Star tree meta size in bytes : {} for star-tree field {}", + metaOut.getFilePointer() - initialMetaFilePointer, + starTreeField.getName() + ); + } + + /** + * Writes the star-tree metadata header. + * + * @param metaOut the IndexOutput to write the header + * @throws IOException if an I/O error occurs while writing the header + */ + private static void writeMetaHeader(IndexOutput metaOut) throws IOException { + // magic marker for sanity + metaOut.writeLong(COMPOSITE_FIELD_MARKER); + + // version + metaOut.writeVInt(VERSION_CURRENT); + } + + /** + * Writes the star-tree metadata. + * + * @param metaOut the IndexOutput to write the metadata + * @param metricAggregatorInfos the list of metric aggregator information + * @param starTreeField the star tree field + * @param numNodes number of nodes in the star tree + * @param segmentAggregatedDocCount the aggregated document count for the segment + * @param dataFilePointer the file pointer to the start of the star-tree data + * @param dataFileLength the length of the star-tree data file + * @throws IOException if an I/O error occurs while writing the metadata + */ + private static void writeMeta( + IndexOutput metaOut, + List metricAggregatorInfos, + StarTreeField starTreeField, + int numNodes, + Integer segmentAggregatedDocCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + + // star tree field name + metaOut.writeString(starTreeField.getName()); + + // star tree field type + metaOut.writeString(CompositeMappedFieldType.CompositeFieldType.STAR_TREE.getName()); + + // number of nodes + metaOut.writeInt(numNodes); + + // number of dimensions + // TODO: Revisit the number of dimensions for timestamps (as we will split timestamp into min, hour, etc.) + metaOut.writeVInt(starTreeField.getDimensionsOrder().size()); + + // dimensions + // TODO: Add sub-dimensions for timestamps (as we will split timestamp into min, hour, etc.) + for (Dimension dimension : starTreeField.getDimensionsOrder()) { + metaOut.writeString(dimension.getField()); + } + + // number of metrics + metaOut.writeVInt(metricAggregatorInfos.size()); + + // metric - metric stat pair + for (MetricAggregatorInfo metricAggregatorInfo : metricAggregatorInfos) { + metaOut.writeString(metricAggregatorInfo.getField()); + int metricStatOrdinal = metricAggregatorInfo.getMetricStat().getMetricOrdinal(); + metaOut.writeVInt(metricStatOrdinal); + } + + // segment aggregated document count + metaOut.writeVInt(segmentAggregatedDocCount); + + // max leaf docs + metaOut.writeVInt(starTreeField.getStarTreeConfig().maxLeafDocs()); + + // number of skip star node creation dimensions + metaOut.writeVInt(starTreeField.getStarTreeConfig().getSkipStarNodeCreationInDims().size()); + + // skip star node creations + for (String dimension : starTreeField.getStarTreeConfig().getSkipStarNodeCreationInDims()) { + metaOut.writeString(dimension); + } + + // star tree build-mode + metaOut.writeByte(starTreeField.getStarTreeConfig().getBuildMode().getBuildModeOrdinal()); + + // star-tree data file pointer + metaOut.writeVLong(dataFilePointer); + + // star-tree data file length + metaOut.writeVLong(dataFileLength); + + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java new file mode 100644 index 0000000000000..7519c85562a8c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java @@ -0,0 +1,363 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats.meta; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.opensearch.index.compositeindex.CompositeIndexMetadata; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Holds the associated metadata for the building of star-tree. + * + * @opensearch.experimental + */ +public class StarTreeMetadata extends CompositeIndexMetadata { + private static final Logger logger = LogManager.getLogger(StarTreeMetadata.class); + + /** + * The index input for reading metadata from the segment file. + */ + private final IndexInput meta; + + /** + * The version of the star tree stored in the segments. + */ + private final int version; + + /** + * The number of the nodes in the respective star tree + */ + private final int numberOfNodes; + + /** + * The name of the star-tree field, used to identify the star-tree. + */ + private final String starTreeFieldName; + + /** + * The type of the star-tree field, indicating the specific implementation or version. + * Here, STAR_TREE field. + */ + private final String starTreeFieldType; + + /** + * List of dimension fields used in the star-tree. + */ + private final List dimensionFields; + + /** + * List of metrics, containing field names and associated metric statistics. + */ + private final List metrics; + + /** + * The total number of documents aggregated in this star-tree segment. + */ + private final Integer segmentAggregatedDocCount; + + /** + * The maximum number of documents allowed in a leaf node. + */ + private final Integer maxLeafDocs; + + /** + * Set of dimensions for which star node creation should be skipped. + */ + private final Set skipStarNodeCreationInDims; + + /** + * The build mode used for constructing the star-tree. + */ + private final StarTreeFieldConfiguration.StarTreeBuildMode starTreeBuildMode; + + /** + * The file pointer to the start of the associated star-tree data in the (.cid) file + */ + private final long dataStartFilePointer; + + /** + * The length of the star-tree data in bytes, used for reading the correct amount of data from (.cid) file + */ + private final long dataLength; + + /** + * A star tree metadata constructor to initialize star tree metadata from the segment file (.cim) using index input. + * + * @param metaIn an index input to read star-tree meta + * @param compositeFieldName name of the composite field. Here, name of the star-tree field. + * @param compositeFieldType type of the composite field. Here, STAR_TREE field. + * @param version The version of the star tree stored in the segments. + * @throws IOException if unable to read star-tree metadata from the file + */ + public StarTreeMetadata( + IndexInput metaIn, + String compositeFieldName, + CompositeMappedFieldType.CompositeFieldType compositeFieldType, + Integer version + ) throws IOException { + super(compositeFieldName, compositeFieldType); + this.meta = metaIn; + try { + this.starTreeFieldName = this.getCompositeFieldName(); + this.starTreeFieldType = this.getCompositeFieldType().getName(); + this.version = version; + this.numberOfNodes = readNumberOfNodes(); + this.dimensionFields = readStarTreeDimensions(); + this.metrics = readMetricEntries(); + this.segmentAggregatedDocCount = readSegmentAggregatedDocCount(); + this.maxLeafDocs = readMaxLeafDocs(); + this.skipStarNodeCreationInDims = readSkipStarNodeCreationInDims(); + this.starTreeBuildMode = readBuildMode(); + this.dataStartFilePointer = readDataStartFilePointer(); + this.dataLength = readDataLength(); + } catch (Exception e) { + logger.error("Unable to read star-tree metadata from the file"); + throw new CorruptIndexException("Unable to read star-tree metadata from the file", metaIn); + } + } + + /** + * A star tree metadata constructor to initialize star tree metadata. + * Used for testing. + * + * @param meta an index input to read star-tree meta + * @param compositeFieldName name of the composite field. Here, name of the star-tree field. + * @param compositeFieldType type of the composite field. Here, STAR_TREE field. + * @param version The version of the star tree stored in the segments. + * @param dimensionFields list of dimension fields + * @param metrics list of metric entries + * @param segmentAggregatedDocCount segment aggregated doc count + * @param maxLeafDocs max leaf docs + * @param skipStarNodeCreationInDims set of dimensions to skip star node creation + * @param starTreeBuildMode star tree build mode + * @param dataStartFilePointer star file pointer to the associated star tree data in (.cid) file + * @param dataLength length of the corresponding star-tree data in (.cid) file + */ + public StarTreeMetadata( + String compositeFieldName, + CompositeMappedFieldType.CompositeFieldType compositeFieldType, + IndexInput meta, + Integer version, + Integer numberOfNodes, + List dimensionFields, + List metrics, + Integer segmentAggregatedDocCount, + Integer maxLeafDocs, + Set skipStarNodeCreationInDims, + StarTreeFieldConfiguration.StarTreeBuildMode starTreeBuildMode, + long dataStartFilePointer, + long dataLength + ) { + super(compositeFieldName, compositeFieldType); + this.meta = meta; + this.starTreeFieldName = compositeFieldName; + this.starTreeFieldType = compositeFieldType.getName(); + this.version = version; + this.numberOfNodes = numberOfNodes; + this.dimensionFields = dimensionFields; + this.metrics = metrics; + this.segmentAggregatedDocCount = segmentAggregatedDocCount; + this.maxLeafDocs = maxLeafDocs; + this.skipStarNodeCreationInDims = skipStarNodeCreationInDims; + this.starTreeBuildMode = starTreeBuildMode; + this.dataStartFilePointer = dataStartFilePointer; + this.dataLength = dataLength; + } + + private int readNumberOfNodes() throws IOException { + return meta.readInt(); + } + + private int readDimensionsCount() throws IOException { + return meta.readVInt(); + } + + private List readStarTreeDimensions() throws IOException { + int dimensionCount = readDimensionsCount(); + List dimensionFields = new ArrayList<>(); + + for (int i = 0; i < dimensionCount; i++) { + dimensionFields.add(meta.readString()); + } + + return dimensionFields; + } + + private int readMetricsCount() throws IOException { + return meta.readVInt(); + } + + private List readMetricEntries() throws IOException { + int metricCount = readMetricsCount(); + + Map starTreeMetricMap = new LinkedHashMap<>(); + for (int i = 0; i < metricCount; i++) { + String metricName = meta.readString(); + int metricStatOrdinal = meta.readVInt(); + MetricStat metricStat = MetricStat.fromMetricOrdinal(metricStatOrdinal); + Metric metric = starTreeMetricMap.computeIfAbsent(metricName, field -> new Metric(field, new ArrayList<>())); + metric.getMetrics().add(metricStat); + } + + return new ArrayList<>(starTreeMetricMap.values()); + } + + private int readSegmentAggregatedDocCount() throws IOException { + return meta.readVInt(); + } + + private int readMaxLeafDocs() throws IOException { + return meta.readVInt(); + } + + private int readSkipStarNodeCreationInDimsCount() throws IOException { + return meta.readVInt(); + } + + private Set readSkipStarNodeCreationInDims() throws IOException { + + int skipStarNodeCreationInDimsCount = readSkipStarNodeCreationInDimsCount(); + Set skipStarNodeCreationInDims = new HashSet<>(); + for (int i = 0; i < skipStarNodeCreationInDimsCount; i++) { + skipStarNodeCreationInDims.add(meta.readString()); + } + return skipStarNodeCreationInDims; + } + + private StarTreeFieldConfiguration.StarTreeBuildMode readBuildMode() throws IOException { + return StarTreeFieldConfiguration.StarTreeBuildMode.fromBuildModeOrdinal(meta.readByte()); + } + + private long readDataStartFilePointer() throws IOException { + return meta.readVLong(); + } + + private long readDataLength() throws IOException { + return meta.readVLong(); + } + + /** + * Returns the name of the star-tree field. + * + * @return star-tree field name + */ + public String getStarTreeFieldName() { + return starTreeFieldName; + } + + /** + * Returns the type of the star tree field. + * + * @return star-tree field type + */ + public String getStarTreeFieldType() { + return starTreeFieldType; + } + + /** + * Returns the list of dimension field numbers. + * + * @return star-tree dimension field numbers + */ + public List getDimensionFields() { + return dimensionFields; + } + + /** + * Returns the list of metric entries. + * + * @return star-tree metric entries + */ + public List getMetrics() { + return metrics; + } + + /** + * Returns the aggregated document count for the star-tree. + * + * @return the aggregated document count for the star-tree. + */ + public Integer getSegmentAggregatedDocCount() { + return segmentAggregatedDocCount; + } + + /** + * Returns the max leaf docs for the star-tree. + * + * @return the max leaf docs. + */ + public Integer getMaxLeafDocs() { + return maxLeafDocs; + } + + /** + * Returns the set of dimensions for which star node will not be created in the star-tree. + * + * @return the set of dimensions. + */ + public Set getSkipStarNodeCreationInDims() { + return skipStarNodeCreationInDims; + } + + /** + * Returns the build mode for the star-tree. + * + * @return the star-tree build mode. + */ + public StarTreeFieldConfiguration.StarTreeBuildMode getStarTreeBuildMode() { + return starTreeBuildMode; + } + + /** + * Returns the file pointer to the start of the star-tree data. + * + * @return start file pointer for star-tree data + */ + public long getDataStartFilePointer() { + return dataStartFilePointer; + } + + /** + * Returns the length of star-tree data + * + * @return star-tree length + */ + public long getDataLength() { + return dataLength; + } + + /** + * Returns the version with which the star tree is stored in the segments + * @return star-tree version + */ + public int getVersion() { + return version; + } + + /** + * Returns the number of nodes in the star tree + * @return number of nodes in the star tree + */ + public int getNumberOfNodes() { + return numberOfNodes; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/package-info.java new file mode 100644 index 0000000000000..a2480f03c4b5a --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/package-info.java @@ -0,0 +1,14 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Meta package for star tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats.meta; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java new file mode 100644 index 0000000000000..89ac4af51e221 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java @@ -0,0 +1,302 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats.node; + +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Iterator; + +/** + * Fixed Length implementation of {@link StarTreeNode}. + *

+ * This class represents a node in a star tree with a fixed-length serialization format. + * It provides efficient storage and retrieval of node information using a RandomAccessInput. + * The node structure includes the methods to access all the constructs of InMemoryTreeNode. + * + *

+ * Key features: + * - Fixed-size serialization for each node, allowing for efficient random access + * - Binary search capability for finding child nodes + * - Support for star nodes, null nodes and other default nodes + * - Iteration over child nodes + *

+ * The class uses specific byte offsets for each field in the serialized format, + * enabling direct access to node properties without parsing the entire node structure. + * + * @opensearch.experimental + */ +public class FixedLengthStarTreeNode implements StarTreeNode { + + /** + * Number of integer fields in the serializable data + */ + public static final int NUM_INT_SERIALIZABLE_FIELDS = 6; + + /** + * Number of long fields in the serializable data + */ + public static final int NUM_LONG_SERIALIZABLE_FIELDS = 1; + + /** + * Number of byte fields in the serializable data + */ + public static final int NUM_BYTE_SERIALIZABLE_FIELDS = 1; + + /** + * Total size in bytes of the serializable data for each node + */ + public static final long SERIALIZABLE_DATA_SIZE_IN_BYTES = (Integer.BYTES * NUM_INT_SERIALIZABLE_FIELDS) + (Long.BYTES + * NUM_LONG_SERIALIZABLE_FIELDS) + (NUM_BYTE_SERIALIZABLE_FIELDS * Byte.BYTES); + + // Byte offsets for each field in the serialized data + static final int DIMENSION_ID_OFFSET = 0; + static final int DIMENSION_VALUE_OFFSET = DIMENSION_ID_OFFSET + Integer.BYTES; + static final int START_DOC_ID_OFFSET = DIMENSION_VALUE_OFFSET + Long.BYTES; + static final int END_DOC_ID_OFFSET = START_DOC_ID_OFFSET + Integer.BYTES; + static final int AGGREGATE_DOC_ID_OFFSET = END_DOC_ID_OFFSET + Integer.BYTES; + static final int STAR_NODE_TYPE_OFFSET = AGGREGATE_DOC_ID_OFFSET + Integer.BYTES; + static final int FIRST_CHILD_ID_OFFSET = STAR_NODE_TYPE_OFFSET + Byte.BYTES; + static final int LAST_CHILD_ID_OFFSET = FIRST_CHILD_ID_OFFSET + Integer.BYTES; + + /** + * Constant representing an invalid node ID + */ + public static final int INVALID_ID = -1; + + /** + * The ID of this node + */ + private final int nodeId; + + /** + * The ID of the first child of this node + */ + private final int firstChildId; + + /** + * The input source for reading node data + */ + RandomAccessInput in; + + /** + * Constructs a FixedLengthStarTreeNode. + * + * @param in The RandomAccessInput to read node data from + * @param nodeId The ID of this node + * @throws IOException If there's an error reading from the input + */ + public FixedLengthStarTreeNode(RandomAccessInput in, int nodeId) throws IOException { + this.in = in; + this.nodeId = nodeId; + firstChildId = getInt(FIRST_CHILD_ID_OFFSET); + } + + /** + * Reads an integer value from the specified offset in the node's data. + * + * @param fieldOffset The offset of the field to read + * @return The integer value at the specified offset + * @throws IOException If there's an error reading from the input + */ + private int getInt(int fieldOffset) throws IOException { + return in.readInt(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + /** + * Reads a long value from the specified offset in the node's data. + * + * @param fieldOffset The offset of the field to read + * @return The long value at the specified offset + * @throws IOException If there's an error reading from the input + */ + private long getLong(int fieldOffset) throws IOException { + return in.readLong(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + /** + * Reads a byte value from the specified offset in the node's data. + * + * @param fieldOffset The offset of the field to read + * @return The byte value at the specified offset + * @throws IOException If there's an error reading from the input + */ + private byte getByte(int fieldOffset) throws IOException { + return in.readByte(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + @Override + public int getDimensionId() throws IOException { + return getInt(DIMENSION_ID_OFFSET); + } + + @Override + public long getDimensionValue() throws IOException { + return getLong(DIMENSION_VALUE_OFFSET); + } + + @Override + public int getChildDimensionId() throws IOException { + if (firstChildId == INVALID_ID) { + return INVALID_ID; + } else { + return in.readInt(firstChildId * SERIALIZABLE_DATA_SIZE_IN_BYTES); + } + } + + @Override + public int getStartDocId() throws IOException { + return getInt(START_DOC_ID_OFFSET); + } + + @Override + public int getEndDocId() throws IOException { + return getInt(END_DOC_ID_OFFSET); + } + + @Override + public int getAggregatedDocId() throws IOException { + return getInt(AGGREGATE_DOC_ID_OFFSET); + } + + @Override + public int getNumChildren() throws IOException { + if (firstChildId == INVALID_ID) { + return 0; + } else { + return getInt(LAST_CHILD_ID_OFFSET) - firstChildId + 1; + } + } + + @Override + public boolean isLeaf() { + return firstChildId == INVALID_ID; + } + + @Override + public byte getStarTreeNodeType() throws IOException { + return getByte(STAR_NODE_TYPE_OFFSET); + } + + @Override + public StarTreeNode getChildStarNode() throws IOException { + return handleStarNode(); + } + + @Override + public StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOException { + // there will be no children for leaf nodes + if (isLeaf()) { + return null; + } + + StarTreeNode resultStarTreeNode = null; + if (null != dimensionValue) { + resultStarTreeNode = binarySearchChild(dimensionValue); + assert null != resultStarTreeNode; + } + return resultStarTreeNode; + } + + /** + * Handles the special case of a star node. + * + * @return The star node if found, null otherwise + * @throws IOException If there's an error reading from the input + */ + private FixedLengthStarTreeNode handleStarNode() throws IOException { + FixedLengthStarTreeNode firstNode = new FixedLengthStarTreeNode(in, firstChildId); + return matchStarTreeNodeTypeOrNull(firstNode, StarTreeNodeType.STAR); + } + + /** + * Checks if the given node matches the specified StarTreeNodeType. + * + * @param firstNode The FixedLengthStarTreeNode to check. + * @param starTreeNodeType The StarTreeNodeType to match against. + * @return The firstNode if its type matches the targetType, null otherwise. + * @throws IOException If an I/O error occurs during the operation. + */ + private static FixedLengthStarTreeNode matchStarTreeNodeTypeOrNull(FixedLengthStarTreeNode firstNode, StarTreeNodeType starTreeNodeType) + throws IOException { + if (firstNode.getStarTreeNodeType() == starTreeNodeType.getValue()) { + return firstNode; + } else { + return null; + } + } + + /** + * Performs a binary search to find a child node with the given dimension value. + * + * @param dimensionValue The dimension value to search for + * @return The child node if found, null otherwise + * @throws IOException If there's an error reading from the input + */ + private FixedLengthStarTreeNode binarySearchChild(long dimensionValue) throws IOException { + + int low = firstChildId; + + // if the current node is star node, increment the low to reduce the search space + if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, firstChildId), StarTreeNodeType.STAR) != null) { + low++; + } + + // if the current node is null node, increment the low to reduce the search space + if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, low), StarTreeNodeType.NULL) != null) { + low++; + } + + int high = getInt(LAST_CHILD_ID_OFFSET); + + while (low <= high) { + int mid = low + (high - low) / 2; + FixedLengthStarTreeNode midNode = new FixedLengthStarTreeNode(in, mid); + long midDimensionValue = midNode.getDimensionValue(); + + if (midDimensionValue == dimensionValue) { + return midNode; + } else if (midDimensionValue < dimensionValue) { + low = mid + 1; + } else { + high = mid - 1; + } + } + return null; + } + + @Override + public Iterator getChildrenIterator() throws IOException { + return new Iterator<>() { + private int currentChildId = firstChildId; + private final int lastChildId = getInt(LAST_CHILD_ID_OFFSET); + + @Override + public boolean hasNext() { + return currentChildId <= lastChildId; + } + + @Override + public FixedLengthStarTreeNode next() { + try { + return new FixedLengthStarTreeNode(in, currentChildId++); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/package-info.java new file mode 100644 index 0000000000000..84271be81f5e4 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Holds classes associated with star tree node with file formats + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats.node; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/package-info.java new file mode 100644 index 0000000000000..917327757fc9b --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/package-info.java @@ -0,0 +1,14 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * File formats for star tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.fileformats; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java similarity index 76% rename from server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java rename to server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java index a5d59a2602633..20f7dcf184391 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java @@ -5,12 +5,14 @@ * this file be licensed under the Apache-2.0 license or a * compatible open source license. */ -package org.opensearch.index.compositeindex.datacube.startree.utils; +package org.opensearch.index.compositeindex.datacube.startree.node; import org.opensearch.common.annotation.ExperimentalApi; import java.util.Map; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; + /** * /** * Represents a node in a tree data structure, specifically designed for a star-tree implementation. @@ -19,9 +21,7 @@ * @opensearch.experimental */ @ExperimentalApi -public class TreeNode { - - public static final int ALL = -1; +public class InMemoryTreeNode { /** * The dimension id for the dimension (field) associated with this star-tree node. @@ -54,16 +54,21 @@ public class TreeNode { public long dimensionValue = ALL; /** - * A flag indicating whether this node is a star node (a node that represents an aggregation of all dimensions). + * A byte indicating whether the node is star node, null node or default node (with dimension value present). */ - public boolean isStarNode = false; + public byte nodeType = 0; /** * A map containing the child nodes of this star-tree node, keyed by their dimension id. */ - public Map children; + public Map children; public long getDimensionValue() { return dimensionValue; } + + public byte getNodeType() { + return nodeType; + } + } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeFactory.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeFactory.java new file mode 100644 index 0000000000000..79b5947d4f00a --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeFactory.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.node; + +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.node.FixedLengthStarTreeNode; + +import java.io.IOException; + +/** + * A factory class for creating off-heap implementations of star-tree nodes. + * + *

This class provides a static factory method to create instances of {@link StarTreeNode} + * from an {@link IndexInput} and {@link StarTreeMetadata}. The implementation uses an + * off-heap data structure to store and access the star-tree data efficiently using random access. + * + * @opensearch.experimental + */ +public class StarTreeFactory { + + /** + * Creates a new instance of {@link StarTreeNode} from the provided {@link IndexInput} and + * {@link StarTreeMetadata}. + * + * @param data The {@link IndexInput} containing the star-tree data. + * @param starTreeMetadata The {@link StarTreeMetadata} containing metadata about the star-tree. + * @return A new instance of {@link StarTreeNode} representing the root of the star-tree. + * @throws IOException If an error occurs while reading the star-tree data. + */ + public static StarTreeNode createStarTree(IndexInput data, StarTreeMetadata starTreeMetadata) throws IOException { + RandomAccessInput in = data.randomAccessSlice(0, starTreeMetadata.getDataLength()); + return new FixedLengthStarTreeNode(in, 0); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java index 59522ffa4be89..fce3e30e9ebf6 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java @@ -20,7 +20,6 @@ */ @ExperimentalApi public interface StarTreeNode { - long ALL = -1l; /** * Returns the dimension ID of the current star-tree node. @@ -86,21 +85,37 @@ public interface StarTreeNode { boolean isLeaf(); /** - * Checks if the current node is a star node. + * Determines the type of the current node in the Star Tree index structure. * - * @return true if the node is a star node, false otherwise - * @throws IOException if an I/O error occurs while reading the star node status + *

The node type can be one of the following: + *

    + *
  • Star Node: Represented by the value -2. + *
  • Null Node: Represented by the value -1. + *
  • Default Node: Represented by the value 0. + *
+ * @see StarTreeNodeType + * + * @return The type of the current node, represented by the corresponding integer value (-2, -1, or 0). + * @throws IOException if an I/O error occurs while reading the node type */ - boolean isStarNode() throws IOException; + byte getStarTreeNodeType() throws IOException; /** - * Returns the child star-tree node for the given dimension value. + * Returns the child node for the given dimension value in the star-tree. * * @param dimensionValue the dimension value * @return the child node for the given dimension value or null if child is not present * @throws IOException if an I/O error occurs while retrieving the child node */ - StarTreeNode getChildForDimensionValue(long dimensionValue) throws IOException; + StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOException; + + /** + * Returns the child star node for a node in the star-tree. + * + * @return the child node for the star node if star child node is not present + * @throws IOException if an I/O error occurs while retrieving the child node + */ + StarTreeNode getChildStarNode() throws IOException; /** * Returns an iterator over the children of the current star-tree node. diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java new file mode 100644 index 0000000000000..4c4725e78ff15 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java @@ -0,0 +1,103 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.node; + +/** + * Represents the different types of nodes in a StarTreeFactory data structure. + * + *

+ * In order to handle different node types, we use a byte value to represent the node type. + * This enum provides a convenient way to map byte values to their corresponding node types. + * + *

+ * Star and Null Nodes are represented as special cases. Default is the general case. + * Star and null nodes are represented with negative ordinal values to ensure that they are + * sorted before the default nodes, which are sorted based on their dimension values. + * + *

+ * The node type can be one of the following: + *

    + *
  • Star Node: Represented by the value -2. A star node is a special node that represents + * all possible values for a dimension.
  • + *
  • Null Node: Represented by the value -1. A null node indicates the absence of any value + * for a dimension.
  • + *
  • Default Node: Represented by the value 0. A default node represents a node with an + * actual dimension value.
  • + *
+ * + * By default, we want to consider nodes as default node. + * + * @opensearch.experimental + * @see StarTreeNode + */ +public enum StarTreeNodeType { + + /** + * Represents a star node type. + * + */ + STAR("star", (byte) -2), + + /** + * Represents a null node type. + */ + NULL("null", (byte) -1), + + /** + * Represents a default node type. + */ + DEFAULT("default", (byte) 0); + + private final String name; + private final byte value; + + /** + * Constructs a new StarTreeNodeType with the given name and value. + * + * @param name the name of the node type + * @param value the value associated with the node type + */ + StarTreeNodeType(String name, byte value) { + this.name = name; + this.value = value; + } + + /** + * Returns the name of the node type. + * + * @return the name of the node type + */ + public String getName() { + return name; + } + + /** + * Returns the value associated with the node type. + * + * @return the value associated with the node type + */ + public byte getValue() { + return value; + } + + /** + * Returns the StarTreeNodeType enum constant with the specified value. + * + * @param value the value of the enum constant to return + * @return the enum constant with the specified value, or null if no such constant exists + */ + public static StarTreeNodeType fromValue(byte value) { + for (StarTreeNodeType nodeType : StarTreeNodeType.values()) { + if (nodeType.getValue() == value) { + return nodeType; + } + } + throw new IllegalStateException("Unrecognized value byte to determine star-tree node type: [" + value + "]"); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java new file mode 100644 index 0000000000000..dc155df4eafca --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java @@ -0,0 +1,111 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; + +import java.util.Collections; +import java.util.List; + +/** + * Util class for building star tree + * + * @opensearch.experimental + */ +public class StarTreeUtils { + + private StarTreeUtils() {} + + public static final int ALL = -1; + + /** + * The suffix appended to dimension field names in the Star Tree index. + */ + public static final String DIMENSION_SUFFIX = "dim"; + + /** + * The suffix appended to metric field names in the Star Tree index. + */ + public static final String METRIC_SUFFIX = "metric"; + + /** + * Returns the full field name for a dimension in the star-tree index. + * + * @param starTreeFieldName star-tree field name + * @param dimensionName name of the dimension + * @return full field name for the dimension in the star-tree index + */ + public static String fullyQualifiedFieldNameForStarTreeDimensionsDocValues(String starTreeFieldName, String dimensionName) { + return starTreeFieldName + "_" + dimensionName + "_" + DIMENSION_SUFFIX; + } + + /** + * Returns the full field name for a metric in the star-tree index. + * + * @param starTreeFieldName star-tree field name + * @param fieldName name of the metric field + * @param metricName name of the metric + * @return full field name for the metric in the star-tree index + */ + public static String fullyQualifiedFieldNameForStarTreeMetricsDocValues(String starTreeFieldName, String fieldName, String metricName) { + return MetricAggregatorInfo.toFieldName(starTreeFieldName, fieldName, metricName) + "_" + METRIC_SUFFIX; + } + + /** + * Get field infos from field names + * + * @param fields field names + * @return field infos + */ + public static FieldInfo[] getFieldInfoList(List fields) { + FieldInfo[] fieldInfoList = new FieldInfo[fields.size()]; + + // field number is not really used. We depend on unique field names to get the desired iterator + int fieldNumber = 0; + + for (String fieldName : fields) { + fieldInfoList[fieldNumber] = getFieldInfo(fieldName, fieldNumber); + fieldNumber++; + } + return fieldInfoList; + } + + /** + * Get new field info instance for a given field name and field number + * @param fieldName name of the field + * @param fieldNumber number of the field + * @return new field info instance + */ + public static FieldInfo getFieldInfo(String fieldName, int fieldNumber) { + return new FieldInfo( + fieldName, + fieldNumber, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index e77f184ac0243..33088e8ccbcb3 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -36,8 +36,10 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; -import org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; import org.opensearch.index.mapper.ContentPath; import org.opensearch.index.mapper.DocumentMapper; import org.opensearch.index.mapper.Mapper; @@ -1314,7 +1316,7 @@ public void test_build_starTreeDataset() throws IOException { Iterator expectedStarTreeDocumentIterator = expectedStarTreeDocuments(); Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); Map> dimValueToDocIdMap = new HashMap<>(); - builder.rootNode.isStarNode = true; + builder.rootNode.nodeType = StarTreeNodeType.STAR.getValue(); traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); Map> expectedDimToValueMap = getExpectedDimToValueMap(); @@ -2784,13 +2786,13 @@ private static StarTreeField getStarTreeFieldWithDocCount(int maxLeafDocs, boole return sf; } - private void traverseStarTree(TreeNode root, Map> dimValueToDocIdMap, boolean traverStarNodes) { - TreeNode starTree = root; + private void traverseStarTree(InMemoryTreeNode root, Map> dimValueToDocIdMap, boolean traverStarNodes) { + InMemoryTreeNode starTree = root; // Use BFS to traverse the star tree - Queue queue = new ArrayDeque<>(); + Queue queue = new ArrayDeque<>(); queue.add(starTree); int currentDimensionId = -1; - TreeNode starTreeNode; + InMemoryTreeNode starTreeNode; List docIds = new ArrayList<>(); while ((starTreeNode = queue.poll()) != null) { int dimensionId = starTreeNode.dimensionId; @@ -2801,17 +2803,17 @@ private void traverseStarTree(TreeNode root, Map> di // store aggregated document of the node int docId = starTreeNode.aggregatedDocId; Map map = dimValueToDocIdMap.getOrDefault(dimensionId, new HashMap<>()); - if (starTreeNode.isStarNode) { + if (starTreeNode.nodeType == StarTreeNodeType.STAR.getValue()) { map.put(Long.MAX_VALUE, docId); } else { map.put(starTreeNode.dimensionValue, docId); } dimValueToDocIdMap.put(dimensionId, map); - if (starTreeNode.children != null && (!traverStarNodes || starTreeNode.isStarNode)) { - Iterator childrenIterator = starTreeNode.children.values().iterator(); + if (starTreeNode.children != null && (!traverStarNodes || starTreeNode.nodeType == StarTreeNodeType.STAR.getValue())) { + Iterator childrenIterator = starTreeNode.children.values().iterator(); while (childrenIterator.hasNext()) { - TreeNode childNode = childrenIterator.next(); + InMemoryTreeNode childNode = childrenIterator.next(); queue.add(childNode); } } @@ -2933,43 +2935,48 @@ public void testMergeFlow() throws IOException { validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } - private void validateStarTree(TreeNode root, int totalDimensions, int maxLeafDocuments, List starTreeDocuments) { + private void validateStarTree( + InMemoryTreeNode root, + int totalDimensions, + int maxLeafDocuments, + List starTreeDocuments + ) { Queue queue = new LinkedList<>(); queue.offer(new Object[] { root, false }); while (!queue.isEmpty()) { Object[] current = queue.poll(); - TreeNode node = (TreeNode) current[0]; + InMemoryTreeNode node = (InMemoryTreeNode) current[0]; boolean currentIsStarNode = (boolean) current[1]; assertNotNull(node); // assert dimensions - if (node.dimensionId != TreeNode.ALL) { + if (node.dimensionId != StarTreeUtils.ALL) { assertTrue(node.dimensionId >= 0 && node.dimensionId < totalDimensions); } if (node.children != null && !node.children.isEmpty()) { assertEquals(node.dimensionId + 1, node.childDimensionId); assertTrue(node.childDimensionId < totalDimensions); - TreeNode starNode = null; + InMemoryTreeNode starNode = null; Object[] nonStarNodeCumulativeMetrics = getMetrics(starTreeDocuments); - for (Map.Entry entry : node.children.entrySet()) { + for (Map.Entry entry : node.children.entrySet()) { Long childDimensionValue = entry.getKey(); - TreeNode child = entry.getValue(); + InMemoryTreeNode child = entry.getValue(); Object[] currMetrics = getMetrics(starTreeDocuments); - if (!child.isStarNode) { + if (child.nodeType != StarTreeNodeType.STAR.getValue()) { // Validate dimension values in documents for (int i = child.startDocId; i < child.endDocId; i++) { StarTreeDocument doc = starTreeDocuments.get(i); int j = 0; addMetrics(doc, currMetrics, j); - if (!child.isStarNode) { + if (child.nodeType != StarTreeNodeType.STAR.getValue()) { Long dimension = doc.dimensions[child.dimensionId]; assertEquals(childDimensionValue, dimension); if (dimension != null) { assertEquals(child.dimensionValue, (long) dimension); } else { // TODO : fix this ? - assertEquals(child.dimensionValue, TreeNode.ALL); + assertEquals(child.dimensionValue, StarTreeUtils.ALL); } } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java new file mode 100644 index 0000000000000..4653ac8b08198 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java @@ -0,0 +1,209 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.data; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeFactory; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Queue; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class StarTreeFileFormatsTests extends OpenSearchTestCase { + + private IndexOutput dataOut; + private IndexInput dataIn; + private Directory directory; + private Integer maxLevels; + private static Integer dimensionValue; + + @Before + public void setup() throws IOException { + directory = newFSDirectory(createTempDir()); + maxLevels = randomIntBetween(2, 5); + dimensionValue = 0; + } + + public void test_StarTreeNode() throws IOException { + + dataOut = directory.createOutput("star-tree-data", IOContext.DEFAULT); + Map inMemoryTreeNodeMap = new LinkedHashMap<>(); + InMemoryTreeNode root = generateSampleTree(inMemoryTreeNodeMap); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, root, inMemoryTreeNodeMap.size(), "star-tree"); + + // asserting on the actual length of the star tree data file + assertEquals(starTreeDataLength, (inMemoryTreeNodeMap.size() * 33L)); + dataOut.close(); + + dataIn = directory.openInput("star-tree-data", IOContext.READONCE); + + StarTreeMetadata starTreeMetadata = mock(StarTreeMetadata.class); + when(starTreeMetadata.getDataLength()).thenReturn(starTreeDataLength); + when(starTreeMetadata.getDataStartFilePointer()).thenReturn(0L); + + StarTreeNode starTreeNode = StarTreeFactory.createStarTree(dataIn, starTreeMetadata); + Queue queue = new ArrayDeque<>(); + queue.add(starTreeNode); + + while ((starTreeNode = queue.poll()) != null) { + + // verify the star node + assertStarTreeNode(starTreeNode, inMemoryTreeNodeMap.get(starTreeNode.getDimensionValue())); + + Iterator childrenIterator = starTreeNode.getChildrenIterator(); + + if (starTreeNode.getChildDimensionId() != -1) { + while (childrenIterator.hasNext()) { + StarTreeNode child = childrenIterator.next(); + if (child.getStarTreeNodeType() == StarTreeNodeType.DEFAULT.getValue()) { + assertStarTreeNode( + starTreeNode.getChildForDimensionValue(child.getDimensionValue()), + inMemoryTreeNodeMap.get(child.getDimensionValue()) + ); + assertNull(starTreeNode.getChildStarNode()); + } + + queue.add(child); + } + } else { + assertTrue(starTreeNode.isLeaf()); + } + } + + dataIn.close(); + + } + + public void test_starTreeSearch() throws IOException { + + dataOut = directory.createOutput("star-tree-data", IOContext.DEFAULT); + Map inMemoryTreeNodeMap = new LinkedHashMap<>(); + InMemoryTreeNode root = generateSampleTree(inMemoryTreeNodeMap); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, root, inMemoryTreeNodeMap.size(), "star-tree"); + + // asserting on the actual length of the star tree data file + assertEquals(starTreeDataLength, (inMemoryTreeNodeMap.size() * 33L)); + dataOut.close(); + + dataIn = directory.openInput("star-tree-data", IOContext.READONCE); + + StarTreeMetadata starTreeMetadata = mock(StarTreeMetadata.class); + when(starTreeMetadata.getDataLength()).thenReturn(starTreeDataLength); + when(starTreeMetadata.getDataStartFilePointer()).thenReturn(0L); + + StarTreeNode starTreeNode = StarTreeFactory.createStarTree(dataIn, starTreeMetadata); + InMemoryTreeNode inMemoryTreeNode = inMemoryTreeNodeMap.get(starTreeNode.getDimensionValue()); + assertNotNull(inMemoryTreeNode); + + for (int i = 0; i < maxLevels - 1; i++) { + InMemoryTreeNode randomChildNode = randomFrom(inMemoryTreeNode.children.values()); + StarTreeNode randomStarTreeChildNode = starTreeNode.getChildForDimensionValue(randomChildNode.dimensionValue); + + assertNotNull(randomStarTreeChildNode); + assertStarTreeNode(randomStarTreeChildNode, randomChildNode); + + starTreeNode = randomStarTreeChildNode; + inMemoryTreeNode = randomChildNode; + + } + dataIn.close(); + } + + private void assertStarTreeNode(StarTreeNode starTreeNode, InMemoryTreeNode treeNode) throws IOException { + assertEquals(starTreeNode.getDimensionId(), treeNode.dimensionId); + assertEquals(starTreeNode.getDimensionValue(), treeNode.dimensionValue); + assertEquals(starTreeNode.getStartDocId(), treeNode.startDocId); + assertEquals(starTreeNode.getEndDocId(), treeNode.endDocId); + assertEquals(starTreeNode.getChildDimensionId(), treeNode.childDimensionId); + assertEquals(starTreeNode.getAggregatedDocId(), treeNode.aggregatedDocId); + assertEquals(starTreeNode.getStarTreeNodeType(), treeNode.nodeType); + + if (starTreeNode.getChildDimensionId() != -1) { + assertFalse(starTreeNode.isLeaf()); + if (treeNode.children != null) { + assertEquals(starTreeNode.getNumChildren(), treeNode.children.values().size()); + } + } else { + assertTrue(starTreeNode.isLeaf()); + } + + } + + public InMemoryTreeNode generateSampleTree(Map inMemoryTreeNodeMap) { + // Create the root node + InMemoryTreeNode root = new InMemoryTreeNode(); + root.dimensionId = 0; + root.startDocId = randomInt(); + root.endDocId = randomInt(); + root.childDimensionId = 1; + root.aggregatedDocId = randomInt(); + root.nodeType = (byte) 0; + root.children = new HashMap<>(); + + inMemoryTreeNodeMap.put(root.dimensionValue, root); + + // Generate the tree recursively + generateTreeRecursively(root, 1, inMemoryTreeNodeMap); + + return root; + } + + private void generateTreeRecursively(InMemoryTreeNode parent, int currentLevel, Map inMemoryTreeNodeMap) { + if (currentLevel >= this.maxLevels) { + return; // Maximum level reached, stop generating children + } + + int numChildren = randomIntBetween(1, 10); + + for (int i = 0; i < numChildren; i++) { + InMemoryTreeNode child = new InMemoryTreeNode(); + dimensionValue++; + child.dimensionId = currentLevel; + child.dimensionValue = dimensionValue; // Assign a unique dimension value for each child + child.startDocId = randomInt(); + child.endDocId = randomInt(); + child.childDimensionId = (currentLevel == this.maxLevels - 1) ? -1 : (currentLevel + 1); + child.aggregatedDocId = randomInt(); + child.nodeType = (byte) 0; + child.children = new HashMap<>(); + + parent.children.put(child.dimensionValue, child); + inMemoryTreeNodeMap.put(child.dimensionValue, child); + + generateTreeRecursively(child, currentLevel + 1, inMemoryTreeNodeMap); + } + } + + public void tearDown() throws Exception { + super.tearDown(); + dataIn.close(); + dataOut.close(); + directory.close(); + } + +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java new file mode 100644 index 0000000000000..62bd74cc0b3fc --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java @@ -0,0 +1,224 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.meta; + +import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.Version; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.NumericDimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter; +import org.opensearch.index.fielddata.IndexNumericFieldData; +import org.opensearch.index.mapper.CompositeMappedFieldType; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.COMPOSITE_FIELD_MARKER; +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter.VERSION_CURRENT; +import static org.opensearch.index.mapper.CompositeMappedFieldType.CompositeFieldType.STAR_TREE; + +public class StarTreeMetadataTests extends OpenSearchTestCase { + + private IndexOutput metaOut; + private IndexInput metaIn; + private StarTreeField starTreeField; + private SegmentWriteState writeState; + private Directory directory; + private FieldInfo[] fieldsInfo; + private List dimensionsOrder; + private List fields = List.of(); + private List metrics; + private List metricAggregatorInfos = new ArrayList<>(); + private int segmentDocumentCount; + private long dataFilePointer; + private long dataFileLength; + + @Before + public void setup() throws IOException { + fields = List.of("field1", "field2", "field3", "field4", "field5", "field6", "field7", "field8", "field9", "field10"); + directory = newFSDirectory(createTempDir()); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + 6, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + new HashMap<>(), + null + ); + + fieldsInfo = new FieldInfo[fields.size()]; + for (int i = 0; i < fieldsInfo.length; i++) { + fieldsInfo[i] = new FieldInfo( + fields.get(i), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + FieldInfos fieldInfos = new FieldInfos(fieldsInfo); + writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + } + + public void test_starTreeMetadata() throws IOException { + dimensionsOrder = List.of( + new NumericDimension("field1"), + new NumericDimension("field3"), + new NumericDimension("field5"), + new NumericDimension("field8") + ); + metrics = List.of( + new Metric("field2", List.of(MetricStat.SUM)), + new Metric("field4", List.of(MetricStat.SUM)), + new Metric("field6", List.of(MetricStat.VALUE_COUNT)) + ); + int maxLeafDocs = randomInt(Integer.MAX_VALUE); + StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( + maxLeafDocs, + Set.of("field10"), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + starTreeField = new StarTreeField("star_tree", dimensionsOrder, metrics, starTreeFieldConfiguration); + + for (Metric metric : metrics) { + for (MetricStat metricType : metric.getMetrics()) { + MetricAggregatorInfo metricAggregatorInfo = new MetricAggregatorInfo( + metricType, + metric.getField(), + starTreeField.getName(), + IndexNumericFieldData.NumericType.DOUBLE + ); + metricAggregatorInfos.add(metricAggregatorInfo); + } + } + + dataFileLength = randomNonNegativeLong(); + dataFilePointer = randomNonNegativeLong(); + segmentDocumentCount = randomInt(Integer.MAX_VALUE); + metaOut = directory.createOutput("star-tree-metadata", IOContext.DEFAULT); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + int numberOfNodes = randomInt(Integer.MAX_VALUE); + starTreeWriter.writeStarTreeMetadata( + metaOut, + starTreeField, + metricAggregatorInfos, + numberOfNodes, + segmentDocumentCount, + dataFilePointer, + dataFileLength + ); + metaOut.close(); + + // reading and asserting the metadata + metaIn = directory.openInput("star-tree-metadata", IOContext.READONCE); + assertEquals(COMPOSITE_FIELD_MARKER, metaIn.readLong()); + assertEquals(VERSION_CURRENT, metaIn.readVInt()); + + String compositeFieldName = metaIn.readString(); + CompositeMappedFieldType.CompositeFieldType compositeFieldType = CompositeMappedFieldType.CompositeFieldType.fromName( + metaIn.readString() + ); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata(metaIn, compositeFieldName, compositeFieldType, VERSION_CURRENT); + assertEquals(starTreeField.getName(), starTreeMetadata.getStarTreeFieldName()); + assertEquals(starTreeField.getName(), starTreeMetadata.getCompositeFieldName()); + assertEquals(STAR_TREE, starTreeMetadata.getCompositeFieldType()); + assertEquals(STAR_TREE.getName(), starTreeMetadata.getStarTreeFieldType()); + assertEquals(starTreeMetadata.getVersion(), VERSION_CURRENT); + assertEquals(starTreeMetadata.getNumberOfNodes(), numberOfNodes); + assertNotNull(starTreeMetadata); + + for (int i = 0; i < dimensionsOrder.size(); i++) { + assertEquals(dimensionsOrder.get(i).getField(), starTreeMetadata.getDimensionFields().get(i)); + } + + assertEquals(starTreeField.getMetrics().size(), starTreeMetadata.getMetrics().size()); + + for (int i = 0; i < starTreeField.getMetrics().size(); i++) { + + Metric expectedMetric = starTreeField.getMetrics().get(i); + Metric resultMetric = starTreeMetadata.getMetrics().get(i); + + assertEquals(expectedMetric.getField(), resultMetric.getField()); + assertEquals(expectedMetric.getMetrics().size(), resultMetric.getMetrics().size()); + + for (int j = 0; j < resultMetric.getMetrics().size(); j++) { + assertEquals(expectedMetric.getMetrics().get(j), resultMetric.getMetrics().get(j)); + } + } + assertEquals(segmentDocumentCount, starTreeMetadata.getSegmentAggregatedDocCount(), 0); + assertEquals(maxLeafDocs, starTreeMetadata.getMaxLeafDocs(), 0); + assertEquals( + starTreeFieldConfiguration.getSkipStarNodeCreationInDims().size(), + starTreeMetadata.getSkipStarNodeCreationInDims().size() + ); + for (String skipStarNodeCreationInDims : starTreeField.getStarTreeConfig().getSkipStarNodeCreationInDims()) { + assertTrue(starTreeMetadata.getSkipStarNodeCreationInDims().contains(skipStarNodeCreationInDims)); + } + assertEquals(starTreeFieldConfiguration.getBuildMode(), starTreeMetadata.getStarTreeBuildMode()); + assertEquals(dataFileLength, starTreeMetadata.getDataLength()); + assertEquals(dataFilePointer, starTreeMetadata.getDataStartFilePointer()); + + metaIn.close(); + + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + metaOut.close(); + metaIn.close(); + directory.close(); + } + +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java new file mode 100644 index 0000000000000..6f24728c24f30 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java @@ -0,0 +1,233 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.node; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeFactory; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class FixedLengthStarTreeNodeTests extends OpenSearchTestCase { + + private IndexOutput dataOut; + private IndexInput dataIn; + private Directory directory; + InMemoryTreeNode node; + InMemoryTreeNode starChild; + InMemoryTreeNode nullChild; + FixedLengthStarTreeNode starTreeNode; + + @Before + public void setup() throws IOException { + directory = newFSDirectory(createTempDir()); + + dataOut = directory.createOutput("star-tree-data", IOContext.DEFAULT); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + + node = new InMemoryTreeNode(); + node.dimensionId = 0; + node.startDocId = randomInt(); + node.endDocId = randomInt(); + node.childDimensionId = 1; + node.aggregatedDocId = randomInt(); + node.nodeType = randomFrom((byte) 0, (byte) -1, (byte) 2); + node.children = new HashMap<>(); + + starChild = new InMemoryTreeNode(); + starChild.dimensionId = node.dimensionId + 1; + starChild.dimensionValue = -1; + starChild.startDocId = randomInt(); + starChild.endDocId = randomInt(); + starChild.childDimensionId = -1; + starChild.aggregatedDocId = randomInt(); + starChild.nodeType = (byte) -2; + starChild.children = new HashMap<>(); + node.children.put(-1L, starChild); + + nullChild = new InMemoryTreeNode(); + nullChild.dimensionId = node.dimensionId + 1; + nullChild.dimensionValue = -1; + nullChild.startDocId = randomInt(); + nullChild.endDocId = randomInt(); + nullChild.childDimensionId = -1; + nullChild.aggregatedDocId = randomInt(); + nullChild.nodeType = (byte) -1; + nullChild.children = new HashMap<>(); + node.children.put(null, nullChild); + + for (int i = 1; i < randomIntBetween(2, 5); i++) { + InMemoryTreeNode child = new InMemoryTreeNode(); + child.dimensionId = node.dimensionId + 1; + child.dimensionValue = node.dimensionValue + i; // Assign a unique dimension value for each child + child.startDocId = randomInt(); + child.endDocId = randomInt(); + child.childDimensionId = -1; + child.aggregatedDocId = randomInt(); + child.nodeType = (byte) 0; + child.children = new HashMap<>(); + node.children.put(child.dimensionValue, child); + } + + long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, node, 1 + node.children.size(), "star-tree"); + + // asserting on the actual length of the star tree data file + assertEquals(starTreeDataLength, 33L * node.children.size() + 33); + dataOut.close(); + + dataIn = directory.openInput("star-tree-data", IOContext.READONCE); + StarTreeMetadata starTreeMetadata = mock(StarTreeMetadata.class); + when(starTreeMetadata.getDataLength()).thenReturn(starTreeDataLength); + when(starTreeMetadata.getDataStartFilePointer()).thenReturn(0L); + + starTreeNode = (FixedLengthStarTreeNode) StarTreeFactory.createStarTree(dataIn, starTreeMetadata); + + } + + public void testOffsets() { + assertEquals(0, FixedLengthStarTreeNode.DIMENSION_ID_OFFSET); + assertEquals(4, FixedLengthStarTreeNode.DIMENSION_VALUE_OFFSET); + assertEquals(12, FixedLengthStarTreeNode.START_DOC_ID_OFFSET); + assertEquals(16, FixedLengthStarTreeNode.END_DOC_ID_OFFSET); + assertEquals(20, FixedLengthStarTreeNode.AGGREGATE_DOC_ID_OFFSET); + assertEquals(24, FixedLengthStarTreeNode.STAR_NODE_TYPE_OFFSET); + assertEquals(25, FixedLengthStarTreeNode.FIRST_CHILD_ID_OFFSET); + assertEquals(29, FixedLengthStarTreeNode.LAST_CHILD_ID_OFFSET); + } + + public void testSerializableDataSize() { + assertEquals(33, FixedLengthStarTreeNode.SERIALIZABLE_DATA_SIZE_IN_BYTES); + } + + public void testGetDimensionId() throws IOException { + assertEquals(node.dimensionId, starTreeNode.getDimensionId()); + } + + public void testGetDimensionValue() throws IOException { + assertEquals(node.dimensionValue, starTreeNode.getDimensionValue()); + } + + public void testGetStartDocId() throws IOException { + assertEquals(node.startDocId, starTreeNode.getStartDocId()); + } + + public void testGetEndDocId() throws IOException { + assertEquals(node.endDocId, starTreeNode.getEndDocId()); + } + + public void testGetAggregatedDocId() throws IOException { + assertEquals(node.aggregatedDocId, starTreeNode.getAggregatedDocId()); + } + + public void testGetNumChildren() throws IOException { + assertEquals(node.children.size(), starTreeNode.getNumChildren()); + } + + public void testIsLeaf() { + assertFalse(starTreeNode.isLeaf()); + } + + public void testGetStarTreeNodeType() throws IOException { + assertEquals(node.getNodeType(), starTreeNode.getStarTreeNodeType()); + } + + public void testGetChildForDimensionValue() throws IOException { + // TODO: Add a test to verify children with star node, null node and default node with default dimension value -1 + long dimensionValue = randomIntBetween(0, node.children.size() - 3); + FixedLengthStarTreeNode childNode = (FixedLengthStarTreeNode) starTreeNode.getChildForDimensionValue(dimensionValue); + assertNotNull(childNode); + assertEquals(dimensionValue, childNode.getDimensionValue()); + } + + public void testGetChildrenIterator() throws IOException { + Iterator iterator = starTreeNode.getChildrenIterator(); + int count = 0; + while (iterator.hasNext()) { + FixedLengthStarTreeNode child = iterator.next(); + assertNotNull(child); + count++; + } + assertEquals(starTreeNode.getNumChildren(), count); + } + + public void testGetChildForStarNode() throws IOException { + // Assuming the first child is a star node in our test data + FixedLengthStarTreeNode starNode = (FixedLengthStarTreeNode) starTreeNode.getChildStarNode(); + assertNotNull(starNode); + assertEquals(StarTreeUtils.ALL, starNode.getDimensionValue()); + } + + public void testGetChildForNullNode() throws IOException { + FixedLengthStarTreeNode nullNode = (FixedLengthStarTreeNode) starTreeNode.getChildForDimensionValue(null); + assertNull(nullNode); + } + + public void testGetChildForInvalidDimensionValue() throws IOException { + long invalidDimensionValue = Long.MAX_VALUE; + assertThrows(AssertionError.class, () -> starTreeNode.getChildForDimensionValue(invalidDimensionValue)); + } + + public void testOnlyRootNodePresent() throws IOException { + + Directory directory = newFSDirectory(createTempDir()); + + IndexOutput dataOut = directory.createOutput("star-tree-data-1", IOContext.DEFAULT); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + + InMemoryTreeNode node = new InMemoryTreeNode(); + node.dimensionId = 0; + node.startDocId = randomInt(); + node.endDocId = randomInt(); + node.childDimensionId = 1; + node.aggregatedDocId = randomInt(); + node.nodeType = randomFrom((byte) 0, (byte) -1, (byte) 2); + node.children = new HashMap<>(); + + long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, node, 1, "star-tree"); + + // asserting on the actual length of the star tree data file + assertEquals(starTreeDataLength, 33); + dataOut.close(); + + IndexInput dataIn = directory.openInput("star-tree-data-1", IOContext.READONCE); + StarTreeMetadata starTreeMetadata = mock(StarTreeMetadata.class); + when(starTreeMetadata.getDataLength()).thenReturn(starTreeDataLength); + when(starTreeMetadata.getDataStartFilePointer()).thenReturn(0L); + + FixedLengthStarTreeNode starTreeNode = (FixedLengthStarTreeNode) StarTreeFactory.createStarTree(dataIn, starTreeMetadata); + + assertEquals(starTreeNode.getNumChildren(), 0); + assertNull(starTreeNode.getChildForDimensionValue(randomLong())); + assertThrows(IllegalArgumentException.class, () -> starTreeNode.getChildrenIterator().next()); + assertThrows(UnsupportedOperationException.class, () -> starTreeNode.getChildrenIterator().remove()); + + dataIn.close(); + directory.close(); + } + + public void tearDown() throws Exception { + super.tearDown(); + dataIn.close(); + dataOut.close(); + directory.close(); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java new file mode 100644 index 0000000000000..81fb620da5af3 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java @@ -0,0 +1,58 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.node; + +import org.opensearch.test.OpenSearchTestCase; + +public class StarTreeNodeTypeTests extends OpenSearchTestCase { + + public void testStarNodeType() { + assertEquals("star", StarTreeNodeType.STAR.getName()); + assertEquals((byte) -2, StarTreeNodeType.STAR.getValue()); + } + + public void testNullNodeType() { + assertEquals("null", StarTreeNodeType.NULL.getName()); + assertEquals((byte) -1, StarTreeNodeType.NULL.getValue()); + } + + public void testDefaultNodeType() { + assertEquals("default", StarTreeNodeType.DEFAULT.getName()); + assertEquals((byte) 0, StarTreeNodeType.DEFAULT.getValue()); + } + + public void testFromValue() { + assertEquals(StarTreeNodeType.STAR, StarTreeNodeType.fromValue((byte) -2)); + assertEquals(StarTreeNodeType.NULL, StarTreeNodeType.fromValue((byte) -1)); + assertEquals(StarTreeNodeType.DEFAULT, StarTreeNodeType.fromValue((byte) 0)); + } + + public void testFromValueInvalid() { + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> StarTreeNodeType.fromValue((byte) 1)); + assertEquals("Unrecognized value byte to determine star-tree node type: [1]", exception.getMessage()); + } + + public void testEnumValues() { + StarTreeNodeType[] values = StarTreeNodeType.values(); + assertEquals(3, values.length); + assertArrayEquals(new StarTreeNodeType[] { StarTreeNodeType.STAR, StarTreeNodeType.NULL, StarTreeNodeType.DEFAULT }, values); + } + + public void testEnumValueOf() { + assertEquals(StarTreeNodeType.STAR, StarTreeNodeType.valueOf("STAR")); + assertEquals(StarTreeNodeType.NULL, StarTreeNodeType.valueOf("NULL")); + assertEquals(StarTreeNodeType.DEFAULT, StarTreeNodeType.valueOf("DEFAULT")); + } + + public void testEnumValueOfInvalid() { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> StarTreeNodeType.valueOf("INVALID")); + assertTrue(exception.getMessage().contains("No enum constant")); + } + +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java new file mode 100644 index 0000000000000..a1d341615969e --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java @@ -0,0 +1,78 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +public class StarTreeUtilsTests extends OpenSearchTestCase { + + public void testFullyQualifiedFieldNameForStarTreeDimensionsDocValues() { + String starTreeFieldName = "myStarTreeField"; + String dimensionName = "dimension1"; + String expectedFieldName = "myStarTreeField_dimension1_dim"; + + String actualFieldName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeDimensionsDocValues(starTreeFieldName, dimensionName); + assertEquals(expectedFieldName, actualFieldName); + } + + public void testFullyQualifiedFieldNameForStarTreeMetricsDocValues() { + String starTreeFieldName = "myStarTreeField"; + String fieldName = "myField"; + String metricName = "metric1"; + String expectedFieldName = "myStarTreeField_myField_metric1_metric"; + + String actualFieldName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues(starTreeFieldName, fieldName, metricName); + assertEquals(expectedFieldName, actualFieldName); + } + + public void testGetFieldInfoList() { + List fieldNames = Arrays.asList("field1", "field2", "field3"); + FieldInfo[] actualFieldInfos = StarTreeUtils.getFieldInfoList(fieldNames); + for (int i = 0; i < fieldNames.size(); i++) { + assertFieldInfos(actualFieldInfos[i], fieldNames.get(i), i); + } + } + + public void testGetFieldInfo() { + String fieldName = UUID.randomUUID().toString(); + int fieldNumber = randomInt(); + assertFieldInfos(StarTreeUtils.getFieldInfo(fieldName, fieldNumber), fieldName, fieldNumber); + + } + + private void assertFieldInfos(FieldInfo actualFieldInfo, String fieldName, Integer fieldNumber) { + assertEquals(fieldName, actualFieldInfo.name); + assertEquals(fieldNumber, actualFieldInfo.number, 0); + assertFalse(actualFieldInfo.hasVectorValues()); + assertTrue(actualFieldInfo.hasNorms()); + assertFalse(actualFieldInfo.hasVectors()); + assertEquals(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, actualFieldInfo.getIndexOptions()); + assertEquals(DocValuesType.SORTED_NUMERIC, actualFieldInfo.getDocValuesType()); + assertEquals(-1, actualFieldInfo.getDocValuesGen()); + assertEquals(Collections.emptyMap(), actualFieldInfo.attributes()); + assertEquals(0, actualFieldInfo.getPointDimensionCount()); + assertEquals(0, actualFieldInfo.getPointIndexDimensionCount()); + assertEquals(0, actualFieldInfo.getPointNumBytes()); + assertEquals(0, actualFieldInfo.getVectorDimension()); + assertEquals(VectorEncoding.FLOAT32, actualFieldInfo.getVectorEncoding()); + assertEquals(VectorSimilarityFunction.EUCLIDEAN, actualFieldInfo.getVectorSimilarityFunction()); + assertFalse(actualFieldInfo.isSoftDeletesField()); + } + +} diff --git a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java index 449b251dddca1..81454b210d6be 100644 --- a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java @@ -21,6 +21,7 @@ import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.NumericDimension; +import org.opensearch.index.compositeindex.datacube.ReadDimension; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.junit.After; @@ -334,6 +335,14 @@ public void testDimensions() { assertNotEquals(n1, n2); } + public void testReadDimensions() { + ReadDimension r1 = new ReadDimension("name"); + ReadDimension r2 = new ReadDimension("name"); + assertEquals(r1, r2); + r2 = new ReadDimension("name1"); + assertNotEquals(r1, r2); + } + public void testStarTreeField() { List m1 = new ArrayList<>(); m1.add(MetricStat.MAX); From 3726c52b31e8504e7fcf9cdc1b52a0a404d6c944 Mon Sep 17 00:00:00 2001 From: Lakshya Taragi <157457166+ltaragi@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:01:58 +0530 Subject: [PATCH 04/10] Populate RecoveryState details for shallow snapshot restore (#15353) --------- Signed-off-by: Lakshya Taragi --- .../remotestore/RemoteRestoreSnapshotIT.java | 35 +++++++++++++++++++ .../opensearch/index/shard/IndexShard.java | 17 +++++++-- .../index/shard/IndexShardTests.java | 4 +-- 3 files changed, 52 insertions(+), 4 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index dc0654c623137..42e44bd3f37c3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -13,12 +13,14 @@ import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.client.Client; import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.io.PathUtils; @@ -34,6 +36,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -73,6 +76,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteRestoreSnapshotIT extends AbstractSnapshotIntegTestCase { @@ -589,6 +593,37 @@ public void testRestoreShallowSnapshotRepository() throws ExecutionException, In ensureGreen(restoredIndexName1); assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + // ensure recovery details are non-zero + RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries(restoredIndexName1).execute().actionGet(); + assertEquals(1, recoveryResponse.getTotalShards()); + assertEquals(1, recoveryResponse.getSuccessfulShards()); + assertEquals(0, recoveryResponse.getFailedShards()); + assertEquals(1, recoveryResponse.shardRecoveryStates().size()); + assertTrue(recoveryResponse.shardRecoveryStates().containsKey(restoredIndexName1)); + assertEquals(1, recoveryResponse.shardRecoveryStates().get(restoredIndexName1).size()); + + RecoveryState recoveryState = recoveryResponse.shardRecoveryStates().get(restoredIndexName1).get(0); + assertEquals(RecoveryState.Stage.DONE, recoveryState.getStage()); + assertEquals(0, recoveryState.getShardId().getId()); + assertTrue(recoveryState.getPrimary()); + assertEquals(RecoverySource.Type.SNAPSHOT, recoveryState.getRecoverySource().getType()); + assertThat(recoveryState.getIndex().time(), greaterThanOrEqualTo(0L)); + + // ensure populated file details + assertTrue(recoveryState.getIndex().totalFileCount() > 0); + assertTrue(recoveryState.getIndex().totalRecoverFiles() > 0); + assertTrue(recoveryState.getIndex().recoveredFileCount() > 0); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), lessThanOrEqualTo(100.0f)); + assertFalse(recoveryState.getIndex().fileDetails().isEmpty()); + + // ensure populated bytes details + assertTrue(recoveryState.getIndex().recoveredBytes() > 0L); + assertTrue(recoveryState.getIndex().totalBytes() > 0L); + assertTrue(recoveryState.getIndex().totalRecoverBytes() > 0L); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), lessThanOrEqualTo(100.0f)); + // indexing some new docs and validating indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); ensureGreen(restoredIndexName1); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 82b68b32f3bf8..4b18df7a7829d 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -5117,10 +5117,23 @@ public void syncSegmentsFromGivenRemoteSegmentStore( } Map uploadedSegments = sourceRemoteDirectory .getSegmentsUploadedToRemoteStore(); - final Directory storeDirectory = store.directory(); store.incRef(); - try { + final Directory storeDirectory; + if (recoveryState.getStage() == RecoveryState.Stage.INDEX) { + storeDirectory = new StoreRecovery.StatsDirectoryWrapper(store.directory(), recoveryState.getIndex()); + for (String file : uploadedSegments.keySet()) { + long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); + if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { + recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), false); + } else { + recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), true); + } + } + } else { + storeDirectory = store.directory(); + } + String segmentsNFile = copySegmentFiles( storeDirectory, sourceRemoteDirectory, diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 3188de13bb00b..377e4e99e9964 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -2896,9 +2896,9 @@ public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { RecoverySource.ExistingStoreRecoverySource.INSTANCE ); routing = ShardRoutingHelper.newWithRestoreSource(routing, new RecoverySource.EmptyStoreRecoverySource()); - target = reinitShard(target, routing); - + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + target.markAsRecovering("from snapshot", new RecoveryState(routing, localNode, null)); target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, primaryTerm, commitGeneration); RemoteSegmentStoreDirectory remoteStoreDirectory = ((RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) target .remoteStore() From 2b843051a4e71d79fd8fb94014cc8a41a0f97560 Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Thu, 29 Aug 2024 13:09:18 +0530 Subject: [PATCH 05/10] Add minTranslogGeneration to remote translog metadata filename (#15489) Signed-off-by: Sachin Kale Co-authored-by: Sachin Kale --- .../index/translog/transfer/TranslogTransferMetadata.java | 1 + .../index/translog/transfer/TranslogTransferManagerTests.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java index 052206d807fa6..acd7574e648a3 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java @@ -101,6 +101,7 @@ public String getFileName() { RemoteStoreUtils.invertLong(generation), RemoteStoreUtils.invertLong(createdAt), String.valueOf(Objects.hash(nodeId)), + RemoteStoreUtils.invertLong(minTranslogGeneration), String.valueOf(CURRENT_VERSION) ) ); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index c6f9838ad2d52..ffa6a0f456f36 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -623,7 +623,7 @@ public void testMetadataConflict() throws InterruptedException { String mdFilename = tm.getFileName(); long count = mdFilename.chars().filter(ch -> ch == METADATA_SEPARATOR.charAt(0)).count(); // There should not be any `_` in mdFile name as it is used a separator . - assertEquals(10, count); + assertEquals(12, count); Thread.sleep(1); TranslogTransferMetadata tm2 = new TranslogTransferMetadata(1, 1, 1, 2, "node--2"); String mdFilename2 = tm2.getFileName(); From b3d58745a8c299f3937729c9fde9a0ee8c67e707 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 29 Aug 2024 10:10:46 -0400 Subject: [PATCH 06/10] Fix ResourceType API annotations (#15497) Signed-off-by: Andriy Redko --- server/src/main/java/org/opensearch/wlm/ResourceType.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/wlm/ResourceType.java b/server/src/main/java/org/opensearch/wlm/ResourceType.java index adf384995c91d..c3f48f5f793ce 100644 --- a/server/src/main/java/org/opensearch/wlm/ResourceType.java +++ b/server/src/main/java/org/opensearch/wlm/ResourceType.java @@ -18,8 +18,10 @@ /** * Enum to hold the resource type + * + * @opensearch.api */ -@PublicApi(since = "2.x") +@PublicApi(since = "2.17.0") public enum ResourceType { CPU("cpu", task -> task.getTotalResourceUtilization(ResourceStats.CPU), true), MEMORY("memory", task -> task.getTotalResourceUtilization(ResourceStats.MEMORY), true); From 30ed15dea695932082d8ac2cdd661c1a669410dc Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Thu, 29 Aug 2024 20:28:43 +0530 Subject: [PATCH 07/10] Change RemoteSegmentStoreDirectory init at given timestamp to ignore pinned timestamp setting (#15457) Signed-off-by: Sachin Kale Co-authored-by: Sachin Kale --- .../index/remote/RemoteStoreUtils.java | 33 +++++++++- .../store/RemoteSegmentStoreDirectory.java | 3 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../RemoteSegmentStoreDirectoryTests.java | 56 +++++++++++++++++ ...toreDirectoryWithPinnedTimestampTests.java | 62 ------------------- 5 files changed, 90 insertions(+), 67 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index b2bc8a0294a49..871e2eb3ce47f 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -391,15 +391,24 @@ public static boolean isSwitchToStrictCompatibilityMode(ClusterUpdateSettingsReq * @param pinnedTimestampSet A set of timestamps representing pinned points in time. * @param getTimestampFunction A function that extracts the timestamp from a metadata file name. * @param prefixFunction A function that extracts a tuple of prefix information from a metadata file name. + * @param ignorePinnedTimestampEnabledSetting A flag to ignore pinned timestamp enabled setting * @return A set of metadata file names that are implicitly locked based on the pinned timestamps. */ public static Set getPinnedTimestampLockedFiles( List metadataFiles, Set pinnedTimestampSet, Function getTimestampFunction, - Function> prefixFunction + Function> prefixFunction, + boolean ignorePinnedTimestampEnabledSetting ) { - return getPinnedTimestampLockedFiles(metadataFiles, pinnedTimestampSet, new HashMap<>(), getTimestampFunction, prefixFunction); + return getPinnedTimestampLockedFiles( + metadataFiles, + pinnedTimestampSet, + new HashMap<>(), + getTimestampFunction, + prefixFunction, + ignorePinnedTimestampEnabledSetting + ); } /** @@ -431,10 +440,28 @@ public static Set getPinnedTimestampLockedFiles( Map metadataFilePinnedTimestampMap, Function getTimestampFunction, Function> prefixFunction + ) { + return getPinnedTimestampLockedFiles( + metadataFiles, + pinnedTimestampSet, + metadataFilePinnedTimestampMap, + getTimestampFunction, + prefixFunction, + false + ); + } + + private static Set getPinnedTimestampLockedFiles( + List metadataFiles, + Set pinnedTimestampSet, + Map metadataFilePinnedTimestampMap, + Function getTimestampFunction, + Function> prefixFunction, + boolean ignorePinnedTimestampEnabledSetting ) { Set implicitLockedFiles = new HashSet<>(); - if (RemoteStoreSettings.isPinnedTimestampsEnabled() == false) { + if (ignorePinnedTimestampEnabledSetting == false && RemoteStoreSettings.isPinnedTimestampsEnabled() == false) { return implicitLockedFiles; } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 26871429e41d6..53b43bbfb3bba 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -196,7 +196,8 @@ public RemoteSegmentMetadata initializeToSpecificTimestamp(long timestamp) throw metadataFiles, Set.of(timestamp), MetadataFilenameUtils::getTimestamp, - MetadataFilenameUtils::getNodeIdByPrimaryTermAndGen + MetadataFilenameUtils::getNodeIdByPrimaryTermAndGen, + true ); if (lockedMetadataFiles.isEmpty()) { return null; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 388e00bedab0c..9c7dfe8850b85 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -307,6 +307,7 @@ import static org.opensearch.env.NodeEnvironment.collectFileCacheDataPath; import static org.opensearch.index.ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreAttributePresent; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; @@ -814,7 +815,7 @@ protected Node( remoteClusterStateCleanupManager = null; } final RemoteStorePinnedTimestampService remoteStorePinnedTimestampService; - if (isRemoteStoreAttributePresent(settings) && CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.get(settings)) { + if (isRemoteDataAttributePresent(settings) && CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.get(settings)) { remoteStorePinnedTimestampService = new RemoteStorePinnedTimestampService( repositoriesServiceReference::get, settings, diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 574c5bf620474..336d4bafd4b66 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -1141,6 +1141,62 @@ public void testMetadataFileNameOrder() { assertEquals(14, count); } + public void testInitializeToSpecificTimestampNoMetadataFiles() throws IOException { + when( + remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, + Integer.MAX_VALUE + ) + ).thenReturn(new ArrayList<>()); + assertNull(remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L)); + } + + public void testInitializeToSpecificTimestampNoMdMatchingTimestamp() throws IOException { + String metadataPrefix = "metadata__1__2__3__4__5__"; + List metadataFiles = new ArrayList<>(); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(2000)); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(3000)); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(4000)); + + when( + remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, + Integer.MAX_VALUE + ) + ).thenReturn(metadataFiles); + assertNull(remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L)); + } + + public void testInitializeToSpecificTimestampMatchingMdFile() throws IOException { + String metadataPrefix = "metadata__1__2__3__4__5__"; + List metadataFiles = new ArrayList<>(); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(1000)); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(2000)); + metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(3000)); + + Map metadata = new HashMap<>(); + metadata.put("_0.cfe", "_0.cfe::_0.cfe__" + UUIDs.base64UUID() + "::1234::512::" + Version.LATEST.major); + metadata.put("_0.cfs", "_0.cfs::_0.cfs__" + UUIDs.base64UUID() + "::2345::1024::" + Version.LATEST.major); + + when( + remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, + Integer.MAX_VALUE + ) + ).thenReturn(metadataFiles); + when(remoteMetadataDirectory.getBlobStream(metadataPrefix + RemoteStoreUtils.invertLong(1000))).thenReturn( + createMetadataFileBytes(metadata, indexShard.getLatestReplicationCheckpoint(), segmentInfos) + ); + + RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L); + assertNotNull(remoteSegmentMetadata); + Map uploadedSegments = remoteSegmentStoreDirectory + .getSegmentsUploadedToRemoteStore(); + assertEquals(2, uploadedSegments.size()); + assertTrue(uploadedSegments.containsKey("_0.cfe")); + assertTrue(uploadedSegments.containsKey("_0.cfs")); + } + private static class WrapperIndexOutput extends IndexOutput { public IndexOutput indexOutput; diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryWithPinnedTimestampTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryWithPinnedTimestampTests.java index b4f93d706bb1e..107d59aa97549 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryWithPinnedTimestampTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryWithPinnedTimestampTests.java @@ -8,8 +8,6 @@ package org.opensearch.index.store; -import org.apache.lucene.util.Version; -import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.support.PlainBlobMetadata; @@ -18,8 +16,6 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.gateway.remote.model.RemotePinnedTimestamps; import org.opensearch.gateway.remote.model.RemoteStorePinnedTimestampsBlobStore; -import org.opensearch.index.remote.RemoteStoreUtils; -import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.node.Node; @@ -31,7 +27,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; @@ -40,7 +35,6 @@ import org.mockito.Mockito; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED; -import static org.opensearch.test.RemoteStoreTestUtils.createMetadataFileBytes; import static org.hamcrest.CoreMatchers.is; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.anyInt; @@ -143,62 +137,6 @@ private void metadataWithOlderTimestamp() { ); } - public void testInitializeToSpecificTimestampNoMetadataFiles() throws IOException { - when( - remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, - Integer.MAX_VALUE - ) - ).thenReturn(new ArrayList<>()); - assertNull(remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L)); - } - - public void testInitializeToSpecificTimestampNoMdMatchingTimestamp() throws IOException { - String metadataPrefix = "metadata__1__2__3__4__5__"; - List metadataFiles = new ArrayList<>(); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(2000)); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(3000)); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(4000)); - - when( - remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, - Integer.MAX_VALUE - ) - ).thenReturn(metadataFiles); - assertNull(remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L)); - } - - public void testInitializeToSpecificTimestampMatchingMdFile() throws IOException { - String metadataPrefix = "metadata__1__2__3__4__5__"; - List metadataFiles = new ArrayList<>(); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(1000)); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(2000)); - metadataFiles.add(metadataPrefix + RemoteStoreUtils.invertLong(3000)); - - Map metadata = new HashMap<>(); - metadata.put("_0.cfe", "_0.cfe::_0.cfe__" + UUIDs.base64UUID() + "::1234::512::" + Version.LATEST.major); - metadata.put("_0.cfs", "_0.cfs::_0.cfs__" + UUIDs.base64UUID() + "::2345::1024::" + Version.LATEST.major); - - when( - remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, - Integer.MAX_VALUE - ) - ).thenReturn(metadataFiles); - when(remoteMetadataDirectory.getBlobStream(metadataPrefix + RemoteStoreUtils.invertLong(1000))).thenReturn( - createMetadataFileBytes(metadata, indexShard.getLatestReplicationCheckpoint(), segmentInfos) - ); - - RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.initializeToSpecificTimestamp(1234L); - assertNotNull(remoteSegmentMetadata); - Map uploadedSegments = remoteSegmentStoreDirectory - .getSegmentsUploadedToRemoteStore(); - assertEquals(2, uploadedSegments.size()); - assertTrue(uploadedSegments.containsKey("_0.cfe")); - assertTrue(uploadedSegments.containsKey("_0.cfs")); - } - public void testDeleteStaleCommitsNoPinnedTimestampMdFilesLatest() throws Exception { metadataFilename = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( 12, From e982a16667bb2c7fa7e6d3e0618f3bb0c070d377 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Thu, 29 Aug 2024 20:31:15 +0530 Subject: [PATCH 08/10] Make balanced shards allocator timebound (#15239) * Make balanced shards allocator time bound to prioritise critical operations waiting in the pending task queue Signed-off-by: Rishab Nahata --- CHANGELOG.md | 1 + .../cluster/routing/RoutingNodes.java | 4 +- .../allocator/BalancedShardsAllocator.java | 46 +- .../allocator/LocalShardsBalancer.java | 46 +- .../common/settings/ClusterSettings.java | 1 + ...TimeBoundBalancedShardsAllocatorTests.java | 479 ++++++++++++++++++ .../decider/DiskThresholdDeciderTests.java | 12 +- .../cluster/OpenSearchAllocationTestCase.java | 11 + 8 files changed, 591 insertions(+), 9 deletions(-) create mode 100644 server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index b7e4548100df3..f8b695205e789 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Workload Management] QueryGroup resource tracking framework changes ([#13897](https://github.com/opensearch-project/OpenSearch/pull/13897)) - Support filtering on a large list encoded by bitmap ([#14774](https://github.com/opensearch-project/OpenSearch/pull/14774)) - Add slice execution listeners to SearchOperationListener interface ([#15153](https://github.com/opensearch-project/OpenSearch/pull/15153)) +- Make balanced shards allocator timebound ([#15239](https://github.com/opensearch-project/OpenSearch/pull/15239)) - Add allowlist setting for ingest-geoip and ingest-useragent ([#15325](https://github.com/opensearch-project/OpenSearch/pull/15325)) - Adding access to noSubMatches and noOverlappingMatches in Hyphenation ([#13895](https://github.com/opensearch-project/OpenSearch/pull/13895)) - Add support for index level max slice count setting for concurrent segment search ([#15336](https://github.com/opensearch-project/OpenSearch/pull/15336)) diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index ab455f52c4195..b5e74821d41e7 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -1439,7 +1439,9 @@ public void remove() { */ public Iterator nodeInterleavedShardIterator(ShardMovementStrategy shardMovementStrategy) { final Queue> queue = new ArrayDeque<>(); - for (Map.Entry entry : nodesToShards.entrySet()) { + List> nodesToShardsEntrySet = new ArrayList<>(nodesToShards.entrySet()); + Randomness.shuffle(nodesToShardsEntrySet); + for (Map.Entry entry : nodesToShardsEntrySet) { queue.add(entry.getValue().copyShards().iterator()); } if (shardMovementStrategy == ShardMovementStrategy.PRIMARY_FIRST) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 212583d1fb14f..a5193ca602f04 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -54,6 +54,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import java.util.HashMap; import java.util.HashSet; @@ -87,6 +88,7 @@ public class BalancedShardsAllocator implements ShardsAllocator { private static final Logger logger = LogManager.getLogger(BalancedShardsAllocator.class); + public static final TimeValue MIN_ALLOCATOR_TIMEOUT = TimeValue.timeValueSeconds(20); public static final Setting INDEX_BALANCE_FACTOR_SETTING = Setting.floatSetting( "cluster.routing.allocation.balance.index", @@ -169,6 +171,23 @@ public class BalancedShardsAllocator implements ShardsAllocator { Property.NodeScope ); + public static final Setting ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.routing.allocation.balanced_shards_allocator.allocator_timeout", + TimeValue.MINUS_ONE, + TimeValue.MINUS_ONE, + timeValue -> { + if (timeValue.compareTo(MIN_ALLOCATOR_TIMEOUT) < 0 && timeValue.compareTo(TimeValue.MINUS_ONE) != 0) { + throw new IllegalArgumentException( + "Setting [" + + "cluster.routing.allocation.balanced_shards_allocator.allocator_timeout" + + "] should be more than 20s or -1ms to disable timeout" + ); + } + }, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private volatile boolean movePrimaryFirst; private volatile ShardMovementStrategy shardMovementStrategy; @@ -181,6 +200,8 @@ public class BalancedShardsAllocator implements ShardsAllocator { private volatile float threshold; private volatile boolean ignoreThrottleInRestore; + private volatile TimeValue allocatorTimeout; + private long startTime; public BalancedShardsAllocator(Settings settings) { this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); @@ -197,6 +218,7 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting setPreferPrimaryShardBalance(PREFER_PRIMARY_SHARD_BALANCE.get(settings)); setPreferPrimaryShardRebalance(PREFER_PRIMARY_SHARD_REBALANCE.get(settings)); setShardMovementStrategy(SHARD_MOVEMENT_STRATEGY_SETTING.get(settings)); + setAllocatorTimeout(ALLOCATOR_TIMEOUT_SETTING.get(settings)); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_BALANCE, this::setPreferPrimaryShardBalance); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVE_PRIMARY_FIRST_SETTING, this::setMovePrimaryFirst); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVEMENT_STRATEGY_SETTING, this::setShardMovementStrategy); @@ -206,6 +228,7 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_REBALANCE, this::setPreferPrimaryShardRebalance); clusterSettings.addSettingsUpdateConsumer(THRESHOLD_SETTING, this::setThreshold); clusterSettings.addSettingsUpdateConsumer(IGNORE_THROTTLE_FOR_REMOTE_RESTORE, this::setIgnoreThrottleInRestore); + clusterSettings.addSettingsUpdateConsumer(ALLOCATOR_TIMEOUT_SETTING, this::setAllocatorTimeout); } /** @@ -284,6 +307,20 @@ private void setThreshold(float threshold) { this.threshold = threshold; } + private void setAllocatorTimeout(TimeValue allocatorTimeout) { + this.allocatorTimeout = allocatorTimeout; + } + + protected boolean allocatorTimedOut() { + if (allocatorTimeout.equals(TimeValue.MINUS_ONE)) { + if (logger.isTraceEnabled()) { + logger.trace("Allocator timeout is disabled. Will not short circuit allocator tasks"); + } + return false; + } + return System.nanoTime() - this.startTime > allocatorTimeout.nanos(); + } + @Override public void allocate(RoutingAllocation allocation) { if (allocation.routingNodes().size() == 0) { @@ -298,8 +335,10 @@ public void allocate(RoutingAllocation allocation) { threshold, preferPrimaryShardBalance, preferPrimaryShardRebalance, - ignoreThrottleInRestore + ignoreThrottleInRestore, + this::allocatorTimedOut ); + this.startTime = System.nanoTime(); localShardsBalancer.allocateUnassigned(); localShardsBalancer.moveShards(); localShardsBalancer.balance(); @@ -321,7 +360,8 @@ public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, f threshold, preferPrimaryShardBalance, preferPrimaryShardRebalance, - ignoreThrottleInRestore + ignoreThrottleInRestore, + () -> false // as we don't need to check if timed out or not while just understanding ShardAllocationDecision ); AllocateUnassignedDecision allocateUnassignedDecision = AllocateUnassignedDecision.NOT_TAKEN; MoveDecision moveDecision = MoveDecision.NOT_TAKEN; @@ -585,7 +625,7 @@ public Balancer( float threshold, boolean preferPrimaryBalance ) { - super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false, false); + super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false, false, () -> false); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 7e4ae58548c55..adb8ee2cf7e85 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -71,6 +72,7 @@ public class LocalShardsBalancer extends ShardsBalancer { private final float avgPrimaryShardsPerNode; private final BalancedShardsAllocator.NodeSorter sorter; private final Set inEligibleTargetNode; + private final Supplier timedOutFunc; private int totalShardCount = 0; public LocalShardsBalancer( @@ -81,7 +83,8 @@ public LocalShardsBalancer( float threshold, boolean preferPrimaryBalance, boolean preferPrimaryRebalance, - boolean ignoreThrottleInRestore + boolean ignoreThrottleInRestore, + Supplier timedOutFunc ) { this.logger = logger; this.allocation = allocation; @@ -99,6 +102,7 @@ public LocalShardsBalancer( this.preferPrimaryRebalance = preferPrimaryRebalance; this.shardMovementStrategy = shardMovementStrategy; this.ignoreThrottleInRestore = ignoreThrottleInRestore; + this.timedOutFunc = timedOutFunc; } /** @@ -344,6 +348,14 @@ private void balanceByWeights() { final BalancedShardsAllocator.ModelNode[] modelNodes = sorter.modelNodes; final float[] weights = sorter.weights; for (String index : buildWeightOrderedIndices()) { + // Terminate if the time allocated to the balanced shards allocator has elapsed + if (timedOutFunc != null && timedOutFunc.get()) { + logger.info( + "Cannot balance any shard in the cluster as time allocated to balanced shards allocator has elapsed" + + ". Skipping indices iteration" + ); + return; + } IndexMetadata indexMetadata = metadata.index(index); // find nodes that have a shard of this index or where shards of this index are allowed to be allocated to, @@ -368,6 +380,14 @@ private void balanceByWeights() { int lowIdx = 0; int highIdx = relevantNodes - 1; while (true) { + // break if the time allocated to the balanced shards allocator has elapsed + if (timedOutFunc != null && timedOutFunc.get()) { + logger.info( + "Cannot balance any shard in the cluster as time allocated to balanced shards allocator has elapsed" + + ". Skipping relevant nodes iteration" + ); + return; + } final BalancedShardsAllocator.ModelNode minNode = modelNodes[lowIdx]; final BalancedShardsAllocator.ModelNode maxNode = modelNodes[highIdx]; advance_range: if (maxNode.numShards(index) > 0) { @@ -572,6 +592,15 @@ void moveShards() { return; } + // Terminate if the time allocated to the balanced shards allocator has elapsed + if (timedOutFunc != null && timedOutFunc.get()) { + logger.info( + "Cannot move any shard in the cluster as time allocated to balanced shards allocator has elapsed" + + ". Skipping shard iteration" + ); + return; + } + ShardRouting shardRouting = it.next(); if (RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shardRouting, allocation))) { @@ -799,8 +828,23 @@ void allocateUnassigned() { int secondaryLength = 0; int primaryLength = primary.length; ArrayUtil.timSort(primary, comparator); + if (logger.isTraceEnabled()) { + logger.trace("Staring allocation of [{}] unassigned shards", primaryLength); + } do { for (int i = 0; i < primaryLength; i++) { + if (timedOutFunc != null && timedOutFunc.get()) { + // TODO - maybe check if we can allow wait for active shards thingy bypass this condition + logger.info( + "Ignoring [{}] unassigned shards for allocation as time allocated to balanced shards allocator has elapsed", + (primaryLength - i) + ); + while (i < primaryLength) { + unassigned.ignoreShard(primary[i], UnassignedInfo.AllocationStatus.NO_ATTEMPT, allocation.changes()); + i++; + } + return; + } ShardRouting shard = primary[i]; final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard); final String assignedNodeId = allocationDecision.getTargetNode() != null diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 8daf9125bb27e..9a6b3f1118709 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -270,6 +270,7 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.SHARD_MOVEMENT_STRATEGY_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, BalancedShardsAllocator.IGNORE_THROTTLE_FOR_REMOTE_RESTORE, + BalancedShardsAllocator.ALLOCATOR_TIMEOUT_SETTING, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, BreakerSettings.CIRCUIT_BREAKER_TYPE, diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java new file mode 100644 index 0000000000000..a10c305686638 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java @@ -0,0 +1,479 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.routing.allocation.allocator; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterInfo; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +import static org.opensearch.cluster.routing.ShardRoutingState.INITIALIZING; +import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; +import static org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator.ALLOCATOR_TIMEOUT_SETTING; + +public class TimeBoundBalancedShardsAllocatorTests extends OpenSearchAllocationTestCase { + + private final DiscoveryNode node1 = newNode("node1", "node1", Collections.singletonMap("zone", "1a")); + private final DiscoveryNode node2 = newNode("node2", "node2", Collections.singletonMap("zone", "1b")); + private final DiscoveryNode node3 = newNode("node3", "node3", Collections.singletonMap("zone", "1c")); + + public void testAllUnassignedShardsAllocatedWhenNoTimeOut() { + int numberOfIndices = 2; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalPrimaryCount = numberOfIndices * numberOfShards; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Settings.Builder settings = Settings.builder(); + // passing total shard count for timed out latch such that no shard times out + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(totalShardCount)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); + int node2Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node2.getId()); + int node3Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node3.getId()); + assertEquals(totalShardCount, initializingShards.size()); + assertEquals(0, allocation.routingNodes().unassigned().ignored().size()); + assertEquals(totalPrimaryCount, node1Recoveries + node2Recoveries + node3Recoveries); + } + + public void testAllUnassignedShardsIgnoredWhenTimedOut() { + int numberOfIndices = 2; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Settings.Builder settings = Settings.builder(); + // passing 0 for timed out latch such that all shard times out + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(0)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); + int node2Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node2.getId()); + int node3Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node3.getId()); + assertEquals(0, initializingShards.size()); + assertEquals(totalShardCount, allocation.routingNodes().unassigned().ignored().size()); + assertEquals(0, node1Recoveries + node2Recoveries + node3Recoveries); + } + + public void testAllocatePartialPrimaryShardsUntilTimedOut() { + int numberOfIndices = 2; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Settings.Builder settings = Settings.builder(); + int shardsToAllocate = randomIntBetween(1, numberOfShards * numberOfIndices); + // passing shards to allocate for timed out latch such that only few primary shards are allocated in this reroute round + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(shardsToAllocate)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); + int node2Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node2.getId()); + int node3Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node3.getId()); + assertEquals(shardsToAllocate, initializingShards.size()); + assertEquals(totalShardCount - shardsToAllocate, allocation.routingNodes().unassigned().ignored().size()); + assertEquals(shardsToAllocate, node1Recoveries + node2Recoveries + node3Recoveries); + } + + public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOut() { + int numberOfIndices = 2; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Settings.Builder settings = Settings.builder(); + int shardsToAllocate = randomIntBetween(numberOfShards * numberOfIndices, totalShardCount); + // passing shards to allocate for timed out latch such that all primary shards and few replica shards are allocated in this reroute + // round + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(shardsToAllocate)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); + int node2Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node2.getId()); + int node3Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node3.getId()); + assertEquals(shardsToAllocate, initializingShards.size()); + assertEquals(totalShardCount - shardsToAllocate, allocation.routingNodes().unassigned().ignored().size()); + assertEquals(numberOfShards * numberOfIndices, node1Recoveries + node2Recoveries + node3Recoveries); + } + + public void testAllShardsMoveWhenExcludedAndTimeoutNotBreached() { + int numberOfIndices = 3; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService(); + state = applyStartedShardsUntilNoChange(state, allocationService); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + int node1ShardCount = state.getRoutingNodes().node("node1").size(); + Settings settings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build(); + int shardsToMove = 10 + 1000; // such that time out is never breached + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(shardsToMove)); + RoutingAllocation allocation = new RoutingAllocation( + allocationDecidersForExcludeAPI(settings), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(node1ShardCount, relocatingShards.size()); + } + + public void testNoShardsMoveWhenExcludedAndTimeoutBreached() { + int numberOfIndices = 3; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService(); + state = applyStartedShardsUntilNoChange(state, allocationService); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + Settings settings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build(); + int shardsToMove = 0; // such that time out is never breached + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(shardsToMove)); + RoutingAllocation allocation = new RoutingAllocation( + allocationDecidersForExcludeAPI(settings), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(0, relocatingShards.size()); + } + + public void testPartialShardsMoveWhenExcludedAndTimeoutBreached() { + int numberOfIndices = 3; + int numberOfShards = 5; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService(); + state = applyStartedShardsUntilNoChange(state, allocationService); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + Settings settings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build(); + // since for moves, it creates an iterator over shards which interleaves between nodes, hence + // for shardsToMove=6, it will have 2 shards from node1, node2, node3 each attempting to move with only + // shards from node1 can actually move. Hence, total moves that will be executed is 2 (6/3). + int shardsToMove = 6; // such that time out is never breached + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(shardsToMove)); + RoutingAllocation allocation = new RoutingAllocation( + allocationDecidersForExcludeAPI(settings), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(shardsToMove / 3, relocatingShards.size()); + } + + public void testClusterRebalancedWhenNotTimedOut() { + int numberOfIndices = 1; + int numberOfShards = 15; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService( + Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() + ); // such that no shards are allocated to node1 + state = applyStartedShardsUntilNoChange(state, allocationService); + int node1ShardCount = state.getRoutingNodes().node("node1").size(); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + assertEquals(0, node1ShardCount); + Settings newSettings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "").build(); + int shardsToMove = 1000; // such that time out is never breached + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(newSettings, new CountDownLatch(shardsToMove)); + RoutingAllocation allocation = new RoutingAllocation( + allocationDecidersForExcludeAPI(newSettings), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(totalShardCount / 3, relocatingShards.size()); + } + + public void testClusterNotRebalancedWhenTimedOut() { + int numberOfIndices = 1; + int numberOfShards = 15; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService( + Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() + ); // such that no shards are allocated to node1 + state = applyStartedShardsUntilNoChange(state, allocationService); + int node1ShardCount = state.getRoutingNodes().node("node1").size(); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + assertEquals(0, node1ShardCount); + Settings newSettings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "").build(); + int shardsToMove = 0; // such that it never balances anything + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(newSettings, new CountDownLatch(shardsToMove)); + RoutingAllocation allocation = new RoutingAllocation( + allocationDecidersForExcludeAPI(newSettings), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(0, relocatingShards.size()); + } + + public void testClusterPartialRebalancedWhenTimedOut() { + int numberOfIndices = 1; + int numberOfShards = 15; + int numberOfReplicas = 1; + int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); + Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); + RoutingTable routingTable = buildRoutingTable(metadata); + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + MockAllocationService allocationService = createAllocationService( + Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() + ); // such that no shards are allocated to node1 + state = applyStartedShardsUntilNoChange(state, allocationService); + int node1ShardCount = state.getRoutingNodes().node("node1").size(); + // check all shards allocated + assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); + assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); + assertEquals(0, node1ShardCount); + Settings newSettings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "").build(); + + // making custom set of allocation deciders such that it never attempts to move shards but always attempts to rebalance + List allocationDeciders = Arrays.asList(new AllocationDecider() { + @Override + public Decision canMoveAnyShard(RoutingAllocation allocation) { + return Decision.NO; + } + }, new AllocationDecider() { + @Override + public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) { + return Decision.YES; + } + }, new SameShardAllocationDecider(newSettings, new ClusterSettings(newSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + int shardsToMove = 3; // such that it only partially balances few shards + // adding +1 as during rebalance we do per index timeout check and then per node check + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(newSettings, new CountDownLatch(shardsToMove + 1)); + RoutingAllocation allocation = new RoutingAllocation( + new AllocationDeciders(allocationDeciders), + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + null, + System.nanoTime() + ); + allocator.allocate(allocation); + List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); + assertEquals(3, relocatingShards.size()); + } + + public void testAllocatorNeverTimedOutIfValueIsMinusOne() { + Settings build = Settings.builder().put("cluster.routing.allocation.balanced_shards_allocator.allocator_timeout", "-1").build(); + BalancedShardsAllocator allocator = new BalancedShardsAllocator(build); + assertFalse(allocator.allocatorTimedOut()); + } + + public void testAllocatorTimeout() { + String settingKey = "cluster.routing.allocation.balanced_shards_allocator.allocator_timeout"; + // Valid setting with timeout = 20s + Settings build = Settings.builder().put(settingKey, "20s").build(); + assertEquals(20, ALLOCATOR_TIMEOUT_SETTING.get(build).getSeconds()); + + // Valid setting with timeout > 20s + build = Settings.builder().put(settingKey, "30000ms").build(); + assertEquals(30, ALLOCATOR_TIMEOUT_SETTING.get(build).getSeconds()); + + // Invalid setting with timeout < 20s + Settings lessThan20sSetting = Settings.builder().put(settingKey, "10s").build(); + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> ALLOCATOR_TIMEOUT_SETTING.get(lessThan20sSetting) + ); + assertEquals("Setting [" + settingKey + "] should be more than 20s or -1ms to disable timeout", iae.getMessage()); + + // Valid setting with timeout = -1 + build = Settings.builder().put(settingKey, "-1").build(); + assertEquals(-1, ALLOCATOR_TIMEOUT_SETTING.get(build).getMillis()); + } + + private RoutingTable buildRoutingTable(Metadata metadata) { + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + for (Map.Entry entry : metadata.getIndices().entrySet()) { + routingTableBuilder.addAsNew(entry.getValue()); + } + return routingTableBuilder.build(); + } + + private Metadata buildMetadata(Metadata.Builder mb, int numberOfIndices, int numberOfShards, int numberOfReplicas) { + for (int i = 0; i < numberOfIndices; i++) { + mb.put( + IndexMetadata.builder("test_" + i) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + ); + } + + return mb.build(); + } + + static class TestBalancedShardsAllocator extends BalancedShardsAllocator { + private final CountDownLatch timedOutLatch; + + public TestBalancedShardsAllocator(Settings settings, CountDownLatch timedOutLatch) { + super(settings); + this.timedOutLatch = timedOutLatch; + } + + @Override + protected boolean allocatorTimedOut() { + if (timedOutLatch.getCount() == 0) { + return true; + } + timedOutLatch.countDown(); + return false; + } + } +} diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 2e24640fe858d..94e91c3f7c3c1 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -530,6 +530,8 @@ public void testDiskThresholdWithAbsoluteSizes() { // Primary should initialize, even though both nodes are over the limit initialize assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); + // below checks are unnecessary as the primary shard is always assigned to node2 as BSA always picks up that node + // first as both node1 and node2 have equal weight as both of them contain zero shards. String nodeWithPrimary, nodeWithoutPrimary; if (clusterState.getRoutingNodes().node("node1").size() == 1) { nodeWithPrimary = "node1"; @@ -679,10 +681,12 @@ public void testDiskThresholdWithAbsoluteSizes() { clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); - // primary shard already has been relocated away - assertThat(clusterState.getRoutingNodes().node(nodeWithPrimary).size(), equalTo(0)); - // node with increased space still has its shard - assertThat(clusterState.getRoutingNodes().node(nodeWithoutPrimary).size(), equalTo(1)); + // primary shard already has been relocated away - this is a wrong expectation as we don't really move + // primary first unless explicitly set by setting. This is caught with PR + // https://github.com/opensearch-project/OpenSearch/pull/15239/ + // as it randomises nodes to check for potential moves + // assertThat(clusterState.getRoutingNodes().node(nodeWithPrimary).size(), equalTo(0)); + // assertThat(clusterState.getRoutingNodes().node(nodeWithoutPrimary).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node4").size(), equalTo(1)); diff --git a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java index 34b8c58a9c5b2..f54ba36203684 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java @@ -48,6 +48,7 @@ import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.cluster.routing.allocation.decider.FilterAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; @@ -213,6 +214,16 @@ protected static AllocationDeciders throttleAllocationDeciders() { ); } + protected static AllocationDeciders allocationDecidersForExcludeAPI(Settings settings) { + return new AllocationDeciders( + Arrays.asList( + new TestAllocateDecision(Decision.YES), + new SameShardAllocationDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + new FilterAllocationDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)) + ) + ); + } + protected ClusterState applyStartedShardsUntilNoChange(ClusterState clusterState, AllocationService service) { ClusterState lastClusterState; do { From e5fadba7b82da4da714cac37aa335a3be230eace Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Thu, 29 Aug 2024 23:34:25 +0800 Subject: [PATCH 09/10] Update version check for fix the bug of missing validation for the search_backpressure.mode setting (#15500) Signed-off-by: Gao Binlong --- .../rest-api-spec/test/cluster.put_settings/10_basic.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml index 2bc5e98465e16..107d298b597d3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml @@ -85,8 +85,8 @@ "Test set invalid search backpressure mode": - skip: - version: "- 2.99.99" - reason: "Parsing and validation of SearchBackpressureMode does not exist in versions < 3.0" + version: "- 2.8.99" + reason: "Fixed in 2.9.0" - do: catch: bad_request From e146f13a69c8bee87a936416e8998710521518a3 Mon Sep 17 00:00:00 2001 From: Pranshu Shukla <55992439+Pranshu-S@users.noreply.github.com> Date: Thu, 29 Aug 2024 21:12:15 +0530 Subject: [PATCH 10/10] Optimize NodeIndicesStats output behind flag (#14454) * Optimize NodeIndicesStats output behind flag Signed-off-by: Pranshu Shukla --- CHANGELOG.md | 1 + .../org/opensearch/nodestats/NodeStatsIT.java | 309 ++++++++++++++ .../admin/indices/stats/CommonStatsFlags.java | 15 + .../opensearch/indices/IndicesService.java | 8 +- .../opensearch/indices/NodeIndicesStats.java | 199 +++++++-- .../admin/cluster/RestNodesStatsAction.java | 1 + .../rest/action/cat/RestNodesAction.java | 1 + .../cluster/node/stats/NodeStatsTests.java | 400 ++++++++++++++++++ 8 files changed, 904 insertions(+), 30 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f8b695205e789..cbfde6a1c1a80 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -32,6 +32,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add concurrent search support for Derived Fields ([#15326](https://github.com/opensearch-project/OpenSearch/pull/15326)) - [Workload Management] Add query group stats constructs ([#15343](https://github.com/opensearch-project/OpenSearch/pull/15343))) - Add runAs to Subject interface and introduce IdentityAwarePlugin extension point ([#14630](https://github.com/opensearch-project/OpenSearch/pull/14630)) +- Optimize NodeIndicesStats output behind flag ([#14454](https://github.com/opensearch-project/OpenSearch/pull/14454)) ### Dependencies - Bump `netty` from 4.1.111.Final to 4.1.112.Final ([#15081](https://github.com/opensearch-project/OpenSearch/pull/15081)) diff --git a/server/src/internalClusterTest/java/org/opensearch/nodestats/NodeStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/nodestats/NodeStatsIT.java index f270cb1399072..22c1679babb52 100644 --- a/server/src/internalClusterTest/java/org/opensearch/nodestats/NodeStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/nodestats/NodeStatsIT.java @@ -10,6 +10,9 @@ import org.opensearch.ExceptionsHelper; import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.node.stats.NodeStats; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.bulk.BulkItemResponse; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.BulkResponse; @@ -19,21 +22,35 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.action.update.UpdateRequest; import org.opensearch.action.update.UpdateResponse; +import org.opensearch.cluster.ClusterState; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.engine.DocumentMissingException; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.index.shard.IndexingStats.Stats.DocStatusStats; +import org.opensearch.indices.NodeIndicesStats; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; import org.opensearch.test.OpenSearchIntegTestCase.Scope; import org.hamcrest.MatcherAssert; +import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.singletonMap; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -243,6 +260,280 @@ public void testNodeIndicesStatsDocStatusStatsCreateDeleteUpdate() { } } + public void testNodeIndicesStatsDocStatsWithAggregations() { + { // Testing Create + final String INDEX = "create_index"; + final String ID = "id"; + DocStatusStats expectedDocStatusStats = new DocStatusStats(); + + IndexResponse response = client().index(new IndexRequest(INDEX).id(ID).source(SOURCE).create(true)).actionGet(); + expectedDocStatusStats.inc(response.status()); + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + + DocStatusStats docStatusStats = client().admin() + .cluster() + .prepareNodesStats() + .setIndices(commonStatsFlags) + .execute() + .actionGet() + .getNodes() + .get(0) + .getIndices() + .getIndexing() + .getTotal() + .getDocStatusStats(); + + assertTrue( + Arrays.equals( + docStatusStats.getDocStatusCounter(), + expectedDocStatusStats.getDocStatusCounter(), + Comparator.comparingLong(AtomicLong::longValue) + ) + ); + } + } + + /** + * Default behavior - without consideration of request level param on level, the NodeStatsRequest always + * returns ShardStats which is aggregated on the coordinator node when creating the XContent. + */ + public void testNodeIndicesStatsXContentWithoutAggregationOnNodes() { + List testLevels = new ArrayList<>(); + testLevels.add("null"); + testLevels.add(NodeIndicesStats.StatsLevel.NODE.getRestName()); + testLevels.add(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + testLevels.add(NodeIndicesStats.StatsLevel.SHARDS.getRestName()); + testLevels.add("unknown"); + + internalCluster().startNode(); + ensureGreen(); + String indexName = "test1"; + assertAcked( + prepareCreate( + indexName, + clusterService().state().getNodes().getSize(), + Settings.builder().put("number_of_shards", 2).put("number_of_replicas", clusterService().state().getNodes().getSize() - 1) + ) + ); + ensureGreen(); + ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + + testLevels.forEach(testLevel -> { + NodesStatsResponse response; + if (!testLevel.equals("null")) { + ArrayList level_arg = new ArrayList<>(); + level_arg.add(testLevel); + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.setLevels(level_arg.toArray(new String[0])); + response = client().admin().cluster().prepareNodesStats().setIndices(commonStatsFlags).get(); + } else { + response = client().admin().cluster().prepareNodesStats().get(); + } + + NodeStats nodeStats = response.getNodes().get(0); + assertNotNull(nodeStats.getIndices().getShardStats(clusterState.metadata().index(indexName).getIndex())); + try { + // Without any param - default is level = nodes + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder = nodeStats.getIndices().toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + Map xContentMap = xContentBuilderToMap(builder); + LinkedHashMap indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES)); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS)); + + // With param containing level as 'indices', the indices stats are returned + builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder = nodeStats.getIndices() + .toXContent( + builder, + new ToXContent.MapParams(Collections.singletonMap("level", NodeIndicesStats.StatsLevel.INDICES.getRestName())) + ); + builder.endObject(); + + xContentMap = xContentBuilderToMap(builder); + indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + assertTrue(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES.getRestName())); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS.getRestName())); + + LinkedHashMap indexLevelStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + assertTrue(indexLevelStats.containsKey(indexName)); + + // With param containing level as 'shards', the shard stats are returned + builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder = nodeStats.getIndices() + .toXContent( + builder, + new ToXContent.MapParams(Collections.singletonMap("level", NodeIndicesStats.StatsLevel.SHARDS.getRestName())) + ); + builder.endObject(); + + xContentMap = xContentBuilderToMap(builder); + indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES.getRestName())); + assertTrue(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS.getRestName())); + + LinkedHashMap shardLevelStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.SHARDS.getRestName()); + assertTrue(shardLevelStats.containsKey(indexName)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + /** + * Aggregated behavior - to avoid unnecessary IO in the form of shard-stats when not required, we not honor the levels on the + * individual data nodes instead and pre-compute information as required. + */ + public void testNodeIndicesStatsXContentWithAggregationOnNodes() { + List testLevels = new ArrayList<>(); + + testLevels.add(MockStatsLevel.NULL); + testLevels.add(MockStatsLevel.NODE); + testLevels.add(MockStatsLevel.INDICES); + testLevels.add(MockStatsLevel.SHARDS); + + internalCluster().startNode(); + ensureGreen(); + String indexName = "test1"; + assertAcked( + prepareCreate( + indexName, + clusterService().state().getNodes().getSize(), + Settings.builder().put("number_of_shards", 2).put("number_of_replicas", clusterService().state().getNodes().getSize() - 1) + ) + ); + ensureGreen(); + + testLevels.forEach(testLevel -> { + NodesStatsResponse response; + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + if (!testLevel.equals(MockStatsLevel.NULL)) { + ArrayList level_arg = new ArrayList<>(); + level_arg.add(testLevel.getRestName()); + + commonStatsFlags.setLevels(level_arg.toArray(new String[0])); + } + response = client().admin().cluster().prepareNodesStats().setIndices(commonStatsFlags).get(); + + NodeStats nodeStats = response.getNodes().get(0); + try { + XContentBuilder builder = XContentFactory.jsonBuilder(); + + builder.startObject(); + + if (!testLevel.equals(MockStatsLevel.SHARDS)) { + final XContentBuilder failedBuilder = builder; + assertThrows( + "Expected shard stats in response for generating [SHARDS] field", + AssertionError.class, + () -> nodeStats.getIndices() + .toXContent( + failedBuilder, + new ToXContent.MapParams( + Collections.singletonMap("level", NodeIndicesStats.StatsLevel.SHARDS.getRestName()) + ) + ) + ); + } else { + builder = nodeStats.getIndices() + .toXContent( + builder, + new ToXContent.MapParams(Collections.singletonMap("level", NodeIndicesStats.StatsLevel.SHARDS.getRestName())) + ); + builder.endObject(); + + Map xContentMap = xContentBuilderToMap(builder); + LinkedHashMap indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + LinkedHashMap indicesStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + LinkedHashMap shardStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.SHARDS.getRestName()); + + assertFalse(shardStats.isEmpty()); + assertNull(indicesStats); + } + + builder = XContentFactory.jsonBuilder(); + builder.startObject(); + + if (!(testLevel.equals(MockStatsLevel.SHARDS) || testLevel.equals(MockStatsLevel.INDICES))) { + final XContentBuilder failedBuilder = builder; + assertThrows( + "Expected shard stats or index stats in response for generating INDICES field", + AssertionError.class, + () -> nodeStats.getIndices() + .toXContent( + failedBuilder, + new ToXContent.MapParams( + Collections.singletonMap("level", NodeIndicesStats.StatsLevel.INDICES.getRestName()) + ) + ) + ); + } else { + builder = nodeStats.getIndices() + .toXContent( + builder, + new ToXContent.MapParams(Collections.singletonMap("level", NodeIndicesStats.StatsLevel.INDICES.getRestName())) + ); + builder.endObject(); + + Map xContentMap = xContentBuilderToMap(builder); + LinkedHashMap indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + LinkedHashMap indicesStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + LinkedHashMap shardStats = (LinkedHashMap) indicesStatsMap.get(NodeIndicesStats.StatsLevel.SHARDS.getRestName()); + + switch (testLevel) { + case SHARDS: + case INDICES: + assertNull(shardStats); + assertFalse(indicesStats.isEmpty()); + break; + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + public void testNodeIndicesStatsUnknownLevelThrowsException() { + MockStatsLevel testLevel = MockStatsLevel.UNKNOWN; + internalCluster().startNode(); + ensureGreen(); + String indexName = "test1"; + assertAcked( + prepareCreate( + indexName, + clusterService().state().getNodes().getSize(), + Settings.builder().put("number_of_shards", 2).put("number_of_replicas", clusterService().state().getNodes().getSize() - 1) + ) + ); + ensureGreen(); + + NodesStatsResponse response; + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + ArrayList level_arg = new ArrayList<>(); + level_arg.add(testLevel.getRestName()); + + commonStatsFlags.setLevels(level_arg.toArray(new String[0])); + response = client().admin().cluster().prepareNodesStats().setIndices(commonStatsFlags).get(); + + assertTrue(response.hasFailures()); + assertEquals("Level provided is not supported by NodeIndicesStats", response.failures().get(0).getCause().getCause().getMessage()); + } + + private Map xContentBuilderToMap(XContentBuilder xContentBuilder) { + return XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(); + } + private void assertDocStatusStats() { DocStatusStats docStatusStats = client().admin() .cluster() @@ -273,4 +564,22 @@ private void updateExpectedDocStatusCounter(Exception e) { expectedDocStatusStats.inc(ExceptionsHelper.status(e)); } + private enum MockStatsLevel { + INDICES(NodeIndicesStats.StatsLevel.INDICES.getRestName()), + SHARDS(NodeIndicesStats.StatsLevel.SHARDS.getRestName()), + NODE(NodeIndicesStats.StatsLevel.NODE.getRestName()), + NULL("null"), + UNKNOWN("unknown"); + + private final String restName; + + MockStatsLevel(String restName) { + this.restName = restName; + } + + public String getRestName() { + return restName; + } + } + } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index ca2685e093d3f..04f39d77ce6c8 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -67,6 +67,7 @@ public class CommonStatsFlags implements Writeable, Cloneable { // Used for metric CACHE_STATS, to determine which caches to report stats for private EnumSet includeCaches = EnumSet.noneOf(CacheType.class); private String[] levels = new String[0]; + private boolean includeIndicesStatsByLevel = false; /** * @param flags flags to set. If no flags are supplied, default flags will be set. @@ -100,6 +101,9 @@ public CommonStatsFlags(StreamInput in) throws IOException { includeCaches = in.readEnumSet(CacheType.class); levels = in.readStringArray(); } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + includeIndicesStatsByLevel = in.readBoolean(); + } } @Override @@ -124,6 +128,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeEnumSet(includeCaches); out.writeStringArrayNullable(levels); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeBoolean(includeIndicesStatsByLevel); + } } /** @@ -262,6 +269,14 @@ public boolean includeSegmentFileSizes() { return this.includeSegmentFileSizes; } + public void setIncludeIndicesStatsByLevel(boolean includeIndicesStatsByLevel) { + this.includeIndicesStatsByLevel = includeIndicesStatsByLevel; + } + + public boolean getIncludeIndicesStatsByLevel() { + return this.includeIndicesStatsByLevel; + } + public boolean isSet(Flag flag) { return flags.contains(flag); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index a78328e24c588..be16d4ea184fa 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -693,8 +693,12 @@ public NodeIndicesStats stats(CommonStatsFlags flags) { break; } } - - return new NodeIndicesStats(commonStats, statsByShard(this, flags), searchRequestStats); + if (flags.getIncludeIndicesStatsByLevel()) { + NodeIndicesStats.StatsLevel statsLevel = NodeIndicesStats.getAcceptedLevel(flags.getLevels()); + return new NodeIndicesStats(commonStats, statsByShard(this, flags), searchRequestStats, statsLevel); + } else { + return new NodeIndicesStats(commonStats, statsByShard(this, flags), searchRequestStats); + } } Map> statsByShard(final IndicesService indicesService, final CommonStatsFlags flags) { diff --git a/server/src/main/java/org/opensearch/indices/NodeIndicesStats.java b/server/src/main/java/org/opensearch/indices/NodeIndicesStats.java index 35b6fd395ee12..83a759cdb71c5 100644 --- a/server/src/main/java/org/opensearch/indices/NodeIndicesStats.java +++ b/server/src/main/java/org/opensearch/indices/NodeIndicesStats.java @@ -32,6 +32,7 @@ package org.opensearch.indices; +import org.opensearch.Version; import org.opensearch.action.admin.indices.stats.CommonStats; import org.opensearch.action.admin.indices.stats.IndexShardStats; import org.opensearch.action.admin.indices.stats.ShardStats; @@ -63,9 +64,11 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; /** * Global information on indices stats running on a specific node. @@ -74,26 +77,27 @@ */ @PublicApi(since = "1.0.0") public class NodeIndicesStats implements Writeable, ToXContentFragment { - private CommonStats stats; - private Map> statsByShard; + protected CommonStats stats; + protected Map statsByIndex; + protected Map> statsByShard; public NodeIndicesStats(StreamInput in) throws IOException { stats = new CommonStats(in); - if (in.readBoolean()) { - int entries = in.readVInt(); - statsByShard = new HashMap<>(); - for (int i = 0; i < entries; i++) { - Index index = new Index(in); - int indexShardListSize = in.readVInt(); - List indexShardStats = new ArrayList<>(indexShardListSize); - for (int j = 0; j < indexShardListSize; j++) { - indexShardStats.add(new IndexShardStats(in)); - } - statsByShard.put(index, indexShardStats); + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + // contains statsByIndex + if (in.readBoolean()) { + statsByIndex = readStatsByIndex(in); } } + if (in.readBoolean()) { + statsByShard = readStatsByShard(in); + } } + /** + * Without passing the information of the levels to the constructor, we return the Node-level aggregated stats as + * {@link CommonStats} along with a hash-map containing Index to List of Shard Stats. + */ public NodeIndicesStats(CommonStats oldStats, Map> statsByShard, SearchRequestStats searchRequestStats) { // this.stats = stats; this.statsByShard = statsByShard; @@ -112,6 +116,90 @@ public NodeIndicesStats(CommonStats oldStats, Map> } } + /** + * Passing the level information to the nodes allows us to aggregate the stats based on the level passed. This + * allows us to aggregate based on NodeLevel (default - if no level is passed) or Index level if `indices` level is + * passed and finally return the statsByShards map if `shards` level is passed. This allows us to reduce ser/de of + * stats and return only the information that is required while returning to the client. + */ + public NodeIndicesStats( + CommonStats oldStats, + Map> statsByShard, + SearchRequestStats searchRequestStats, + StatsLevel level + ) { + // make a total common stats from old ones and current ones + this.stats = oldStats; + for (List shardStatsList : statsByShard.values()) { + for (IndexShardStats indexShardStats : shardStatsList) { + for (ShardStats shardStats : indexShardStats.getShards()) { + stats.add(shardStats.getStats()); + } + } + } + + if (this.stats.search != null) { + this.stats.search.setSearchRequestStats(searchRequestStats); + } + + if (level != null) { + switch (level) { + case INDICES: + this.statsByIndex = createStatsByIndex(statsByShard); + break; + case SHARDS: + this.statsByShard = statsByShard; + break; + } + } + } + + /** + * By default, the levels passed from the transport action will be a list of strings, since NodeIndicesStats can + * only aggregate on one level, we pick the first accepted level else we ignore if no known level is passed. Level is + * selected based on enum defined in {@link StatsLevel} + * + * Note - we are picking the first level as multiple levels are not supported in the previous versions. + * @param levels - levels sent in the request. + * + * @return Corresponding identified enum {@link StatsLevel} + */ + public static StatsLevel getAcceptedLevel(String[] levels) { + if (levels != null && levels.length > 0) { + Optional level = Arrays.stream(StatsLevel.values()) + .filter(field -> field.getRestName().equals(levels[0])) + .findFirst(); + return level.orElseThrow(() -> new IllegalArgumentException("Level provided is not supported by NodeIndicesStats")); + } + return null; + } + + private Map readStatsByIndex(StreamInput in) throws IOException { + Map statsByIndex = new HashMap<>(); + int indexEntries = in.readVInt(); + for (int i = 0; i < indexEntries; i++) { + Index index = new Index(in); + CommonStats commonStats = new CommonStats(in); + statsByIndex.put(index, commonStats); + } + return statsByIndex; + } + + private Map> readStatsByShard(StreamInput in) throws IOException { + Map> statsByShard = new HashMap<>(); + int entries = in.readVInt(); + for (int i = 0; i < entries; i++) { + Index index = new Index(in); + int indexShardListSize = in.readVInt(); + List indexShardStats = new ArrayList<>(indexShardListSize); + for (int j = 0; j < indexShardListSize; j++) { + indexShardStats.add(new IndexShardStats(in)); + } + statsByShard.put(index, indexShardStats); + } + return statsByShard; + } + @Nullable public StoreStats getStore() { return stats.getStore(); @@ -195,7 +283,31 @@ public RecoveryStats getRecoveryStats() { @Override public void writeTo(StreamOutput out) throws IOException { stats.writeTo(out); + + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeBoolean(statsByIndex != null); + if (statsByIndex != null) { + writeStatsByIndex(out); + } + } + out.writeBoolean(statsByShard != null); + if (statsByShard != null) { + writeStatsByShards(out); + } + } + + private void writeStatsByIndex(StreamOutput out) throws IOException { + if (statsByIndex != null) { + out.writeVInt(statsByIndex.size()); + for (Map.Entry entry : statsByIndex.entrySet()) { + entry.getKey().writeTo(out); + entry.getValue().writeTo(out); + } + } + } + + private void writeStatsByShards(StreamOutput out) throws IOException { if (statsByShard != null) { out.writeVInt(statsByShard.size()); for (Map.Entry> entry : statsByShard.entrySet()) { @@ -210,29 +322,46 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - final String level = params.param("level", "node"); - final boolean isLevelValid = "indices".equalsIgnoreCase(level) - || "node".equalsIgnoreCase(level) - || "shards".equalsIgnoreCase(level); + final String level = params.param("level", StatsLevel.NODE.getRestName()); + final boolean isLevelValid = StatsLevel.NODE.getRestName().equalsIgnoreCase(level) + || StatsLevel.INDICES.getRestName().equalsIgnoreCase(level) + || StatsLevel.SHARDS.getRestName().equalsIgnoreCase(level); if (!isLevelValid) { - throw new IllegalArgumentException("level parameter must be one of [indices] or [node] or [shards] but was [" + level + "]"); + throw new IllegalArgumentException( + "level parameter must be one of [" + + StatsLevel.INDICES.getRestName() + + "] or [" + + StatsLevel.NODE.getRestName() + + "] or [" + + StatsLevel.SHARDS.getRestName() + + "] but was [" + + level + + "]" + ); } // "node" level - builder.startObject(Fields.INDICES); + builder.startObject(StatsLevel.INDICES.getRestName()); stats.toXContent(builder, params); - if ("indices".equals(level)) { - Map indexStats = createStatsByIndex(); - builder.startObject(Fields.INDICES); - for (Map.Entry entry : indexStats.entrySet()) { + if (StatsLevel.INDICES.getRestName().equals(level)) { + assert statsByIndex != null || statsByShard != null : "Expected shard stats or index stats in response for generating [" + + StatsLevel.INDICES + + "] field"; + if (statsByIndex == null) { + statsByIndex = createStatsByIndex(statsByShard); + } + + builder.startObject(StatsLevel.INDICES.getRestName()); + for (Map.Entry entry : statsByIndex.entrySet()) { builder.startObject(entry.getKey().getName()); entry.getValue().toXContent(builder, params); builder.endObject(); } builder.endObject(); - } else if ("shards".equals(level)) { - builder.startObject("shards"); + } else if (StatsLevel.SHARDS.getRestName().equals(level)) { + builder.startObject(StatsLevel.SHARDS.getRestName()); + assert statsByShard != null : "Expected shard stats in response for generating [" + StatsLevel.SHARDS + "] field"; for (Map.Entry> entry : statsByShard.entrySet()) { builder.startArray(entry.getKey().getName()); for (IndexShardStats indexShardStats : entry.getValue()) { @@ -251,7 +380,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - private Map createStatsByIndex() { + private Map createStatsByIndex(Map> statsByShard) { Map statsMap = new HashMap<>(); for (Map.Entry> entry : statsByShard.entrySet()) { if (!statsMap.containsKey(entry.getKey())) { @@ -281,7 +410,21 @@ public List getShardStats(Index index) { * * @opensearch.internal */ - static final class Fields { - static final String INDICES = "indices"; + @PublicApi(since = "3.0.0") + public enum StatsLevel { + INDICES("indices"), + SHARDS("shards"), + NODE("node"); + + private final String restName; + + StatsLevel(String restName) { + this.restName = restName; + } + + public String getRestName() { + return restName; + } + } } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index ed9c0b171aa56..0119731e4a0d7 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -233,6 +233,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC String[] levels = Strings.splitStringByCommaToArray(request.param("level")); nodesStatsRequest.indices().setLevels(levels); nodesStatsRequest.setIncludeDiscoveryNodes(false); + nodesStatsRequest.indices().setIncludeIndicesStatsByLevel(true); return channel -> client.admin().cluster().nodesStats(nodesStatsRequest, new NodesResponseRestListener<>(channel)); } diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java index 0330fe627ccd0..1aa40b50290cd 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java @@ -148,6 +148,7 @@ public void processResponse(final NodesInfoResponse nodesInfoResponse) { NodesStatsRequest.Metric.PROCESS.metricName(), NodesStatsRequest.Metric.SCRIPT.metricName() ); + nodesStatsRequest.indices().setIncludeIndicesStatsByLevel(true); client.admin().cluster().nodesStats(nodesStatsRequest, new RestResponseListener(channel) { @Override public RestResponse buildResponse(NodesStatsResponse nodesStatsResponse) throws Exception { diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index f7bc96bdfe769..a0225a0bf6193 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -32,13 +32,19 @@ package org.opensearch.action.admin.cluster.node.stats; +import org.opensearch.Version; import org.opensearch.action.admin.indices.stats.CommonStats; import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.action.admin.indices.stats.IndexShardStats; +import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.action.search.SearchRequestStats; import org.opensearch.cluster.coordination.PendingClusterStateStats; import org.opensearch.cluster.coordination.PersistedStateStats; import org.opensearch.cluster.coordination.PublishClusterStateStats; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.cluster.service.ClusterStateStats; @@ -52,17 +58,31 @@ import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.indices.breaker.AllCircuitBreakerStats; import org.opensearch.core.indices.breaker.CircuitBreakerStats; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.discovery.DiscoveryStats; import org.opensearch.gateway.remote.RemotePersistenceStats; import org.opensearch.http.HttpStats; import org.opensearch.index.ReplicationStats; import org.opensearch.index.SegmentReplicationRejectionStats; +import org.opensearch.index.cache.query.QueryCacheStats; +import org.opensearch.index.engine.SegmentsStats; +import org.opensearch.index.fielddata.FieldDataStats; +import org.opensearch.index.flush.FlushStats; import org.opensearch.index.remote.RemoteSegmentStats; import org.opensearch.index.remote.RemoteTranslogTransferTracker; +import org.opensearch.index.shard.DocsStats; +import org.opensearch.index.shard.IndexingStats; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.StoreStats; import org.opensearch.index.translog.RemoteTranslogStats; import org.opensearch.indices.NodeIndicesStats; import org.opensearch.ingest.IngestStats; @@ -82,17 +102,20 @@ import org.opensearch.ratelimitting.admissioncontrol.stats.AdmissionControllerStats; import org.opensearch.script.ScriptCacheStats; import org.opensearch.script.ScriptStats; +import org.opensearch.search.suggest.completion.CompletionStats; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; import org.opensearch.threadpool.ThreadPoolStats; import org.opensearch.transport.TransportStats; import java.io.IOException; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -1065,4 +1088,381 @@ private static RemoteTranslogTransferTracker.Stats getRandomRemoteTranslogTransf private OperationStats getPipelineStats(List pipelineStats, String id) { return pipelineStats.stream().filter(p1 -> p1.getPipelineId().equals(id)).findFirst().map(p2 -> p2.getStats()).orElse(null); } + + public static class MockNodeIndicesStats extends NodeIndicesStats { + + public MockNodeIndicesStats(StreamInput in) throws IOException { + super(in); + } + + public MockNodeIndicesStats( + CommonStats oldStats, + Map> statsByShard, + SearchRequestStats searchRequestStats + ) { + super(oldStats, statsByShard, searchRequestStats); + } + + public MockNodeIndicesStats( + CommonStats oldStats, + Map> statsByShard, + SearchRequestStats searchRequestStats, + StatsLevel level + ) { + super(oldStats, statsByShard, searchRequestStats, level); + } + + public CommonStats getStats() { + return this.stats; + } + + public Map getStatsByIndex() { + return this.statsByIndex; + } + + public Map> getStatsByShard() { + return this.statsByShard; + } + } + + public void testOldVersionNodes() throws IOException { + long numDocs = randomLongBetween(0, 10000); + long numDeletedDocs = randomLongBetween(0, 100); + CommonStats commonStats = new CommonStats(CommonStatsFlags.NONE); + + commonStats.docs = new DocsStats(numDocs, numDeletedDocs, 0); + commonStats.store = new StoreStats(100, 0L); + commonStats.indexing = new IndexingStats(); + DocsStats hostDocStats = new DocsStats(numDocs, numDeletedDocs, 0); + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.clear(); + commonStatsFlags.set(CommonStatsFlags.Flag.Docs, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Store, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Indexing, true); + + Index newIndex = new Index("index", "_na_"); + + MockNodeIndicesStats mockNodeIndicesStats = generateMockNodeIndicesStats(commonStats, newIndex, commonStatsFlags, null); + + // To test out scenario when the incoming node stats response is from a node with an older ES Version. + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.setVersion(Version.V_2_13_0); + mockNodeIndicesStats.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + in.setVersion(Version.V_2_13_0); + MockNodeIndicesStats newNodeIndicesStats = new MockNodeIndicesStats(in); + + List incomingIndexStats = newNodeIndicesStats.getStatsByShard().get(newIndex); + incomingIndexStats.forEach(indexShardStats -> { + ShardStats shardStats = Arrays.stream(indexShardStats.getShards()).findFirst().get(); + DocsStats incomingDocStats = shardStats.getStats().docs; + + assertEquals(incomingDocStats.getCount(), hostDocStats.getCount()); + assertEquals(incomingDocStats.getTotalSizeInBytes(), hostDocStats.getTotalSizeInBytes()); + assertEquals(incomingDocStats.getAverageSizeInBytes(), hostDocStats.getAverageSizeInBytes()); + assertEquals(incomingDocStats.getDeleted(), hostDocStats.getDeleted()); + }); + } + } + } + + public void testNodeIndicesStatsSerialization() throws IOException { + long numDocs = randomLongBetween(0, 10000); + long numDeletedDocs = randomLongBetween(0, 100); + List levelParams = new ArrayList<>(); + levelParams.add(NodeIndicesStats.StatsLevel.INDICES); + levelParams.add(NodeIndicesStats.StatsLevel.SHARDS); + levelParams.add(NodeIndicesStats.StatsLevel.NODE); + CommonStats commonStats = new CommonStats(CommonStatsFlags.NONE); + + commonStats.docs = new DocsStats(numDocs, numDeletedDocs, 0); + commonStats.store = new StoreStats(100, 0L); + commonStats.indexing = new IndexingStats(); + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.clear(); + commonStatsFlags.set(CommonStatsFlags.Flag.Docs, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Store, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Indexing, true); + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + + levelParams.forEach(level -> { + Index newIndex = new Index("index", "_na_"); + + MockNodeIndicesStats mockNodeIndicesStats = generateMockNodeIndicesStats(commonStats, newIndex, commonStatsFlags, level); + + // To test out scenario when the incoming node stats response is from a node with an older ES Version. + try (BytesStreamOutput out = new BytesStreamOutput()) { + mockNodeIndicesStats.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + MockNodeIndicesStats newNodeIndicesStats = new MockNodeIndicesStats(in); + switch (level) { + case NODE: + assertNull(newNodeIndicesStats.getStatsByIndex()); + assertNull(newNodeIndicesStats.getStatsByShard()); + break; + case INDICES: + assertNull(newNodeIndicesStats.getStatsByShard()); + assertNotNull(newNodeIndicesStats.getStatsByIndex()); + break; + case SHARDS: + assertNull(newNodeIndicesStats.getStatsByIndex()); + assertNotNull(newNodeIndicesStats.getStatsByShard()); + break; + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + public void testNodeIndicesStatsToXContent() { + long numDocs = randomLongBetween(0, 10000); + long numDeletedDocs = randomLongBetween(0, 100); + List levelParams = new ArrayList<>(); + levelParams.add(NodeIndicesStats.StatsLevel.INDICES); + levelParams.add(NodeIndicesStats.StatsLevel.SHARDS); + levelParams.add(NodeIndicesStats.StatsLevel.NODE); + CommonStats commonStats = new CommonStats(CommonStatsFlags.NONE); + + commonStats.docs = new DocsStats(numDocs, numDeletedDocs, 0); + commonStats.store = new StoreStats(100, 0L); + commonStats.indexing = new IndexingStats(); + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags(); + commonStatsFlags.clear(); + commonStatsFlags.set(CommonStatsFlags.Flag.Docs, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Store, true); + commonStatsFlags.set(CommonStatsFlags.Flag.Indexing, true); + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + + levelParams.forEach(level -> { + + Index newIndex = new Index("index", "_na_"); + + MockNodeIndicesStats mockNodeIndicesStats = generateMockNodeIndicesStats(commonStats, newIndex, commonStatsFlags, level); + + XContentBuilder builder = null; + try { + builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder = mockNodeIndicesStats.toXContent( + builder, + new ToXContent.MapParams(Collections.singletonMap("level", level.getRestName())) + ); + builder.endObject(); + + Map xContentMap = xContentBuilderToMap(builder); + LinkedHashMap indicesStatsMap = (LinkedHashMap) xContentMap.get(NodeIndicesStats.StatsLevel.INDICES.getRestName()); + + switch (level) { + case NODE: + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES.getRestName())); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS.getRestName())); + break; + case INDICES: + assertTrue(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES.getRestName())); + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS.getRestName())); + break; + case SHARDS: + assertFalse(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.INDICES.getRestName())); + assertTrue(indicesStatsMap.containsKey(NodeIndicesStats.StatsLevel.SHARDS.getRestName())); + break; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + }); + } + + public void testNodeIndicesStatsWithAndWithoutAggregations() throws IOException { + + CommonStatsFlags commonStatsFlags = new CommonStatsFlags( + CommonStatsFlags.Flag.Docs, + CommonStatsFlags.Flag.Store, + CommonStatsFlags.Flag.Indexing, + CommonStatsFlags.Flag.Completion, + CommonStatsFlags.Flag.Flush, + CommonStatsFlags.Flag.FieldData, + CommonStatsFlags.Flag.QueryCache, + CommonStatsFlags.Flag.Segments + ); + + int numberOfIndexes = randomIntBetween(1, 3); + List indexList = new ArrayList<>(); + for (int i = 0; i < numberOfIndexes; i++) { + Index index = new Index("test-index-" + i, "_na_"); + indexList.add(index); + } + + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + HashMap> statsByShards = createRandomShardByStats(indexList); + + final MockNodeIndicesStats nonAggregatedNodeIndicesStats = new MockNodeIndicesStats( + new CommonStats(commonStatsFlags), + statsByShards, + new SearchRequestStats(clusterSettings) + ); + + commonStatsFlags.setIncludeIndicesStatsByLevel(true); + + Arrays.stream(NodeIndicesStats.StatsLevel.values()).forEach(level -> { + MockNodeIndicesStats aggregatedNodeIndicesStats = new MockNodeIndicesStats( + new CommonStats(commonStatsFlags), + statsByShards, + new SearchRequestStats(clusterSettings), + level + ); + + XContentBuilder nonAggregatedBuilder = null; + XContentBuilder aggregatedBuilder = null; + try { + nonAggregatedBuilder = XContentFactory.jsonBuilder(); + nonAggregatedBuilder.startObject(); + nonAggregatedBuilder = nonAggregatedNodeIndicesStats.toXContent( + nonAggregatedBuilder, + new ToXContent.MapParams(Collections.singletonMap("level", level.getRestName())) + ); + nonAggregatedBuilder.endObject(); + Map nonAggregatedContentMap = xContentBuilderToMap(nonAggregatedBuilder); + + aggregatedBuilder = XContentFactory.jsonBuilder(); + aggregatedBuilder.startObject(); + aggregatedBuilder = aggregatedNodeIndicesStats.toXContent( + aggregatedBuilder, + new ToXContent.MapParams(Collections.singletonMap("level", level.getRestName())) + ); + aggregatedBuilder.endObject(); + Map aggregatedContentMap = xContentBuilderToMap(aggregatedBuilder); + + assertEquals(aggregatedContentMap, nonAggregatedContentMap); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private CommonStats createRandomCommonStats() { + CommonStats commonStats = new CommonStats(CommonStatsFlags.NONE); + commonStats.docs = new DocsStats(randomLongBetween(0, 10000), randomLongBetween(0, 100), randomLongBetween(0, 1000)); + commonStats.store = new StoreStats(randomLongBetween(0, 100), randomLongBetween(0, 1000)); + commonStats.indexing = new IndexingStats(); + commonStats.completion = new CompletionStats(); + commonStats.flush = new FlushStats(randomLongBetween(0, 100), randomLongBetween(0, 100), randomLongBetween(0, 100)); + commonStats.fieldData = new FieldDataStats(randomLongBetween(0, 100), randomLongBetween(0, 100), null); + commonStats.queryCache = new QueryCacheStats( + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100) + ); + commonStats.segments = new SegmentsStats(); + + return commonStats; + } + + private HashMap> createRandomShardByStats(List indexes) { + DiscoveryNode localNode = new DiscoveryNode("node", buildNewFakeTransportAddress(), Version.CURRENT); + HashMap> statsByShards = new HashMap<>(); + indexes.forEach(index -> { + List indexShardStatsList = new ArrayList<>(); + + int numberOfShards = randomIntBetween(1, 4); + for (int i = 0; i < numberOfShards; i++) { + ShardRoutingState shardRoutingState = ShardRoutingState.fromValue((byte) randomIntBetween(2, 3)); + + ShardRouting shardRouting = TestShardRouting.newShardRouting( + index.getName(), + i, + localNode.getId(), + randomBoolean(), + shardRoutingState + ); + + Path path = createTempDir().resolve("indices") + .resolve(shardRouting.shardId().getIndex().getUUID()) + .resolve(String.valueOf(shardRouting.shardId().id())); + + ShardStats shardStats = new ShardStats( + shardRouting, + new ShardPath(false, path, path, shardRouting.shardId()), + createRandomCommonStats(), + null, + null, + null + ); + List shardStatsList = new ArrayList<>(); + shardStatsList.add(shardStats); + IndexShardStats indexShardStats = new IndexShardStats(shardRouting.shardId(), shardStatsList.toArray(new ShardStats[0])); + indexShardStatsList.add(indexShardStats); + } + statsByShards.put(index, indexShardStatsList); + }); + + return statsByShards; + } + + private Map xContentBuilderToMap(XContentBuilder xContentBuilder) { + return XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(); + } + + public MockNodeIndicesStats generateMockNodeIndicesStats( + CommonStats commonStats, + Index index, + CommonStatsFlags commonStatsFlags, + NodeIndicesStats.StatsLevel level + ) { + DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT); + Map> statsByShard = new HashMap<>(); + List indexShardStatsList = new ArrayList<>(); + Index statsIndex = null; + for (int i = 0; i < 2; i++) { + ShardRoutingState shardRoutingState = ShardRoutingState.fromValue((byte) randomIntBetween(2, 3)); + ShardRouting shardRouting = TestShardRouting.newShardRouting( + index.getName(), + i, + localNode.getId(), + randomBoolean(), + shardRoutingState + ); + + if (statsIndex == null) { + statsIndex = shardRouting.shardId().getIndex(); + } + + Path path = createTempDir().resolve("indices") + .resolve(shardRouting.shardId().getIndex().getUUID()) + .resolve(String.valueOf(shardRouting.shardId().id())); + + ShardStats shardStats = new ShardStats( + shardRouting, + new ShardPath(false, path, path, shardRouting.shardId()), + commonStats, + null, + null, + null + ); + IndexShardStats indexShardStats = new IndexShardStats(shardRouting.shardId(), new ShardStats[] { shardStats }); + indexShardStatsList.add(indexShardStats); + } + + statsByShard.put(statsIndex, indexShardStatsList); + + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + if (commonStatsFlags.getIncludeIndicesStatsByLevel()) { + return new MockNodeIndicesStats( + new CommonStats(commonStatsFlags), + statsByShard, + new SearchRequestStats(clusterSettings), + level + ); + } else { + return new MockNodeIndicesStats(new CommonStats(commonStatsFlags), statsByShard, new SearchRequestStats(clusterSettings)); + } + } }