From 5e1f6554a22aefdecba469916030b833c44a42d6 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Tue, 13 Aug 2024 09:48:25 -0700 Subject: [PATCH 01/14] Add http request content stream support (#111438) --- .../Netty4IncrementalRequestHandlingIT.java | 436 ++++++++++++++++++ .../http/netty4/Netty4HttpAggregator.java | 46 ++ .../netty4/Netty4HttpPipeliningHandler.java | 50 +- .../http/netty4/Netty4HttpRequest.java | 34 +- .../netty4/Netty4HttpRequestBodyStream.java | 114 +++++ .../netty4/Netty4HttpServerTransport.java | 2 +- .../transport/netty4/Netty4Utils.java | 10 + .../Netty4HttpRequestBodyStreamTests.java | 122 +++++ .../Netty4HttpServerTransportTests.java | 2 +- .../java/org/elasticsearch/http/HttpBody.java | 103 +++++ .../http/HttpClientStatsTracker.java | 4 +- .../org/elasticsearch/http/HttpRequest.java | 4 +- .../elasticsearch/rest/BaseRestHandler.java | 13 +- .../elasticsearch/rest/RestController.java | 6 +- .../org/elasticsearch/rest/RestRequest.java | 19 +- .../http/HttpClientStatsTrackerTests.java | 4 +- .../elasticsearch/http/TestHttpRequest.java | 5 +- .../rest/RestControllerTests.java | 7 +- .../elasticsearch/rest/RestRequestTests.java | 3 +- .../test/rest/FakeRestRequest.java | 15 +- .../audit/logfile/LoggingAuditTrailTests.java | 2 +- 21 files changed, 953 insertions(+), 48 deletions(-) create mode 100644 modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java create mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java create mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java create mode 100644 server/src/main/java/org/elasticsearch/http/HttpBody.java diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java new file mode 100644 index 0000000000000..4de7ca97ed51b --- /dev/null +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -0,0 +1,436 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http.netty4; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.DefaultHttpRequest; +import io.netty.handler.codec.http.DefaultLastHttpContent; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpClientCodec; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.LastHttpContent; + +import org.elasticsearch.ESNetty4IntegTestCase; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.netty4.Netty4Utils; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH; +import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON; +import static io.netty.handler.codec.http.HttpMethod.POST; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; + +@ESIntegTestCase.ClusterScope(numDataNodes = 1) +public class Netty4IncrementalRequestHandlingIT extends ESNetty4IntegTestCase { + + // ensure empty http content has single 0 size chunk + public void testEmptyContent() throws Exception { + try (var ctx = setupClientCtx()) { + var totalRequests = randomIntBetween(1, 10); + for (int reqNo = 0; reqNo < totalRequests; reqNo++) { + var opaqueId = opaqueId(reqNo); + + // send request with empty content + ctx.clientChannel.writeAndFlush(fullHttpRequest(opaqueId, Unpooled.EMPTY_BUFFER)); + var handler = ctx.awaitRestChannelAccepted(opaqueId); + handler.stream.next(); + + // should receive a single empty chunk + var recvChunk = safePoll(handler.recvChunks); + assertTrue(recvChunk.isLast); + assertEquals(0, recvChunk.chunk.length()); + recvChunk.chunk.close(); + + // send response to process following request + handler.sendResponse(new RestResponse(RestStatus.OK, "")); + } + assertBusy(() -> assertEquals("should receive all server responses", totalRequests, ctx.clientRespQueue.size())); + } + } + + // ensures content integrity, no loses and re-order + public void testReceiveAllChunks() throws Exception { + try (var ctx = setupClientCtx()) { + var totalRequests = randomIntBetween(1, 10); + for (int reqNo = 0; reqNo < totalRequests; reqNo++) { + var opaqueId = opaqueId(reqNo); + + // this dataset will be compared with one on server side + var dataSize = randomIntBetween(1024, 10 * 1024 * 1024); + var sendData = Unpooled.wrappedBuffer(randomByteArrayOfLength(dataSize)); + sendData.retain(); + ctx.clientChannel.writeAndFlush(fullHttpRequest(opaqueId, sendData)); + + var handler = ctx.awaitRestChannelAccepted(opaqueId); + + var recvData = Unpooled.buffer(dataSize); + while (true) { + handler.stream.next(); + var recvChunk = safePoll(handler.recvChunks); + try (recvChunk.chunk) { + recvData.writeBytes(Netty4Utils.toByteBuf(recvChunk.chunk)); + if (recvChunk.isLast) { + break; + } + } + } + + assertEquals("sent and received payloads are not the same", sendData, recvData); + handler.sendResponse(new RestResponse(RestStatus.OK, "")); + } + assertBusy(() -> assertEquals("should receive all server responses", totalRequests, ctx.clientRespQueue.size())); + } + } + + // ensures that all queued chunks are released when connection closed + public void testClientConnectionCloseMidStream() throws Exception { + try (var ctx = setupClientCtx()) { + var opaqueId = opaqueId(0); + + // write half of http request + ctx.clientChannel.write(httpRequest(opaqueId, 2 * 1024)); + ctx.clientChannel.writeAndFlush(randomContent(1024, false)); + + // await stream handler is ready and request full content + var handler = ctx.awaitRestChannelAccepted(opaqueId); + assertBusy(() -> assertEquals(1, handler.stream.chunkQueue().size())); + + // enable auto-read to receive channel close event + handler.stream.channel().config().setAutoRead(true); + + // terminate connection and wait resources are released + ctx.clientChannel.close(); + assertBusy(() -> assertEquals(0, handler.stream.chunkQueue().size())); + } + } + + // ensures that all queued chunks are released when server decides to close connection + public void testServerCloseConnectionMidStream() throws Exception { + try (var ctx = setupClientCtx()) { + var opaqueId = opaqueId(0); + + // write half of http request + ctx.clientChannel.write(httpRequest(opaqueId, 2 * 1024)); + ctx.clientChannel.writeAndFlush(randomContent(1024, false)); + + // await stream handler is ready and request full content + var handler = ctx.awaitRestChannelAccepted(opaqueId); + assertBusy(() -> assertEquals(1, handler.stream.chunkQueue().size())); + + // terminate connection on server and wait resources are released + handler.channel.request().getHttpChannel().close(); + assertBusy(() -> assertEquals(0, handler.stream.chunkQueue().size())); + } + } + + // ensure that client's socket buffers data when server is not consuming data + public void testClientBackpressure() throws Exception { + try (var ctx = setupClientCtx()) { + var opaqueId = opaqueId(0); + var payloadSize = MBytes(50); + ctx.clientChannel.writeAndFlush(httpRequest(opaqueId, payloadSize)); + for (int i = 0; i < 5; i++) { + ctx.clientChannel.writeAndFlush(randomContent(MBytes(10), false)); + } + assertFalse( + "should not flush last content immediately", + ctx.clientChannel.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT).isDone() + ); + + var handler = ctx.awaitRestChannelAccepted(opaqueId); + + // Read buffers for socket and channel usually within few MBytes range all together. + // This test assumes that buffers will not exceed 10 MBytes, in other words there should + // be less than 10 MBytes in fly between http client's socket and rest handler. This + // loop ensures that reading 10 MBytes of content on server side should free almost + // same size in client's channel write buffer. + for (int mb = 0; mb <= 50; mb += 10) { + var minBufSize = payloadSize - MBytes(10 + mb); + var maxBufSize = payloadSize - MBytes(mb); + // it is hard to tell that client's channel is no logger flushing data + // it might take a few busy-iterations before channel buffer flush to kernel + // and bytesBeforeWritable will stop changing + assertBusy(() -> { + var bufSize = ctx.clientChannel.bytesBeforeWritable(); + assertTrue( + "client's channel buffer should be in range [" + minBufSize + "," + maxBufSize + "], got " + bufSize, + bufSize >= minBufSize && bufSize <= maxBufSize + ); + }); + handler.consumeBytes(MBytes(10)); + } + assertTrue(handler.stream.hasLast()); + } + } + + private String opaqueId(int reqNo) { + return getTestName() + "-" + reqNo; + } + + static int MBytes(int m) { + return m * 1024 * 1024; + } + + static T safePoll(BlockingDeque queue) { + try { + var t = queue.poll(SAFE_AWAIT_TIMEOUT.seconds(), TimeUnit.SECONDS); + assertNotNull("queue is empty", t); + return t; + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new AssertionError(e); + } + } + + static FullHttpRequest fullHttpRequest(String opaqueId, ByteBuf content) { + var req = new DefaultFullHttpRequest(HTTP_1_1, POST, ControlServerRequestPlugin.ROUTE, Unpooled.wrappedBuffer(content)); + req.headers().add(CONTENT_LENGTH, content.readableBytes()); + req.headers().add(CONTENT_TYPE, APPLICATION_JSON); + req.headers().add(Task.X_OPAQUE_ID_HTTP_HEADER, opaqueId); + return req; + } + + static HttpRequest httpRequest(String opaqueId, int contentLength) { + return httpRequest(ControlServerRequestPlugin.ROUTE, opaqueId, contentLength); + } + + static HttpRequest httpRequest(String uri, String opaqueId, int contentLength) { + var req = new DefaultHttpRequest(HTTP_1_1, POST, uri); + req.headers().add(CONTENT_LENGTH, contentLength); + req.headers().add(CONTENT_TYPE, APPLICATION_JSON); + req.headers().add(Task.X_OPAQUE_ID_HTTP_HEADER, opaqueId); + return req; + } + + static HttpContent randomContent(int size, boolean isLast) { + var buf = Unpooled.wrappedBuffer(randomByteArrayOfLength(size)); + if (isLast) { + return new DefaultLastHttpContent(buf); + } else { + return new DefaultHttpContent(buf); + } + } + + Ctx setupClientCtx() throws Exception { + var nodeName = internalCluster().getRandomNodeName(); + var clientRespQueue = new LinkedBlockingDeque<>(16); + var bootstrap = bootstrapClient(nodeName, clientRespQueue); + var channel = bootstrap.connect().sync().channel(); + return new Ctx(getTestName(), nodeName, bootstrap, channel, clientRespQueue); + } + + Bootstrap bootstrapClient(String node, BlockingQueue queue) { + var httpServer = internalCluster().getInstance(HttpServerTransport.class, node); + var remoteAddr = randomFrom(httpServer.boundAddress().boundAddresses()); + return new Bootstrap().group(new NioEventLoopGroup(1)) + .channel(NioSocketChannel.class) + .remoteAddress(remoteAddr.getAddress(), remoteAddr.getPort()) + .handler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) { + var p = ch.pipeline(); + p.addLast(new HttpClientCodec()); + p.addLast(new HttpObjectAggregator(4096)); + p.addLast(new SimpleChannelInboundHandler() { + @Override + protected void channelRead0(ChannelHandlerContext ctx, FullHttpResponse msg) { + msg.retain(); + queue.add(msg); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + queue.add(cause); + } + }); + } + }); + } + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.concatLists(List.of(ControlServerRequestPlugin.class), super.nodePlugins()); + } + + @Override + protected boolean addMockHttpTransport() { + return false; // enable http + } + + record Ctx(String testName, String nodeName, Bootstrap clientBootstrap, Channel clientChannel, BlockingDeque clientRespQueue) + implements + AutoCloseable { + + @Override + public void close() throws Exception { + safeGet(clientChannel.close()); + safeGet(clientBootstrap.config().group().shutdownGracefully()); + clientRespQueue.forEach(o -> { if (o instanceof FullHttpResponse resp) resp.release(); }); + for (var opaqueId : ControlServerRequestPlugin.handlers.keySet()) { + if (opaqueId.startsWith(testName)) { + var handler = ControlServerRequestPlugin.handlers.get(opaqueId); + handler.recvChunks.forEach(c -> c.chunk.close()); + handler.channel.request().getHttpChannel().close(); + ControlServerRequestPlugin.handlers.remove(opaqueId); + } + } + } + + ServerRequestHandler awaitRestChannelAccepted(String opaqueId) throws Exception { + assertBusy(() -> assertTrue(ControlServerRequestPlugin.handlers.containsKey(opaqueId))); + var handler = ControlServerRequestPlugin.handlers.get(opaqueId); + safeAwait(handler.channelAccepted); + return handler; + } + } + + static class ServerRequestHandler implements BaseRestHandler.RequestBodyChunkConsumer { + final SubscribableListener channelAccepted = new SubscribableListener<>(); + final String opaqueId; + final BlockingDeque recvChunks = new LinkedBlockingDeque<>(); + final Netty4HttpRequestBodyStream stream; + RestChannel channel; + + boolean recvLast = false; + + ServerRequestHandler(String opaqueId, Netty4HttpRequestBodyStream stream) { + this.opaqueId = opaqueId; + this.stream = stream; + } + + @Override + public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boolean isLast) { + recvChunks.add(new Chunk(chunk, isLast)); + } + + @Override + public void accept(RestChannel channel) throws Exception { + this.channel = channel; + channelAccepted.onResponse(null); + } + + void sendResponse(RestResponse response) { + channel.sendResponse(response); + } + + void consumeBytes(int bytes) { + if (recvLast) { + return; + } + while (bytes > 0) { + stream.next(); + var recvChunk = safePoll(recvChunks); + bytes -= recvChunk.chunk.length(); + recvChunk.chunk.close(); + if (recvChunk.isLast) { + recvLast = true; + break; + } + } + } + + Future onChannelThread(Callable task) { + return this.stream.channel().eventLoop().submit(task); + } + + record Chunk(ReleasableBytesReference chunk, boolean isLast) {} + } + + // takes full control of rest handler from the outside + public static class ControlServerRequestPlugin extends Plugin implements ActionPlugin { + + static final String ROUTE = "/_test/request-stream"; + + static final ConcurrentHashMap handlers = new ConcurrentHashMap<>(); + + @Override + public Collection getRestHandlers( + Settings settings, + NamedWriteableRegistry namedWriteableRegistry, + RestController restController, + ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, + SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster, + Predicate clusterSupportsFeature + ) { + return List.of(new BaseRestHandler() { + @Override + public String getName() { + return ROUTE; + } + + @Override + public List routes() { + return List.of(new Route(RestRequest.Method.POST, ROUTE)); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) { + var stream = (Netty4HttpRequestBodyStream) request.contentStream(); + var opaqueId = request.getHeaders().get(Task.X_OPAQUE_ID_HTTP_HEADER).get(0); + var handler = new ServerRequestHandler(opaqueId, stream); + handlers.put(opaqueId, handler); + return handler; + } + }); + } + } + +} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java new file mode 100644 index 0000000000000..16f1c2bbd2e37 --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java @@ -0,0 +1,46 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http.netty4; + +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; + +import org.elasticsearch.http.HttpPreRequest; +import org.elasticsearch.http.netty4.internal.HttpHeadersAuthenticatorUtils; + +import java.util.function.Predicate; + +public class Netty4HttpAggregator extends HttpObjectAggregator { + private static final Predicate IGNORE_TEST = (req) -> req.uri().startsWith("/_test/request-stream") == false; + + private final Predicate decider; + private boolean shouldAggregate; + + public Netty4HttpAggregator(int maxContentLength) { + this(maxContentLength, IGNORE_TEST); + } + + public Netty4HttpAggregator(int maxContentLength, Predicate decider) { + super(maxContentLength); + this.decider = decider; + } + + @Override + public boolean acceptInboundMessage(Object msg) throws Exception { + if (msg instanceof HttpRequest request) { + var preReq = HttpHeadersAuthenticatorUtils.asHttpPreRequest(request); + shouldAggregate = decider.test(preReq); + } + if (shouldAggregate) { + return super.acceptInboundMessage(msg); + } else { + return false; + } + } +} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpPipeliningHandler.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpPipeliningHandler.java index d8eadf4fca95d..b08c93a4dc240 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpPipeliningHandler.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpPipeliningHandler.java @@ -20,8 +20,11 @@ import io.netty.handler.codec.http.DefaultHttpResponse; import io.netty.handler.codec.http.DefaultLastHttpContent; import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpObject; +import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.ssl.SslCloseCompletionEvent; import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; @@ -71,6 +74,12 @@ private record ChunkedWrite(PromiseCombiner combiner, ChannelPromise onDone, Chu @Nullable private ChunkedWrite currentChunkedWrite; + /** + * HTTP request content stream for current request, it's null if there is no current request or request is fully-aggregated + */ + @Nullable + private Netty4HttpRequestBodyStream currentRequestStream; + /* * The current read and write sequence numbers. Read sequence numbers are attached to requests in the order they are read from the * channel, and then transferred to responses. A response is not written to the channel context until its sequence number matches the @@ -110,23 +119,38 @@ public Netty4HttpPipeliningHandler( public void channelRead(final ChannelHandlerContext ctx, final Object msg) { activityTracker.startActivity(); try { - assert msg instanceof FullHttpRequest : "Should have fully aggregated message already but saw [" + msg + "]"; - final FullHttpRequest fullHttpRequest = (FullHttpRequest) msg; - final Netty4HttpRequest netty4HttpRequest; - if (fullHttpRequest.decoderResult().isFailure()) { - final Throwable cause = fullHttpRequest.decoderResult().cause(); - final Exception nonError; - if (cause instanceof Error) { - ExceptionsHelper.maybeDieOnAnotherThread(cause); - nonError = new Exception(cause); + if (msg instanceof HttpRequest request) { + final Netty4HttpRequest netty4HttpRequest; + if (request.decoderResult().isFailure()) { + final Throwable cause = request.decoderResult().cause(); + final Exception nonError; + if (cause instanceof Error) { + ExceptionsHelper.maybeDieOnAnotherThread(cause); + nonError = new Exception(cause); + } else { + nonError = (Exception) cause; + } + netty4HttpRequest = new Netty4HttpRequest(readSequence++, (FullHttpRequest) request, nonError); } else { - nonError = (Exception) cause; + assert currentRequestStream == null : "current stream must be null for new request"; + if (request instanceof FullHttpRequest fullHttpRequest) { + netty4HttpRequest = new Netty4HttpRequest(readSequence++, fullHttpRequest); + currentRequestStream = null; + } else { + var contentStream = new Netty4HttpRequestBodyStream(ctx.channel()); + currentRequestStream = contentStream; + netty4HttpRequest = new Netty4HttpRequest(readSequence++, request, contentStream); + } } - netty4HttpRequest = new Netty4HttpRequest(readSequence++, fullHttpRequest, nonError); + handlePipelinedRequest(ctx, netty4HttpRequest); } else { - netty4HttpRequest = new Netty4HttpRequest(readSequence++, fullHttpRequest); + assert msg instanceof HttpContent : "expect HttpContent got " + msg; + assert currentRequestStream != null : "current stream must exists before handling http content"; + currentRequestStream.handleNettyContent((HttpContent) msg); + if (msg instanceof LastHttpContent) { + currentRequestStream = null; + } } - handlePipelinedRequest(ctx, netty4HttpRequest); } finally { activityTracker.stopActivity(); } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java index fa4babea21555..e6b9841e3731d 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java @@ -12,6 +12,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.EmptyHttpHeaders; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpHeaders; @@ -21,6 +22,7 @@ import io.netty.handler.codec.http.cookie.ServerCookieEncoder; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.http.HttpResponse; import org.elasticsearch.rest.ChunkedRestResponseBodyPart; @@ -40,22 +42,40 @@ public class Netty4HttpRequest implements HttpRequest { private final FullHttpRequest request; - private final BytesReference content; + private final HttpBody content; private final Map> headers; private final AtomicBoolean released; private final Exception inboundException; private final boolean pooled; private final int sequence; + Netty4HttpRequest(int sequence, io.netty.handler.codec.http.HttpRequest request, Netty4HttpRequestBodyStream contentStream) { + this( + sequence, + new DefaultFullHttpRequest( + request.protocolVersion(), + request.method(), + request.uri(), + Unpooled.EMPTY_BUFFER, + request.headers(), + EmptyHttpHeaders.INSTANCE + ), + new AtomicBoolean(false), + false, + contentStream, + null + ); + } + Netty4HttpRequest(int sequence, FullHttpRequest request) { - this(sequence, request, new AtomicBoolean(false), true, Netty4Utils.toBytesReference(request.content())); + this(sequence, request, new AtomicBoolean(false), true, Netty4Utils.fullHttpBodyFrom(request.content())); } Netty4HttpRequest(int sequence, FullHttpRequest request, Exception inboundException) { - this(sequence, request, new AtomicBoolean(false), true, Netty4Utils.toBytesReference(request.content()), inboundException); + this(sequence, request, new AtomicBoolean(false), true, Netty4Utils.fullHttpBodyFrom(request.content()), inboundException); } - private Netty4HttpRequest(int sequence, FullHttpRequest request, AtomicBoolean released, boolean pooled, BytesReference content) { + private Netty4HttpRequest(int sequence, FullHttpRequest request, AtomicBoolean released, boolean pooled, HttpBody content) { this(sequence, request, released, pooled, content, null); } @@ -64,7 +84,7 @@ private Netty4HttpRequest( FullHttpRequest request, AtomicBoolean released, boolean pooled, - BytesReference content, + HttpBody content, Exception inboundException ) { this.sequence = sequence; @@ -87,7 +107,7 @@ public String uri() { } @Override - public BytesReference content() { + public HttpBody body() { assert released.get() == false; return content; } @@ -119,7 +139,7 @@ public HttpRequest releaseAndCopy() { ), new AtomicBoolean(false), false, - Netty4Utils.toBytesReference(copiedContent) + content ); } finally { release(); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java new file mode 100644 index 0000000000000..8497e3ee8a40d --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -0,0 +1,114 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http.netty4; + +import io.netty.channel.Channel; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.LastHttpContent; + +import org.elasticsearch.http.HttpBody; +import org.elasticsearch.transport.netty4.Netty4Utils; + +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * Netty based implementation of {@link HttpBody.Stream}. + * This implementation utilize {@link io.netty.channel.ChannelConfig#setAutoRead(boolean)} + * to prevent entire payload buffering. But sometimes upstream can send few chunks of data despite + * autoRead=off. In this case chunks will be queued until downstream calls {@link Stream#next()} + */ +public class Netty4HttpRequestBodyStream implements HttpBody.Stream { + + private final Channel channel; + private final Queue chunkQueue = new ArrayDeque<>(); + private boolean requested = false; + private boolean hasLast = false; + private HttpBody.ChunkHandler handler; + + public Netty4HttpRequestBodyStream(Channel channel) { + this.channel = channel; + channel.closeFuture().addListener((f) -> releaseQueuedChunks()); + channel.config().setAutoRead(false); + } + + @Override + public ChunkHandler handler() { + return handler; + } + + @Override + public void setHandler(ChunkHandler chunkHandler) { + this.handler = chunkHandler; + } + + private void sendQueuedOrRead() { + assert channel.eventLoop().inEventLoop(); + requested = true; + var chunk = chunkQueue.poll(); + if (chunk == null) { + channel.read(); + } else { + sendChunk(chunk); + } + } + + @Override + public void next() { + assert handler != null : "handler must be set before requesting next chunk"; + if (channel.eventLoop().inEventLoop()) { + sendQueuedOrRead(); + } else { + channel.eventLoop().submit(this::sendQueuedOrRead); + } + } + + public void handleNettyContent(HttpContent httpContent) { + assert handler != null : "handler must be set before processing http content"; + if (requested && chunkQueue.isEmpty()) { + sendChunk(httpContent); + } else { + chunkQueue.add(httpContent); + } + if (httpContent instanceof LastHttpContent) { + hasLast = true; + channel.config().setAutoRead(true); + } + } + + // visible for test + Channel channel() { + return channel; + } + + // visible for test + Queue chunkQueue() { + return chunkQueue; + } + + // visible for test + boolean hasLast() { + return hasLast; + } + + private void sendChunk(HttpContent httpContent) { + assert requested; + requested = false; + var bytesRef = Netty4Utils.toReleasableBytesReference(httpContent.content()); + var isLast = httpContent instanceof LastHttpContent; + handler.onNext(bytesRef, isLast); + } + + private void releaseQueuedChunks() { + while (chunkQueue.isEmpty() == false) { + chunkQueue.poll().release(); + } + } + +} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java index f38bd1107ab33..129c3b9c3d33c 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java @@ -366,7 +366,7 @@ protected HttpMessage createMessage(String[] initialLine) throws Exception { ); } // combines the HTTP message pieces into a single full HTTP request (with headers and body) - final HttpObjectAggregator aggregator = new HttpObjectAggregator(handlingSettings.maxContentLength()); + final HttpObjectAggregator aggregator = new Netty4HttpAggregator(handlingSettings.maxContentLength()); aggregator.setMaxCumulationBufferComponents(transport.maxCompositeBufferComponents); ch.pipeline() .addLast("decoder_compress", new HttpContentDecompressor()) // this handles request body decompression diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java index b3596c75999ec..f57aa0e680fa1 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java @@ -27,11 +27,13 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.recycler.Recycler; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Booleans; import org.elasticsearch.core.SuppressForbidden; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.transport.TransportException; import java.io.IOException; @@ -128,6 +130,14 @@ public static BytesReference toBytesReference(final ByteBuf buffer) { } } + public static ReleasableBytesReference toReleasableBytesReference(final ByteBuf buffer) { + return new ReleasableBytesReference(toBytesReference(buffer), buffer::release); + } + + public static HttpBody.Full fullHttpBodyFrom(final ByteBuf buf) { + return new HttpBody.ByteRefHttpBody(toBytesReference(buf)); + } + public static Recycler createRecycler(Settings settings) { // If this method is called by super ctor the processors will not be set. Accessing NettyAllocator initializes netty's internals // setting the processors. We must do it ourselves first just in case. diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java new file mode 100644 index 0000000000000..00066ffaf0201 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java @@ -0,0 +1,122 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http.netty4; + +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.DefaultLastHttpContent; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.flow.FlowControlHandler; + +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.http.HttpBody; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class Netty4HttpRequestBodyStreamTests extends ESTestCase { + + EmbeddedChannel channel; + Netty4HttpRequestBodyStream stream; + static HttpBody.ChunkHandler discardHandler = (chunk, isLast) -> chunk.close(); + + @Before + public void createStream() { + channel = new EmbeddedChannel(); + stream = new Netty4HttpRequestBodyStream(channel); + stream.setHandler(discardHandler); // set default handler, each test might override one + channel.pipeline().addLast(new SimpleChannelInboundHandler() { + @Override + protected void channelRead0(ChannelHandlerContext ctx, HttpContent msg) { + msg.retain(); + stream.handleNettyContent(msg); + } + }); + } + + // ensures that no chunks are sent downstream without request + public void testEnqueueChunksBeforeRequest() { + var totalChunks = randomIntBetween(1, 100); + for (int i = 0; i < totalChunks; i++) { + channel.writeInbound(randomContent(1024)); + } + assertEquals(totalChunks, stream.chunkQueue().size()); + } + + // ensures all queued chunks can be flushed downstream + public void testFlushQueued() { + var chunks = new ArrayList(); + var totalBytes = new AtomicInteger(); + stream.setHandler((chunk, isLast) -> { + chunks.add(chunk); + totalBytes.addAndGet(chunk.length()); + }); + // enqueue chunks + var chunkSize = 1024; + var totalChunks = randomIntBetween(1, 100); + for (int i = 0; i < totalChunks; i++) { + channel.writeInbound(randomContent(chunkSize)); + } + // consume all chunks + for (var i = 0; i < totalChunks; i++) { + stream.next(); + } + assertEquals(totalChunks, chunks.size()); + assertEquals(chunkSize * totalChunks, totalBytes.get()); + } + + // ensures that we read from channel when chunks queue is empty + // and pass next chunk downstream without queuing + public void testReadFromChannel() { + var gotChunks = new ArrayList(); + var gotLast = new AtomicBoolean(false); + stream.setHandler((chunk, isLast) -> { + gotChunks.add(chunk); + gotLast.set(isLast); + }); + channel.pipeline().addFirst(new FlowControlHandler()); // block all incoming messages, need explicit channel.read() + var chunkSize = 1024; + var totalChunks = randomIntBetween(1, 32); + for (int i = 0; i < totalChunks - 1; i++) { + channel.writeInbound(randomContent(chunkSize)); + } + channel.writeInbound(randomLastContent(chunkSize)); + + for (int i = 0; i < totalChunks; i++) { + assertEquals("should not enqueue chunks", 0, stream.chunkQueue().size()); + stream.next(); + assertEquals("each next() should produce single chunk", i + 1, gotChunks.size()); + } + assertTrue("should receive last content", gotLast.get()); + } + + HttpContent randomContent(int size, boolean isLast) { + var buf = Unpooled.wrappedBuffer(randomByteArrayOfLength(size)); + if (isLast) { + return new DefaultLastHttpContent(buf); + } else { + return new DefaultHttpContent(buf); + } + } + + HttpContent randomContent(int size) { + return randomContent(size, false); + } + + HttpContent randomLastContent(int size) { + return randomContent(size, true); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java index 18df654dd435f..fae40c03b1b8d 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java @@ -905,7 +905,7 @@ public void dispatchBadRequest(final RestChannel channel, final ThreadContext th assertThat(channel.request().getHttpRequest().header(headerReference.get()), is(headerValueReference.get())); assertThat(channel.request().getHttpRequest().method(), is(translateRequestMethod(httpMethodReference.get()))); // assert content is dropped - assertThat(channel.request().getHttpRequest().content().utf8ToString(), is("")); + assertThat(channel.request().getHttpRequest().body().asFull().bytes().utf8ToString(), is("")); try { channel.sendResponse(new RestResponse(channel, (Exception) ((ElasticsearchWrapperException) cause).getCause())); } catch (IOException e) { diff --git a/server/src/main/java/org/elasticsearch/http/HttpBody.java b/server/src/main/java/org/elasticsearch/http/HttpBody.java new file mode 100644 index 0000000000000..b4d88b837b117 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/http/HttpBody.java @@ -0,0 +1,103 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.core.Nullable; + +/** + * A super-interface for different HTTP content implementations + */ +public sealed interface HttpBody permits HttpBody.Full, HttpBody.Stream { + + static Full fromBytesReference(BytesReference bytesRef) { + return new ByteRefHttpBody(bytesRef); + } + + static Full empty() { + return new ByteRefHttpBody(BytesArray.EMPTY); + } + + default boolean isFull() { + return this instanceof Full; + } + + default boolean isStream() { + return this instanceof Stream; + } + + /** + * Assumes that HTTP body is a full content. If not sure, use {@link HttpBody#isFull()}. + */ + default Full asFull() { + assert this instanceof Full; + return (Full) this; + } + + /** + * Assumes that HTTP body is a lazy-stream. If not sure, use {@link HttpBody#isStream()}. + */ + default Stream asStream() { + assert this instanceof Stream; + return (Stream) this; + } + + /** + * Full content represents a complete http body content that can be accessed immediately. + */ + non-sealed interface Full extends HttpBody { + BytesReference bytes(); + } + + /** + * Stream is a lazy-loaded content. Stream supports only single handler, this handler must be + * set before requesting next chunk. + */ + non-sealed interface Stream extends HttpBody { + /** + * Returns current handler + */ + @Nullable + ChunkHandler handler(); + + /** + * Sets handler that can handle next chunk + */ + void setHandler(ChunkHandler chunkHandler); + + /** + * Request next chunk of data from the network. The size of the chunk depends on following + * factors. If request is not compressed then chunk size will be up to + * {@link HttpTransportSettings#SETTING_HTTP_MAX_CHUNK_SIZE}. If request is compressed then + * chunk size will be up to max_chunk_size * compression_ratio. Multiple calls can be + * deduplicated when next chunk is not yet available. It's recommended to call "next" once + * for every chunk. + *
+         * {@code
+         *     stream.setHandler((chunk, isLast) -> {
+         *         processChunk(chunk);
+         *         if (isLast == false) {
+         *             stream.next();
+         *         }
+         *     });
+         * }
+         * 
+ */ + void next(); + } + + @FunctionalInterface + interface ChunkHandler { + void onNext(ReleasableBytesReference chunk, boolean isLast); + } + + record ByteRefHttpBody(BytesReference bytes) implements Full {} +} diff --git a/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java index 9f7a4fdc2ee6e..9097610dc7722 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java @@ -227,7 +227,9 @@ synchronized void update(HttpRequest httpRequest, HttpChannel httpChannel, long lastRequestTimeMillis = currentTimeMillis; lastUri = httpRequest.uri(); requestCount += 1; - requestSizeBytes += httpRequest.content().length(); + if (httpRequest.body().isFull()) { + requestSizeBytes += httpRequest.body().asFull().bytes().length(); + } } private static String getFirstValueForHeader(final HttpRequest request, final String header) { diff --git a/server/src/main/java/org/elasticsearch/http/HttpRequest.java b/server/src/main/java/org/elasticsearch/http/HttpRequest.java index b41f82def5013..ca6e51f2cec08 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRequest.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRequest.java @@ -28,7 +28,7 @@ enum HttpVersion { HTTP_1_1 } - BytesReference content(); + HttpBody body(); List strictCookies(); @@ -47,7 +47,7 @@ enum HttpVersion { Exception getInboundException(); /** - * Release any resources associated with this request. Implementations should be idempotent. The behavior of {@link #content()} + * Release any resources associated with this request. Implementations should be idempotent. The behavior of {@link #body()} * after this method has been invoked is undefined and implementation specific. */ void release(); diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 27432050c8b45..227c1edc0867e 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -11,6 +11,7 @@ import org.apache.lucene.search.spell.LevenshteinDistance; import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.util.set.Sets; @@ -117,12 +118,18 @@ public final void handleRequest(RestRequest request, RestChannel channel, NodeCl throw new IllegalArgumentException(unrecognized(request, unconsumedParams, candidateParams, "parameter")); } - if (request.hasContent() && request.isContentConsumed() == false) { + if (request.hasContent() && (request.isContentConsumed() == false && request.isFullContent())) { throw new IllegalArgumentException( "request [" + request.method() + " " + request.path() + "] does not support having a body" ); } + if (request.isStreamedContent()) { + assert action instanceof RequestBodyChunkConsumer; + var chunkConsumer = (RequestBodyChunkConsumer) action; + request.contentStream().setHandler((chunk, isLast) -> chunkConsumer.handleChunk(channel, chunk, isLast)); + } + usageCount.increment(); // execute the action action.accept(channel); @@ -180,6 +187,10 @@ protected interface RestChannelConsumer extends CheckedConsumer 0) { + if (request.hasContent()) { if (isContentTypeDisallowed(request) || handler.mediaTypesValid(request) == false) { sendContentTypeErrorMessage(request.getAllHeaderValues("Content-Type"), channel); return; @@ -454,6 +453,9 @@ private void dispatchRequest( return; } } + // TODO: estimate streamed content size for circuit breaker, + // something like http_max_chunk_size * avg_compression_ratio(for compressed content) + final int contentLength = request.isFullContent() ? request.contentLength() : 0; try { if (handler.canTripCircuitBreaker()) { inFlightRequestsBreaker(circuitBreakerService).addEstimateBytesAndMaybeBreak(contentLength, ""); diff --git a/server/src/main/java/org/elasticsearch/rest/RestRequest.java b/server/src/main/java/org/elasticsearch/rest/RestRequest.java index fb227f471256d..e48677f46d57a 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestRequest.java +++ b/server/src/main/java/org/elasticsearch/rest/RestRequest.java @@ -24,6 +24,7 @@ import org.elasticsearch.core.RestApiVersion; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.telemetry.tracing.Traceable; @@ -304,16 +305,28 @@ public final String path() { } public boolean hasContent() { - return contentLength() > 0; + return isStreamedContent() || contentLength() > 0; } public int contentLength() { - return httpRequest.content().length(); + return httpRequest.body().asFull().bytes().length(); + } + + public boolean isFullContent() { + return httpRequest.body().isFull(); } public BytesReference content() { this.contentConsumed = true; - return httpRequest.content(); + return httpRequest.body().asFull().bytes(); + } + + public boolean isStreamedContent() { + return httpRequest.body().isStream(); + } + + public HttpBody.Stream contentStream() { + return httpRequest.body().asStream(); } /** diff --git a/server/src/test/java/org/elasticsearch/http/HttpClientStatsTrackerTests.java b/server/src/test/java/org/elasticsearch/http/HttpClientStatsTrackerTests.java index 7de283bab2ea1..a1129e4a717fd 100644 --- a/server/src/test/java/org/elasticsearch/http/HttpClientStatsTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/http/HttpClientStatsTrackerTests.java @@ -120,7 +120,7 @@ public void testStatsCollection() { assertThat(clientStats.remoteAddress(), equalTo(NetworkAddress.format(httpChannel.getRemoteAddress()))); assertThat(clientStats.lastUri(), equalTo(httpRequest1.uri())); assertThat(clientStats.requestCount(), equalTo(1L)); - requestLength += httpRequest1.content().length(); + requestLength += httpRequest1.body().asFull().bytes().length(); assertThat(clientStats.requestSizeBytes(), equalTo(requestLength)); assertThat(clientStats.closedTimeMillis(), equalTo(-1L)); assertThat(clientStats.openedTimeMillis(), equalTo(openTimeMillis)); @@ -150,7 +150,7 @@ public void testStatsCollection() { assertThat(clientStats.remoteAddress(), equalTo(NetworkAddress.format(httpChannel.getRemoteAddress()))); assertThat(clientStats.lastUri(), equalTo(httpRequest2.uri())); assertThat(clientStats.requestCount(), equalTo(2L)); - requestLength += httpRequest2.content().length(); + requestLength += httpRequest2.body().asFull().bytes().length(); assertThat(clientStats.requestSizeBytes(), equalTo(requestLength)); assertThat(clientStats.closedTimeMillis(), equalTo(-1L)); assertThat(clientStats.openedTimeMillis(), equalTo(openTimeMillis)); diff --git a/server/src/test/java/org/elasticsearch/http/TestHttpRequest.java b/server/src/test/java/org/elasticsearch/http/TestHttpRequest.java index d6cf010a90471..8cd61453a3391 100644 --- a/server/src/test/java/org/elasticsearch/http/TestHttpRequest.java +++ b/server/src/test/java/org/elasticsearch/http/TestHttpRequest.java @@ -9,7 +9,6 @@ package org.elasticsearch.http; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.rest.ChunkedRestResponseBodyPart; import org.elasticsearch.rest.RestRequest; @@ -49,8 +48,8 @@ public String uri() { } @Override - public BytesReference content() { - return BytesArray.EMPTY; + public HttpBody body() { + return HttpBody.empty(); } @Override diff --git a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java index 8853665ae2641..1d946681661e7 100644 --- a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.http.HttpHeadersValidationException; import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpRequest; @@ -831,11 +832,11 @@ public String uri() { } @Override - public BytesReference content() { + public HttpBody body() { if (hasContent) { - return new BytesArray("test"); + return HttpBody.fromBytesReference(new BytesArray("test")); } - return BytesArray.EMPTY; + return HttpBody.empty(); } @Override diff --git a/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java b/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java index 17c66b888b320..8a0ca5ba6c8a5 100644 --- a/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java +++ b/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.test.ESTestCase; @@ -87,7 +88,7 @@ public void testContentLengthDoesNotConsumesContent() { private void runConsumesContentTest(final CheckedConsumer consumer, final boolean expected) { final HttpRequest httpRequest = mock(HttpRequest.class); when(httpRequest.uri()).thenReturn(""); - when(httpRequest.content()).thenReturn(new BytesArray(new byte[1])); + when(httpRequest.body()).thenReturn(HttpBody.fromBytesReference(new BytesArray(new byte[1]))); when(httpRequest.getHeaders()).thenReturn( Collections.singletonMap("Content-Type", Collections.singletonList(randomFrom("application/json", "application/x-ndjson"))) ); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java index ab3543d8f2bb7..0595e6cbb1c97 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.http.HttpResponse; @@ -54,24 +55,24 @@ public static class FakeHttpRequest implements HttpRequest { private final Method method; private final String uri; - private final BytesReference content; + private final HttpBody content; private final Map> headers; private final Exception inboundException; public FakeHttpRequest(Method method, String uri, BytesReference content, Map> headers) { - this(method, uri, content, headers, null); + this(method, uri, content == null ? HttpBody.empty() : HttpBody.fromBytesReference(content), headers, null); } private FakeHttpRequest( Method method, String uri, - BytesReference content, + HttpBody content, Map> headers, Exception inboundException ) { this.method = method; this.uri = uri; - this.content = content == null ? BytesArray.EMPTY : content; + this.content = content; this.headers = headers; this.inboundException = inboundException; } @@ -87,7 +88,7 @@ public String uri() { } @Override - public BytesReference content() { + public HttpBody body() { return content; } @@ -195,7 +196,7 @@ public static class Builder { private Map params = new HashMap<>(); - private BytesReference content = BytesArray.EMPTY; + private HttpBody content = HttpBody.empty(); private String path = "/"; @@ -221,7 +222,7 @@ public Builder withParams(Map params) { } public Builder withContent(BytesReference contentBytes, XContentType xContentType) { - this.content = contentBytes; + this.content = HttpBody.fromBytesReference(contentBytes); if (xContentType != null) { headers.put("Content-Type", Collections.singletonList(xContentType.mediaType())); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java index cb524a48d0ec7..5adc1e351931d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java @@ -2644,7 +2644,7 @@ public void testAuthenticationSuccessRest() throws Exception { checkedFields.put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId); checkedFields.put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); if (includeRequestBody && Strings.hasLength(request.content())) { - checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, request.getHttpRequest().content().utf8ToString()); + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, request.getHttpRequest().body().asFull().bytes().utf8ToString()); } if (params.isEmpty() == false) { checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "foo=bar&evac=true"); From 478baf1459c5be0812118efdfcaf0bd6d143567a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 20 Aug 2024 11:23:49 -0600 Subject: [PATCH 02/14] Allow incremental bulk request execution (#111865) Allow a single bulk request to be passed to Elasticsearch in multiple parts. Once a certain memory threshold or number of operations have been received, the request can be split and submitted for processing. --- .../Netty4IncrementalRequestHandlingIT.java | 9 +- .../http/netty4/Netty4HttpAggregator.java | 9 +- .../netty4/Netty4HttpRequestBodyStream.java | 9 +- .../Netty4HttpRequestBodyStreamTests.java | 9 +- .../action/bulk/IncrementalBulkIT.java | 429 ++++++++++++++++++ .../org/elasticsearch/TransportVersions.java | 1 + .../action/bulk/BulkOperation.java | 167 ++++--- .../action/bulk/BulkRequest.java | 34 ++ .../action/bulk/BulkRequestModifier.java | 9 +- .../action/bulk/BulkResponse.java | 28 ++ .../action/bulk/IncrementalBulkService.java | 209 +++++++++ .../java/org/elasticsearch/http/HttpBody.java | 9 +- .../elasticsearch/node/NodeConstruction.java | 3 + .../elasticsearch/test/ESIntegTestCase.java | 50 +- 14 files changed, 875 insertions(+), 100 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index 4de7ca97ed51b..767dc887bef13 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http.netty4; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java index 16f1c2bbd2e37..9fdfe6964ee04 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http.netty4; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index 8497e3ee8a40d..e27b4c0a8c739 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http.netty4; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java index 00066ffaf0201..3448fd15d79d6 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http.netty4; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java new file mode 100644 index 0000000000000..bef6c818e926b --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java @@ -0,0 +1,429 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.bulk; + +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexingPressure; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.ingest.IngestClientIT; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class IncrementalBulkIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return List.of(IngestClientIT.ExtendedIngestTestPlugin.class); + } + + public void testSingleBulkRequest() { + String index = "test"; + createIndex(index); + + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class); + + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + IndexRequest indexRequest = indexRequest(index); + + PlainActionFuture future = new PlainActionFuture<>(); + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + handler.lastItems(List.of(indexRequest), refCounted::decRef, future); + + BulkResponse bulkResponse = future.actionGet(); + assertNoFailures(bulkResponse); + + refresh(index); + + assertResponse(prepareSearch(index).setQuery(QueryBuilders.matchAllQuery()), searchResponse -> { + assertNoFailures(searchResponse); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo((long) 1)); + }); + + assertFalse(refCounted.hasReferences()); + } + + public void testMultipleBulkPartsWithBackoff() { + ExecutorService executorService = Executors.newFixedThreadPool(1); + + try (Releasable ignored = executorService::shutdown;) { + String index = "test"; + createIndex(index); + + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class); + long docs = randomIntBetween(200, 400); + + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + + BulkResponse bulkResponse = executeBulk(docs, index, handler, executorService); + assertNoFailures(bulkResponse); + + refresh(index); + + assertResponse(prepareSearch(index).setQuery(QueryBuilders.matchAllQuery()), searchResponse -> { + assertNoFailures(searchResponse); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(docs)); + }); + } + } + + public void testGlobalBulkFailure() throws InterruptedException { + ExecutorService executorService = Executors.newFixedThreadPool(1); + CountDownLatch blockingLatch = new CountDownLatch(1); + + try (Releasable ignored = executorService::shutdown; Releasable ignored2 = blockingLatch::countDown) { + String index = "test"; + createIndex(index); + + String randomNodeName = internalCluster().getRandomNodeName(); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, randomNodeName); + ThreadPool threadPool = internalCluster().getInstance(ThreadPool.class, randomNodeName); + + int threadCount = threadPool.info(ThreadPool.Names.WRITE).getMax(); + long queueSize = threadPool.info(ThreadPool.Names.WRITE).getQueueSize().singles(); + blockWritePool(threadCount, threadPool, blockingLatch); + + Runnable runnable = () -> {}; + for (int i = 0; i < queueSize; i++) { + threadPool.executor(ThreadPool.Names.WRITE).execute(runnable); + } + + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + if (randomBoolean()) { + expectThrows( + EsRejectedExecutionException.class, + () -> executeBulk(randomIntBetween(200, 400), index, handler, executorService) + ); + } else { + PlainActionFuture future = new PlainActionFuture<>(); + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + handler.lastItems(List.of(indexRequest(index)), refCounted::decRef, future); + assertFalse(refCounted.hasReferences()); + expectThrows(EsRejectedExecutionException.class, future::actionGet); + } + } + } + + public void testBulkLevelBulkFailureAfterFirstIncrementalRequest() throws Exception { + ExecutorService executorService = Executors.newFixedThreadPool(1); + + try (Releasable ignored = executorService::shutdown) { + String index = "test"; + createIndex(index); + + String randomNodeName = internalCluster().getRandomNodeName(); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, randomNodeName); + ThreadPool threadPool = internalCluster().getInstance(ThreadPool.class, randomNodeName); + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + PlainActionFuture future = new PlainActionFuture<>(); + + int threadCount = threadPool.info(ThreadPool.Names.WRITE).getMax(); + long queueSize = threadPool.info(ThreadPool.Names.WRITE).getQueueSize().singles(); + + CountDownLatch blockingLatch1 = new CountDownLatch(1); + + AtomicBoolean nextRequested = new AtomicBoolean(true); + AtomicLong hits = new AtomicLong(0); + try (Releasable ignored2 = blockingLatch1::countDown;) { + blockWritePool(threadCount, threadPool, blockingLatch1); + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextRequested.set(true)); + hits.incrementAndGet(); + } + } + assertBusy(() -> assertTrue(nextRequested.get())); + + CountDownLatch blockingLatch2 = new CountDownLatch(1); + + try (Releasable ignored3 = blockingLatch2::countDown;) { + blockWritePool(threadCount, threadPool, blockingLatch2); + Runnable runnable = () -> {}; + // Fill Queue + for (int i = 0; i < queueSize; i++) { + threadPool.executor(ThreadPool.Names.WRITE).execute(runnable); + } + + handler.lastItems(List.of(indexRequest(index)), refCounted::decRef, future); + } + + // Should not throw because some succeeded + BulkResponse bulkResponse = future.actionGet(); + + assertTrue(bulkResponse.hasFailures()); + BulkItemResponse[] items = bulkResponse.getItems(); + assertThat(Arrays.stream(items).filter(r -> r.isFailed() == false).count(), equalTo(hits.get())); + assertThat(items[items.length - 1].getFailure().getCause(), instanceOf(EsRejectedExecutionException.class)); + + refresh(index); + + assertResponse(prepareSearch(index).setQuery(QueryBuilders.matchAllQuery()), searchResponse -> { + assertNoFailures(searchResponse); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(hits.get())); + }); + } + } + + public void testShortCircuitShardLevelFailure() throws Exception { + String index = "test"; + createIndex(index, 2, 0); + + String coordinatingOnlyNode = internalCluster().startCoordinatingOnlyNode(Settings.EMPTY); + + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, coordinatingOnlyNode); + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + + AtomicBoolean nextRequested = new AtomicBoolean(true); + AtomicLong hits = new AtomicLong(0); + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextRequested.set(true)); + hits.incrementAndGet(); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + + String node = findShard(resolveIndex(index), 0); + String secondShardNode = findShard(resolveIndex(index), 1); + IndexingPressure primaryPressure = internalCluster().getInstance(IndexingPressure.class, node); + long memoryLimit = primaryPressure.stats().getMemoryLimit(); + try (Releasable releasable = primaryPressure.markPrimaryOperationStarted(10, memoryLimit, false)) { + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextRequested.set(true)); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + } + + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextRequested.set(true)); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + + PlainActionFuture future = new PlainActionFuture<>(); + handler.lastItems(List.of(indexRequest(index)), refCounted::decRef, future); + + BulkResponse bulkResponse = future.actionGet(); + assertTrue(bulkResponse.hasFailures()); + for (int i = 0; i < hits.get(); ++i) { + assertFalse(bulkResponse.getItems()[i].isFailed()); + } + + boolean shardsOnDifferentNodes = node.equals(secondShardNode) == false; + for (int i = (int) hits.get(); i < bulkResponse.getItems().length; ++i) { + BulkItemResponse item = bulkResponse.getItems()[i]; + if (item.getResponse() != null && item.getResponse().getShardId().id() == 1 && shardsOnDifferentNodes) { + assertFalse(item.isFailed()); + } else { + assertTrue(item.isFailed()); + assertThat(item.getFailure().getCause().getCause(), instanceOf(EsRejectedExecutionException.class)); + } + } + } + + public void testShortCircuitShardLevelFailureWithIngestNodeHop() throws Exception { + String dataOnlyNode = internalCluster().startDataOnlyNode(); + String index = "test1"; + + // We ensure that the index is assigned to a non-ingest node to ensure that indexing pressure does not reject at the coordinating + // level. + createIndex( + index, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.allocation.require._name", dataOnlyNode) + .build() + ); + + String pipelineId = "pipeline_id"; + BytesReference pipelineSource = BytesReference.bytes( + jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject() + ); + + putJsonPipeline(pipelineId, pipelineSource); + + // By adding an ingest pipeline and sending the request to a coordinating node without the ingest role, we ensure that we are + // testing the serialization of shard level requests over the wire. This is because the transport bulk action will be dispatched to + // a node with the ingest role. + String coordinatingOnlyNode = internalCluster().startCoordinatingOnlyNode(Settings.EMPTY); + + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, coordinatingOnlyNode); + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + + AtomicBoolean nextRequested = new AtomicBoolean(true); + AtomicLong hits = new AtomicLong(0); + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index).setPipeline(pipelineId)), refCounted::decRef, () -> nextRequested.set(true)); + hits.incrementAndGet(); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + + String node = findShard(resolveIndex(index), 0); + assertThat(node, equalTo(dataOnlyNode)); + IndexingPressure primaryPressure = internalCluster().getInstance(IndexingPressure.class, node); + long memoryLimit = primaryPressure.stats().getMemoryLimit(); + try (Releasable releasable = primaryPressure.markPrimaryOperationStarted(10, memoryLimit, false)) { + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index).setPipeline(pipelineId)), refCounted::decRef, () -> nextRequested.set(true)); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + } + + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index).setPipeline(pipelineId)), refCounted::decRef, () -> nextRequested.set(true)); + } + + assertBusy(() -> assertTrue(nextRequested.get())); + + PlainActionFuture future = new PlainActionFuture<>(); + handler.lastItems(List.of(indexRequest(index)), refCounted::decRef, future); + + BulkResponse bulkResponse = future.actionGet(); + assertTrue(bulkResponse.hasFailures()); + for (int i = 0; i < hits.get(); ++i) { + assertFalse(bulkResponse.getItems()[i].isFailed()); + } + + for (int i = (int) hits.get(); i < bulkResponse.getItems().length; ++i) { + BulkItemResponse item = bulkResponse.getItems()[i]; + assertTrue(item.isFailed()); + assertThat(item.getFailure().getCause().getCause(), instanceOf(EsRejectedExecutionException.class)); + } + } + + private static void blockWritePool(int threadCount, ThreadPool threadPool, CountDownLatch blockingLatch) throws InterruptedException { + CountDownLatch startedLatch = new CountDownLatch(threadCount); + for (int i = 0; i < threadCount; i++) { + threadPool.executor(ThreadPool.Names.WRITE).execute(() -> { + startedLatch.countDown(); + try { + blockingLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + } + startedLatch.await(); + } + + private BulkResponse executeBulk(long docs, String index, IncrementalBulkService.Handler handler, ExecutorService executorService) { + ConcurrentLinkedQueue> queue = new ConcurrentLinkedQueue<>(); + for (int i = 0; i < docs; i++) { + IndexRequest indexRequest = indexRequest(index); + queue.add(indexRequest); + } + + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + PlainActionFuture future = new PlainActionFuture<>(); + Runnable r = new Runnable() { + + @Override + public void run() { + int toRemove = Math.min(randomIntBetween(5, 10), queue.size()); + ArrayList> docs = new ArrayList<>(); + for (int i = 0; i < toRemove; i++) { + docs.add(queue.poll()); + } + + if (queue.isEmpty()) { + handler.lastItems(docs, refCounted::decRef, future); + } else { + refCounted.incRef(); + handler.addItems(docs, refCounted::decRef, () -> executorService.execute(this)); + } + } + }; + + executorService.execute(r); + + BulkResponse bulkResponse = future.actionGet(); + assertFalse(refCounted.hasReferences()); + return bulkResponse; + } + + private static IndexRequest indexRequest(String index) { + IndexRequest indexRequest = new IndexRequest(); + indexRequest.index(index); + indexRequest.source(Map.of("field", randomAlphaOfLength(10))); + return indexRequest; + } + + protected static String findShard(Index index, int shardId) { + for (String node : internalCluster().getNodeNames()) { + var indicesService = internalCluster().getInstance(IndicesService.class, node); + IndexService indexService = indicesService.indexService(index); + if (indexService != null) { + IndexShard shard = indexService.getShardOrNull(shardId); + if (shard != null && shard.isActive() && shard.routingEntry().primary()) { + return node; + } + } + } + throw new AssertionError("IndexShard instance not found for shard " + new ShardId(index, shardId)); + } +} diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index c25825475aa9c..cef4bd14d992b 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -218,6 +218,7 @@ static TransportVersion def(int id) { public static final TransportVersion ML_SCHEDULED_EVENT_TIME_SHIFT_CONFIGURATION = def(8_742_00_0); public static final TransportVersion SIMULATE_COMPONENT_TEMPLATES_SUBSTITUTIONS = def(8_743_00_0); public static final TransportVersion ML_INFERENCE_IBM_WATSONX_EMBEDDINGS_ADDED = def(8_744_00_0); + public static final TransportVersion BULK_INCREMENTAL_STATE = def(8_745_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java index 98e3548ecf30e..d22719cbf750b 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java @@ -95,6 +95,7 @@ final class BulkOperation extends ActionRunnable { private final OriginSettingClient rolloverClient; private final Set failureStoresToBeRolledOver = ConcurrentCollections.newConcurrentSet(); private final Set failedRolloverRequests = ConcurrentCollections.newConcurrentSet(); + private final Map shortCircuitShardFailures = ConcurrentCollections.newConcurrentMap(); private final FailureStoreMetrics failureStoreMetrics; BulkOperation( @@ -164,6 +165,7 @@ final class BulkOperation extends ActionRunnable { this.observer = observer; this.failureStoreDocumentConverter = failureStoreDocumentConverter; this.rolloverClient = new OriginSettingClient(client, LAZY_ROLLOVER_ORIGIN); + this.shortCircuitShardFailures.putAll(bulkRequest.incrementalState().shardLevelFailures()); this.failureStoreMetrics = failureStoreMetrics; } @@ -403,7 +405,12 @@ private void redirectFailuresOrCompleteBulkOperation() { private void completeBulkOperation() { listener.onResponse( - new BulkResponse(responses.toArray(new BulkItemResponse[responses.length()]), buildTookInMillis(startTimeNanos)) + new BulkResponse( + responses.toArray(new BulkItemResponse[responses.length()]), + buildTookInMillis(startTimeNanos), + BulkResponse.NO_INGEST_TOOK, + new BulkRequest.IncrementalState(shortCircuitShardFailures) + ) ); // Allow memory for bulk shard request items to be reclaimed before all items have been completed bulkRequest = null; @@ -429,90 +436,102 @@ private void discardRedirectsAndFinish(Exception exception) { } private void executeBulkShardRequest(BulkShardRequest bulkShardRequest, Releasable releaseOnFinish) { - client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { + ShardId shardId = bulkShardRequest.shardId(); - // Lazily get the cluster state to avoid keeping it around longer than it is needed - private ClusterState clusterState = null; + // Short circuit the shark level request with the existing shard failure. + if (shortCircuitShardFailures.containsKey(shardId)) { + handleShardFailure(bulkShardRequest, clusterService.state(), shortCircuitShardFailures.get(shardId)); + releaseOnFinish.close(); + } else { + client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { - private ClusterState getClusterState() { - if (clusterState == null) { - clusterState = clusterService.state(); - } - return clusterState; - } + // Lazily get the cluster state to avoid keeping it around longer than it is needed + private ClusterState clusterState = null; - @Override - public void onResponse(BulkShardResponse bulkShardResponse) { - for (int idx = 0; idx < bulkShardResponse.getResponses().length; idx++) { - // We zip the requests and responses together so that we can identify failed documents and potentially store them - BulkItemResponse bulkItemResponse = bulkShardResponse.getResponses()[idx]; - BulkItemRequest bulkItemRequest = bulkShardRequest.items()[idx]; - - if (bulkItemResponse.isFailed()) { - assert bulkItemRequest.id() == bulkItemResponse.getItemId() : "Bulk items were returned out of order"; - processFailure(bulkItemRequest, bulkItemResponse.getFailure().getCause()); - addFailure(bulkItemResponse); - } else { - bulkItemResponse.getResponse().setShardInfo(bulkShardResponse.getShardInfo()); - responses.set(bulkItemResponse.getItemId(), bulkItemResponse); + private ClusterState getClusterState() { + if (clusterState == null) { + clusterState = clusterService.state(); } + return clusterState; } - completeShardOperation(); - } - @Override - public void onFailure(Exception e) { - // create failures for all relevant requests - for (BulkItemRequest request : bulkShardRequest.items()) { - final String indexName = request.index(); - DocWriteRequest docWriteRequest = request.request(); - - processFailure(request, e); - addFailure(docWriteRequest, request.id(), indexName, e); + @Override + public void onResponse(BulkShardResponse bulkShardResponse) { + for (int idx = 0; idx < bulkShardResponse.getResponses().length; idx++) { + // We zip the requests and responses together so that we can identify failed documents and potentially store them + BulkItemResponse bulkItemResponse = bulkShardResponse.getResponses()[idx]; + BulkItemRequest bulkItemRequest = bulkShardRequest.items()[idx]; + + if (bulkItemResponse.isFailed()) { + assert bulkItemRequest.id() == bulkItemResponse.getItemId() : "Bulk items were returned out of order"; + processFailure(bulkItemRequest, getClusterState(), bulkItemResponse.getFailure().getCause()); + addFailure(bulkItemResponse); + } else { + bulkItemResponse.getResponse().setShardInfo(bulkShardResponse.getShardInfo()); + responses.set(bulkItemResponse.getItemId(), bulkItemResponse); + } + } + completeShardOperation(); } - completeShardOperation(); - } - private void completeShardOperation() { - // Clear our handle on the cluster state to allow it to be cleaned up - clusterState = null; - releaseOnFinish.close(); - } + @Override + public void onFailure(Exception e) { + assert shortCircuitShardFailures.containsKey(shardId) == false; + shortCircuitShardFailures.put(shardId, e); - private void processFailure(BulkItemRequest bulkItemRequest, Exception cause) { - var error = ExceptionsHelper.unwrapCause(cause); - var errorType = ElasticsearchException.getExceptionName(error); - DocWriteRequest docWriteRequest = bulkItemRequest.request(); - DataStream failureStoreCandidate = getRedirectTargetCandidate(docWriteRequest, getClusterState().metadata()); - // If the candidate is not null, the BulkItemRequest targets a data stream, but we'll still have to check if - // it has the failure store enabled. - if (failureStoreCandidate != null) { - // Do not redirect documents to a failure store that were already headed to one. - var isFailureStoreDoc = docWriteRequest instanceof IndexRequest indexRequest && indexRequest.isWriteToFailureStore(); - if (isFailureStoreDoc == false - && failureStoreCandidate.isFailureStoreEnabled() - && error instanceof VersionConflictEngineException == false) { - // Redirect to failure store. - maybeMarkFailureStoreForRollover(failureStoreCandidate); - addDocumentToRedirectRequests(bulkItemRequest, cause, failureStoreCandidate.getName()); - failureStoreMetrics.incrementFailureStore( - bulkItemRequest.index(), - errorType, - FailureStoreMetrics.ErrorLocation.SHARD - ); - } else { - // If we can't redirect to a failure store (because either the data stream doesn't have the failure store enabled - // or this request was already targeting a failure store), we increment the rejected counter. - failureStoreMetrics.incrementRejected( - bulkItemRequest.index(), - errorType, - FailureStoreMetrics.ErrorLocation.SHARD, - isFailureStoreDoc - ); - } + // create failures for all relevant requests + handleShardFailure(bulkShardRequest, getClusterState(), e); + completeShardOperation(); } + + private void completeShardOperation() { + // Clear our handle on the cluster state to allow it to be cleaned up + clusterState = null; + releaseOnFinish.close(); + } + }); + } + } + + private void handleShardFailure(BulkShardRequest bulkShardRequest, ClusterState clusterState, Exception e) { + // create failures for all relevant requests + for (BulkItemRequest request : bulkShardRequest.items()) { + final String indexName = request.index(); + DocWriteRequest docWriteRequest = request.request(); + + processFailure(request, clusterState, e); + addFailure(docWriteRequest, request.id(), indexName, e); + } + } + + private void processFailure(BulkItemRequest bulkItemRequest, ClusterState clusterState, Exception cause) { + var error = ExceptionsHelper.unwrapCause(cause); + var errorType = ElasticsearchException.getExceptionName(error); + DocWriteRequest docWriteRequest = bulkItemRequest.request(); + DataStream failureStoreCandidate = getRedirectTargetCandidate(docWriteRequest, clusterState.metadata()); + // If the candidate is not null, the BulkItemRequest targets a data stream, but we'll still have to check if + // it has the failure store enabled. + if (failureStoreCandidate != null) { + // Do not redirect documents to a failure store that were already headed to one. + var isFailureStoreDoc = docWriteRequest instanceof IndexRequest indexRequest && indexRequest.isWriteToFailureStore(); + if (isFailureStoreDoc == false + && failureStoreCandidate.isFailureStoreEnabled() + && error instanceof VersionConflictEngineException == false) { + // Redirect to failure store. + maybeMarkFailureStoreForRollover(failureStoreCandidate); + addDocumentToRedirectRequests(bulkItemRequest, cause, failureStoreCandidate.getName()); + failureStoreMetrics.incrementFailureStore(bulkItemRequest.index(), errorType, FailureStoreMetrics.ErrorLocation.SHARD); + } else { + // If we can't redirect to a failure store (because either the data stream doesn't have the failure store enabled + // or this request was already targeting a failure store), we increment the rejected counter. + failureStoreMetrics.incrementRejected( + bulkItemRequest.index(), + errorType, + FailureStoreMetrics.ErrorLocation.SHARD, + isFailureStoreDoc + ); } - }); + } } /** diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 1425dde28ea3b..65c7a9909057b 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -11,6 +11,7 @@ import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; @@ -27,9 +28,11 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.RestApiVersion; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.transport.RawIndexingDataTransportRequest; import org.elasticsearch.xcontent.XContentType; @@ -72,6 +75,7 @@ public class BulkRequest extends ActionRequest private final Set indices = new HashSet<>(); protected TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT; + private IncrementalState incrementalState = IncrementalState.EMPTY; private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT; private RefreshPolicy refreshPolicy = RefreshPolicy.NONE; private String globalPipeline; @@ -93,6 +97,11 @@ public BulkRequest(StreamInput in) throws IOException { for (DocWriteRequest request : requests) { indices.add(Objects.requireNonNull(request.index(), "request index must not be null")); } + if (in.getTransportVersion().onOrAfter(TransportVersions.BULK_INCREMENTAL_STATE)) { + incrementalState = new BulkRequest.IncrementalState(in); + } else { + incrementalState = BulkRequest.IncrementalState.EMPTY; + } } public BulkRequest(@Nullable String globalIndex) { @@ -327,6 +336,10 @@ public final BulkRequest timeout(TimeValue timeout) { return this; } + public void incrementalState(IncrementalState incrementalState) { + this.incrementalState = incrementalState; + } + /** * Note for internal callers (NOT high level rest client), * the global parameter setting is ignored when used with: @@ -365,6 +378,10 @@ public TimeValue timeout() { return timeout; } + public IncrementalState incrementalState() { + return incrementalState; + } + public String pipeline() { return globalPipeline; } @@ -436,6 +453,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(requests, DocWriteRequest::writeDocumentRequest); refreshPolicy.writeTo(out); out.writeTimeValue(timeout); + if (out.getTransportVersion().onOrAfter(TransportVersions.BULK_INCREMENTAL_STATE)) { + incrementalState.writeTo(out); + } } @Override @@ -486,6 +506,20 @@ public Map getComponentTemplateSubstitutions() throws return Map.of(); } + record IncrementalState(Map shardLevelFailures) implements Writeable { + + static final IncrementalState EMPTY = new IncrementalState(Collections.emptyMap()); + + IncrementalState(StreamInput in) throws IOException { + this(in.readMap(ShardId::new, input -> input.readException())); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(shardLevelFailures, (o, s) -> s.writeTo(o), StreamOutput::writeException); + } + } + /* * This copies this bulk request, but without all of its inner requests or the set of indices found in those requests */ diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestModifier.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestModifier.java index 3e47c78a76354..282e4d33fb83b 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestModifier.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestModifier.java @@ -114,7 +114,12 @@ BulkRequest getBulkRequest() { ActionListener wrapActionListenerIfNeeded(long ingestTookInMillis, ActionListener actionListener) { if (itemResponses.isEmpty()) { return actionListener.map( - response -> new BulkResponse(response.getItems(), response.getTook().getMillis(), ingestTookInMillis) + response -> new BulkResponse( + response.getItems(), + response.getTook().getMillis(), + ingestTookInMillis, + response.getIncrementalState() + ) ); } else { return actionListener.map(response -> { @@ -139,7 +144,7 @@ ActionListener wrapActionListenerIfNeeded(long ingestTookInMillis, assertResponsesAreCorrect(bulkResponses, allResponses); } - return new BulkResponse(allResponses, response.getTook().getMillis(), ingestTookInMillis); + return new BulkResponse(allResponses, response.getTook().getMillis(), ingestTookInMillis, response.getIncrementalState()); }); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkResponse.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkResponse.java index 8f12341d71e7b..b02d7acf66d14 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkResponse.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkResponse.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.bulk; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; @@ -37,12 +38,18 @@ public class BulkResponse extends ActionResponse implements Iterable releasables = new ArrayList<>(4); + private final ArrayList responses = new ArrayList<>(2); + private boolean globalFailure = false; + private boolean incrementalRequestSubmitted = false; + private Exception bulkActionLevelFailure = null; + private BulkRequest bulkRequest = null; + + private Handler(Client client, @Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh) { + this.client = client; + this.waitForActiveShards = waitForActiveShards != null ? ActiveShardCount.parseString(waitForActiveShards) : null; + this.timeout = timeout; + this.refresh = refresh; + createNewBulkRequest(BulkRequest.IncrementalState.EMPTY); + } + + public void addItems(List> items, Releasable releasable, Runnable nextItems) { + if (bulkActionLevelFailure != null) { + shortCircuitDueToTopLevelFailure(items, releasable); + nextItems.run(); + } else { + assert bulkRequest != null; + internalAddItems(items, releasable); + + if (shouldBackOff()) { + final boolean isFirstRequest = incrementalRequestSubmitted == false; + incrementalRequestSubmitted = true; + + client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { + + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + createNewBulkRequest(bulkResponse.getIncrementalState()); + } + + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + } + }, nextItems::run)); + } else { + nextItems.run(); + } + } + } + + private boolean shouldBackOff() { + // TODO: Implement Real Memory Logic + return bulkRequest.requests().size() >= 16; + } + + public void lastItems(List> items, Releasable releasable, ActionListener listener) { + if (bulkActionLevelFailure != null) { + shortCircuitDueToTopLevelFailure(items, releasable); + errorResponse(listener); + } else { + assert bulkRequest != null; + internalAddItems(items, releasable); + + client.bulk(bulkRequest, new ActionListener<>() { + + private final boolean isFirstRequest = incrementalRequestSubmitted == false; + + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + listener.onResponse(combineResponses()); + } + + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + errorResponse(listener); + } + }); + } + } + + private void shortCircuitDueToTopLevelFailure(List> items, Releasable releasable) { + assert releasables.isEmpty(); + assert bulkRequest == null; + if (globalFailure == false) { + addItemLevelFailures(items); + } + Releasables.close(releasable); + } + + private void errorResponse(ActionListener listener) { + if (globalFailure) { + listener.onFailure(bulkActionLevelFailure); + } else { + listener.onResponse(combineResponses()); + } + } + + private void handleBulkFailure(boolean isFirstRequest, Exception e) { + assert bulkActionLevelFailure == null; + globalFailure = isFirstRequest; + bulkActionLevelFailure = e; + addItemLevelFailures(bulkRequest.requests()); + releaseCurrentReferences(); + } + + private void addItemLevelFailures(List> items) { + BulkItemResponse[] bulkItemResponses = new BulkItemResponse[items.size()]; + int idx = 0; + for (DocWriteRequest item : items) { + BulkItemResponse.Failure failure = new BulkItemResponse.Failure(item.index(), item.id(), bulkActionLevelFailure); + bulkItemResponses[idx++] = BulkItemResponse.failure(idx, item.opType(), failure); + } + + responses.add(new BulkResponse(bulkItemResponses, 0, 0)); + } + + private void internalAddItems(List> items, Releasable releasable) { + bulkRequest.add(items); + releasables.add(releasable); + } + + private void createNewBulkRequest(BulkRequest.IncrementalState incrementalState) { + bulkRequest = new BulkRequest(); + bulkRequest.incrementalState(incrementalState); + + if (waitForActiveShards != null) { + bulkRequest.waitForActiveShards(waitForActiveShards); + } + if (timeout != null) { + bulkRequest.timeout(timeout); + } + if (refresh != null) { + bulkRequest.setRefreshPolicy(refresh); + } + } + + private void releaseCurrentReferences() { + bulkRequest = null; + releasables.forEach(Releasable::close); + releasables.clear(); + } + + private BulkResponse combineResponses() { + long tookInMillis = 0; + long ingestTookInMillis = 0; + int itemResponseCount = 0; + for (BulkResponse response : responses) { + tookInMillis += response.getTookInMillis(); + ingestTookInMillis += response.getIngestTookInMillis(); + itemResponseCount += response.getItems().length; + } + BulkItemResponse[] bulkItemResponses = new BulkItemResponse[itemResponseCount]; + int i = 0; + for (BulkResponse response : responses) { + for (BulkItemResponse itemResponse : response.getItems()) { + bulkItemResponses[i++] = itemResponse; + } + } + + return new BulkResponse(bulkItemResponses, tookInMillis, ingestTookInMillis); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/http/HttpBody.java b/server/src/main/java/org/elasticsearch/http/HttpBody.java index b4d88b837b117..6e4470adaec9d 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpBody.java +++ b/server/src/main/java/org/elasticsearch/http/HttpBody.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http; diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index b6a63aefcfaff..3f8644dda140c 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.cluster.repositories.reservedstate.ReservedRepositoryAction; import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction; import org.elasticsearch.action.bulk.FailureStoreMetrics; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.datastreams.autosharding.DataStreamAutoShardingService; import org.elasticsearch.action.ingest.ReservedPipelineAction; import org.elasticsearch.action.search.SearchExecutionStatsCollector; @@ -979,6 +980,7 @@ private void construct( ); final HttpServerTransport httpServerTransport = serviceProvider.newHttpTransport(pluginsService, networkModule); final IndexingPressure indexingLimits = new IndexingPressure(settings); + final IncrementalBulkService incrementalBulkService = new IncrementalBulkService(client); SnapshotsService snapshotsService = new SnapshotsService( settings, @@ -1140,6 +1142,7 @@ private void construct( b.bind(PageCacheRecycler.class).toInstance(pageCacheRecycler); b.bind(IngestService.class).toInstance(ingestService); b.bind(IndexingPressure.class).toInstance(indexingLimits); + b.bind(IncrementalBulkService.class).toInstance(incrementalBulkService); b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService()); b.bind(MetaStateService.class).toInstance(metaStateService); b.bind(IndicesService.class).toInstance(indicesService); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 9132474fa9415..a8f269f265dc4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplainRequest; import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplainResponse; @@ -48,6 +49,8 @@ import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.bulk.IncrementalBulkService; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.ingest.DeletePipelineRequest; import org.elasticsearch.action.ingest.DeletePipelineTransportAction; @@ -192,6 +195,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; @@ -1776,11 +1780,49 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma ); logger.info("Index [{}] docs async: [{}] bulk: [{}] partitions [{}]", builders.size(), false, true, partition.size()); for (List segmented : partition) { - BulkRequestBuilder bulkBuilder = client().prepareBulk(); - for (IndexRequestBuilder indexRequestBuilder : segmented) { - bulkBuilder.add(indexRequestBuilder); + BulkResponse actionGet; + if (randomBoolean()) { + BulkRequestBuilder bulkBuilder = client().prepareBulk(); + for (IndexRequestBuilder indexRequestBuilder : segmented) { + bulkBuilder.add(indexRequestBuilder); + } + actionGet = bulkBuilder.get(); + } else { + IncrementalBulkService bulkService = internalCluster().getInstance(IncrementalBulkService.class); + IncrementalBulkService.Handler handler = bulkService.newBulkRequest(); + + ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + segmented.forEach(b -> queue.add(b.request())); + + PlainActionFuture future = new PlainActionFuture<>(); + AtomicInteger runs = new AtomicInteger(0); + Runnable r = new Runnable() { + + @Override + public void run() { + int toRemove = Math.min(randomIntBetween(5, 10), queue.size()); + ArrayList> docs = new ArrayList<>(); + for (int i = 0; i < toRemove; i++) { + docs.add(queue.poll()); + } + + if (queue.isEmpty()) { + handler.lastItems(docs, () -> {}, future); + } else { + handler.addItems(docs, () -> {}, () -> { + // Every 10 runs dispatch to new thread to prevent stackoverflow + if (runs.incrementAndGet() % 10 == 0) { + new Thread(this).start(); + } else { + this.run(); + } + }); + } + } + }; + r.run(); + actionGet = future.actionGet(); } - BulkResponse actionGet = bulkBuilder.get(); assertThat(actionGet.hasFailures() ? actionGet.buildFailureMessage() : "", actionGet.hasFailures(), equalTo(false)); } } From 1b77421cf8b5aa8db0cb657d8c4bfbbee6f86429 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Mon, 26 Aug 2024 13:10:52 -0700 Subject: [PATCH 03/14] handle 100-continue and oversized streaming request (#112179) --- .../Netty4IncrementalRequestHandlingIT.java | 138 +++++++++++++++--- .../http/netty4/Netty4HttpAggregator.java | 53 ++++++- 2 files changed, 167 insertions(+), 24 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index 767dc887bef13..9e21324203914 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -11,6 +11,7 @@ import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufInputStream; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -25,11 +26,16 @@ import io.netty.handler.codec.http.DefaultLastHttpContent; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpChunkedInput; import io.netty.handler.codec.http.HttpClientCodec; import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.stream.ChunkedStream; +import io.netty.handler.stream.ChunkedWriteHandler; import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.action.support.SubscribableListener; @@ -42,9 +48,13 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.BaseRestHandler; @@ -62,9 +72,7 @@ import java.util.List; import java.util.concurrent.BlockingDeque; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; @@ -79,6 +87,13 @@ @ESIntegTestCase.ClusterScope(numDataNodes = 1) public class Netty4IncrementalRequestHandlingIT extends ESNetty4IntegTestCase { + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings)); + builder.put(HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH.getKey(), new ByteSizeValue(50, ByteSizeUnit.MB)); + return builder.build(); + } + // ensure empty http content has single 0 size chunk public void testEmptyContent() throws Exception { try (var ctx = setupClientCtx()) { @@ -112,7 +127,7 @@ public void testReceiveAllChunks() throws Exception { var opaqueId = opaqueId(reqNo); // this dataset will be compared with one on server side - var dataSize = randomIntBetween(1024, 10 * 1024 * 1024); + var dataSize = randomIntBetween(1024, maxContentLength()); var sendData = Unpooled.wrappedBuffer(randomByteArrayOfLength(dataSize)); sendData.retain(); ctx.clientChannel.writeAndFlush(fullHttpRequest(opaqueId, sendData)); @@ -213,12 +228,98 @@ public void testClientBackpressure() throws Exception { bufSize >= minBufSize && bufSize <= maxBufSize ); }); - handler.consumeBytes(MBytes(10)); + handler.readBytes(MBytes(10)); } assertTrue(handler.stream.hasLast()); } } + // ensures that server reply 100-continue on acceptable request size + public void test100Continue() throws Exception { + try (var ctx = setupClientCtx()) { + for (int reqNo = 0; reqNo < randomIntBetween(2, 10); reqNo++) { + var id = opaqueId(reqNo); + var acceptableContentLength = randomIntBetween(0, maxContentLength()); + + // send request header and await 100-continue + var req = httpRequest(id, acceptableContentLength); + HttpUtil.set100ContinueExpected(req, true); + ctx.clientChannel.writeAndFlush(req); + var resp = (FullHttpResponse) safePoll(ctx.clientRespQueue); + assertEquals(HttpResponseStatus.CONTINUE, resp.status()); + resp.release(); + + // send content + var content = randomContent(acceptableContentLength, true); + ctx.clientChannel.writeAndFlush(content); + + // consume content and reply 200 + var handler = ctx.awaitRestChannelAccepted(id); + var consumed = handler.readAllBytes(); + assertEquals(acceptableContentLength, consumed); + handler.sendResponse(new RestResponse(RestStatus.OK, "")); + + resp = (FullHttpResponse) safePoll(ctx.clientRespQueue); + assertEquals(HttpResponseStatus.OK, resp.status()); + resp.release(); + } + } + } + + // ensures that server reply 413-too-large on oversized request with expect-100-continue + public void test413TooLargeOnExpect100Continue() throws Exception { + try (var ctx = setupClientCtx()) { + for (int reqNo = 0; reqNo < randomIntBetween(2, 10); reqNo++) { + var id = opaqueId(reqNo); + var oversized = maxContentLength() + 1; + + // send request header and await 413 too large + var req = httpRequest(id, oversized); + HttpUtil.set100ContinueExpected(req, true); + ctx.clientChannel.writeAndFlush(req); + var resp = (FullHttpResponse) safePoll(ctx.clientRespQueue); + assertEquals(HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE, resp.status()); + resp.release(); + + // terminate request + ctx.clientChannel.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); + } + } + } + + // ensures that oversized chunked encoded request has no limits at http layer + // rest handler is responsible for oversized requests + public void testOversizedChunkedEncodingNoLimits() throws Exception { + try (var ctx = setupClientCtx()) { + for (var reqNo = 0; reqNo < randomIntBetween(2, 10); reqNo++) { + var id = opaqueId(reqNo); + var contentSize = maxContentLength() + 1; + var content = randomByteArrayOfLength(contentSize); + var is = new ByteBufInputStream(Unpooled.wrappedBuffer(content)); + var chunkedIs = new ChunkedStream(is); + var httpChunkedIs = new HttpChunkedInput(chunkedIs, LastHttpContent.EMPTY_LAST_CONTENT); + var req = httpRequest(id, 0); + HttpUtil.setTransferEncodingChunked(req, true); + + ctx.clientChannel.pipeline().addLast(new ChunkedWriteHandler()); + ctx.clientChannel.writeAndFlush(req); + ctx.clientChannel.writeAndFlush(httpChunkedIs); + var handler = ctx.awaitRestChannelAccepted(id); + var consumed = handler.readAllBytes(); + assertEquals(contentSize, consumed); + handler.sendResponse(new RestResponse(RestStatus.OK, "")); + + var resp = (FullHttpResponse) safePoll(ctx.clientRespQueue); + assertEquals(HttpResponseStatus.OK, resp.status()); + resp.release(); + } + } + } + + private int maxContentLength() { + return HttpHandlingSettings.fromSettings(internalCluster().getInstance(Settings.class)).maxContentLength(); + } + private String opaqueId(int reqNo) { return getTestName() + "-" + reqNo; } @@ -369,24 +470,25 @@ void sendResponse(RestResponse response) { channel.sendResponse(response); } - void consumeBytes(int bytes) { - if (recvLast) { - return; - } - while (bytes > 0) { - stream.next(); - var recvChunk = safePoll(recvChunks); - bytes -= recvChunk.chunk.length(); - recvChunk.chunk.close(); - if (recvChunk.isLast) { - recvLast = true; - break; + int readBytes(int bytes) { + var consumed = 0; + if (recvLast == false) { + while (consumed < bytes) { + stream.next(); + var recvChunk = safePoll(recvChunks); + consumed += recvChunk.chunk.length(); + recvChunk.chunk.close(); + if (recvChunk.isLast) { + recvLast = true; + break; + } } } + return consumed; } - Future onChannelThread(Callable task) { - return this.stream.channel().eventLoop().submit(task); + int readAllBytes() { + return readBytes(Integer.MAX_VALUE); } record Chunk(ReleasableBytesReference chunk, boolean isLast) {} diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java index 9fdfe6964ee04..bab01f047ee9e 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java @@ -9,19 +9,32 @@ package org.elasticsearch.http.netty4; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.HttpObject; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; import org.elasticsearch.http.HttpPreRequest; import org.elasticsearch.http.netty4.internal.HttpHeadersAuthenticatorUtils; import java.util.function.Predicate; +/** + * A wrapper around {@link HttpObjectAggregator}. Provides optional content aggregation based on + * predicate. {@link HttpObjectAggregator} also handles Expect: 100-continue and oversized content. + * Unfortunately, Netty does not provide handlers for oversized messages beyond HttpObjectAggregator. + */ public class Netty4HttpAggregator extends HttpObjectAggregator { private static final Predicate IGNORE_TEST = (req) -> req.uri().startsWith("/_test/request-stream") == false; private final Predicate decider; - private boolean shouldAggregate; + private boolean aggregating = true; + private boolean ignoreContentAfterContinueResponse = false; public Netty4HttpAggregator(int maxContentLength) { this(maxContentLength, IGNORE_TEST); @@ -33,15 +46,43 @@ public Netty4HttpAggregator(int maxContentLength, Predicate deci } @Override - public boolean acceptInboundMessage(Object msg) throws Exception { + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + assert msg instanceof HttpObject; if (msg instanceof HttpRequest request) { var preReq = HttpHeadersAuthenticatorUtils.asHttpPreRequest(request); - shouldAggregate = decider.test(preReq); + aggregating = decider.test(preReq); + } + if (aggregating || msg instanceof FullHttpRequest) { + super.channelRead(ctx, msg); + } else { + handle(ctx, (HttpObject) msg); } - if (shouldAggregate) { - return super.acceptInboundMessage(msg); + } + + private void handle(ChannelHandlerContext ctx, HttpObject msg) { + if (msg instanceof HttpRequest request) { + var continueResponse = newContinueResponse(request, maxContentLength(), ctx.pipeline()); + if (continueResponse != null) { + // there are 3 responses expected: 100, 413, 417 + // on 100 we pass request further and reply to client to continue + // on 413/417 we ignore following content + ctx.writeAndFlush(continueResponse); + var resp = (FullHttpResponse) continueResponse; + if (resp.status() != HttpResponseStatus.CONTINUE) { + ignoreContentAfterContinueResponse = true; + return; + } + HttpUtil.set100ContinueExpected(request, false); + } + ignoreContentAfterContinueResponse = false; + ctx.fireChannelRead(msg); } else { - return false; + var httpContent = (HttpContent) msg; + if (ignoreContentAfterContinueResponse) { + httpContent.release(); + } else { + ctx.fireChannelRead(msg); + } } } } From cbcbc348635c7079e3864891f1362be578747165 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Tue, 27 Aug 2024 10:33:57 -0700 Subject: [PATCH 04/14] release stream chunk queue on bad request (#112227) --- .../Netty4IncrementalRequestHandlingIT.java | 28 +++++++++++++++++++ .../http/netty4/Netty4HttpRequest.java | 3 +- .../netty4/Netty4HttpRequestBodyStream.java | 21 +++++++++++++- .../java/org/elasticsearch/http/HttpBody.java | 6 +++- 4 files changed, 55 insertions(+), 3 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index 9e21324203914..c85f845887203 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -316,6 +316,29 @@ public void testOversizedChunkedEncodingNoLimits() throws Exception { } } + // ensures that we dont leak buffers in stream on 400-bad-request + // some bad requests are dispatched from rest-controller before reaching rest handler + // test relies on netty's buffer leak detection + public void testBadRequestReleaseQueuedChunks() throws Exception { + try (var ctx = setupClientCtx()) { + for (var reqNo = 0; reqNo < randomIntBetween(2, 10); reqNo++) { + var id = opaqueId(reqNo); + var contentSize = randomIntBetween(0, maxContentLength()); + var req = httpRequest(id, contentSize); + var content = randomContent(contentSize, true); + + // set unacceptable content-type + req.headers().set(CONTENT_TYPE, "unknown"); + ctx.clientChannel.writeAndFlush(req); + ctx.clientChannel.writeAndFlush(content); + + var resp = (FullHttpResponse) safePoll(ctx.clientRespQueue); + assertEquals(HttpResponseStatus.BAD_REQUEST, resp.status()); + resp.release(); + } + } + } + private int maxContentLength() { return HttpHandlingSettings.fromSettings(internalCluster().getInstance(Settings.class)).maxContentLength(); } @@ -514,6 +537,11 @@ public Collection getRestHandlers( Predicate clusterSupportsFeature ) { return List.of(new BaseRestHandler() { + @Override + public boolean allowsUnsafeBuffers() { + return true; + } + @Override public String getName() { return ROUTE; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java index e6b9841e3731d..4f1bf0f01478b 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java @@ -61,7 +61,7 @@ public class Netty4HttpRequest implements HttpRequest { EmptyHttpHeaders.INSTANCE ), new AtomicBoolean(false), - false, + true, contentStream, null ); @@ -116,6 +116,7 @@ public HttpBody body() { public void release() { if (pooled && released.compareAndSet(false, true)) { request.release(); + content.close(); } } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index e27b4c0a8c739..a7783626e6b80 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -31,11 +31,12 @@ public class Netty4HttpRequestBodyStream implements HttpBody.Stream { private final Queue chunkQueue = new ArrayDeque<>(); private boolean requested = false; private boolean hasLast = false; + private boolean closing = false; private HttpBody.ChunkHandler handler; public Netty4HttpRequestBodyStream(Channel channel) { this.channel = channel; - channel.closeFuture().addListener((f) -> releaseQueuedChunks()); + channel.closeFuture().addListener((f) -> doClose()); channel.config().setAutoRead(false); } @@ -71,6 +72,10 @@ public void next() { } public void handleNettyContent(HttpContent httpContent) { + if (closing) { + httpContent.release(); + return; + } assert handler != null : "handler must be set before processing http content"; if (requested && chunkQueue.isEmpty()) { sendChunk(httpContent); @@ -112,4 +117,18 @@ private void releaseQueuedChunks() { } } + @Override + public void close() { + if (channel.eventLoop().inEventLoop()) { + doClose(); + } else { + channel.eventLoop().submit(this::doClose); + } + } + + private void doClose() { + closing = true; + releaseQueuedChunks(); + channel.config().setAutoRead(true); + } } diff --git a/server/src/main/java/org/elasticsearch/http/HttpBody.java b/server/src/main/java/org/elasticsearch/http/HttpBody.java index 6e4470adaec9d..f8761fac7a67f 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpBody.java +++ b/server/src/main/java/org/elasticsearch/http/HttpBody.java @@ -13,11 +13,12 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; /** * A super-interface for different HTTP content implementations */ -public sealed interface HttpBody permits HttpBody.Full, HttpBody.Stream { +public sealed interface HttpBody extends Releasable permits HttpBody.Full, HttpBody.Stream { static Full fromBytesReference(BytesReference bytesRef) { return new ByteRefHttpBody(bytesRef); @@ -56,6 +57,9 @@ default Stream asStream() { */ non-sealed interface Full extends HttpBody { BytesReference bytes(); + + @Override + default void close() {} } /** From c00768a116fcf31a1245cacced05a75ac290a69e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 28 Aug 2024 16:49:30 -0600 Subject: [PATCH 05/14] Split bulks based on memory usage (#112267) This commit splits bulks once memory usage for indexing pressure has passed a configurable threshold. --- .../action/bulk/IncrementalBulkIT.java | 74 +++++++++++ .../action/bulk/BulkOperation.java | 2 +- .../action/bulk/BulkRequest.java | 6 +- .../action/bulk/IncrementalBulkService.java | 125 +++++++++++------- .../bulk/TransportAbstractBulkAction.java | 7 +- .../common/settings/ClusterSettings.java | 1 + .../elasticsearch/index/IndexingPressure.java | 12 ++ .../elasticsearch/node/NodeConstruction.java | 2 +- .../elasticsearch/test/ESIntegTestCase.java | 4 + 9 files changed, 182 insertions(+), 51 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java index bef6c818e926b..1bae6a3e75078 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java @@ -46,6 +46,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; public class IncrementalBulkIT extends ESIntegTestCase { @@ -55,6 +56,14 @@ protected Collection> nodePlugins() { return List.of(IngestClientIT.ExtendedIngestTestPlugin.class); } + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(IndexingPressure.SPLIT_BULK_THRESHOLD.getKey(), "512B") + .build(); + } + public void testSingleBulkRequest() { String index = "test"; createIndex(index); @@ -81,6 +90,71 @@ public void testSingleBulkRequest() { assertFalse(refCounted.hasReferences()); } + public void testIndexingPressureRejection() { + String index = "test"; + createIndex(index); + + String nodeName = internalCluster().getRandomNodeName(); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, nodeName); + IndexingPressure indexingPressure = internalCluster().getInstance(IndexingPressure.class, nodeName); + + try (Releasable r = indexingPressure.markCoordinatingOperationStarted(1, indexingPressure.stats().getMemoryLimit(), true)) { + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + + if (randomBoolean()) { + AtomicBoolean nextPage = new AtomicBoolean(false); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextPage.set(true)); + assertTrue(nextPage.get()); + } + + PlainActionFuture future = new PlainActionFuture<>(); + handler.lastItems(List.of(indexRequest(index)), refCounted::decRef, future); + + expectThrows(EsRejectedExecutionException.class, future::actionGet); + assertFalse(refCounted.hasReferences()); + } + } + + public void testIncrementalBulkRequestMemoryBackOff() throws Exception { + String index = "test"; + createIndex(index); + + String nodeName = internalCluster().getRandomNodeName(); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, nodeName); + IndexingPressure indexingPressure = internalCluster().getInstance(IndexingPressure.class, nodeName); + + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + AtomicBoolean nextPage = new AtomicBoolean(false); + + IndexRequest indexRequest = indexRequest(index); + long total = indexRequest.ramBytesUsed(); + while (total < 512) { + refCounted.incRef(); + handler.addItems(List.of(indexRequest), refCounted::decRef, () -> nextPage.set(true)); + assertTrue(nextPage.get()); + nextPage.set(false); + indexRequest = indexRequest(index); + total += indexRequest.ramBytesUsed(); + } + + assertThat(indexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes(), greaterThan(0L)); + refCounted.incRef(); + handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextPage.set(true)); + + assertBusy(() -> assertThat(indexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes(), equalTo(0L))); + + PlainActionFuture future = new PlainActionFuture<>(); + handler.lastItems(List.of(indexRequest), refCounted::decRef, future); + + BulkResponse bulkResponse = future.actionGet(); + assertNoFailures(bulkResponse); + assertFalse(refCounted.hasReferences()); + } + public void testMultipleBulkPartsWithBackoff() { ExecutorService executorService = Executors.newFixedThreadPool(1); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java index d22719cbf750b..350967684f00d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java @@ -409,7 +409,7 @@ private void completeBulkOperation() { responses.toArray(new BulkItemResponse[responses.length()]), buildTookInMillis(startTimeNanos), BulkResponse.NO_INGEST_TOOK, - new BulkRequest.IncrementalState(shortCircuitShardFailures) + new BulkRequest.IncrementalState(shortCircuitShardFailures, bulkRequest.incrementalState().indexingPressureAccounted()) ) ); // Allow memory for bulk shard request items to be reclaimed before all items have been completed diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 65c7a9909057b..558901f102299 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -506,12 +506,12 @@ public Map getComponentTemplateSubstitutions() throws return Map.of(); } - record IncrementalState(Map shardLevelFailures) implements Writeable { + record IncrementalState(Map shardLevelFailures, boolean indexingPressureAccounted) implements Writeable { - static final IncrementalState EMPTY = new IncrementalState(Collections.emptyMap()); + static final IncrementalState EMPTY = new IncrementalState(Collections.emptyMap(), false); IncrementalState(StreamInput in) throws IOException { - this(in.readMap(ShardId::new, input -> input.readException())); + this(in.readMap(ShardId::new, input -> input.readException()), false); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java index 55c6b254f7e1d..529de5ee8406f 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java @@ -9,24 +9,30 @@ package org.elasticsearch.action.bulk; +import org.apache.lucene.util.Accountable; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.IndexingPressure; import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class IncrementalBulkService { private final Client client; + private final IndexingPressure indexingPressure; - public IncrementalBulkService(Client client) { + public IncrementalBulkService(Client client, IndexingPressure indexingPressure) { this.client = client; + this.indexingPressure = indexingPressure; } public Handler newBulkRequest() { @@ -34,12 +40,15 @@ public Handler newBulkRequest() { } public Handler newBulkRequest(@Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh) { - return new Handler(client, waitForActiveShards, timeout, refresh); + return new Handler(client, indexingPressure, waitForActiveShards, timeout, refresh); } public static class Handler { + public static final BulkRequest.IncrementalState EMPTY_STATE = new BulkRequest.IncrementalState(Collections.emptyMap(), true); + private final Client client; + private final IndexingPressure indexingPressure; private final ActiveShardCount waitForActiveShards; private final TimeValue timeout; private final String refresh; @@ -51,12 +60,19 @@ public static class Handler { private Exception bulkActionLevelFailure = null; private BulkRequest bulkRequest = null; - private Handler(Client client, @Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh) { + private Handler( + Client client, + IndexingPressure indexingPressure, + @Nullable String waitForActiveShards, + @Nullable TimeValue timeout, + @Nullable String refresh + ) { this.client = client; + this.indexingPressure = indexingPressure; this.waitForActiveShards = waitForActiveShards != null ? ActiveShardCount.parseString(waitForActiveShards) : null; this.timeout = timeout; this.refresh = refresh; - createNewBulkRequest(BulkRequest.IncrementalState.EMPTY); + createNewBulkRequest(EMPTY_STATE); } public void addItems(List> items, Releasable releasable, Runnable nextItems) { @@ -65,35 +81,39 @@ public void addItems(List> items, Releasable releasable, Runn nextItems.run(); } else { assert bulkRequest != null; - internalAddItems(items, releasable); - - if (shouldBackOff()) { - final boolean isFirstRequest = incrementalRequestSubmitted == false; - incrementalRequestSubmitted = true; - - client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { - - @Override - public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); - createNewBulkRequest(bulkResponse.getIncrementalState()); - } - - @Override - public void onFailure(Exception e) { - handleBulkFailure(isFirstRequest, e); - } - }, nextItems::run)); + if (internalAddItems(items, releasable)) { + if (shouldBackOff()) { + final boolean isFirstRequest = incrementalRequestSubmitted == false; + incrementalRequestSubmitted = true; + + client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { + + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + createNewBulkRequest( + new BulkRequest.IncrementalState(bulkResponse.getIncrementalState().shardLevelFailures(), true) + ); + } + + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + } + }, nextItems)); + } else { + nextItems.run(); + } } else { nextItems.run(); } + } } private boolean shouldBackOff() { - // TODO: Implement Real Memory Logic - return bulkRequest.requests().size() >= 16; + return indexingPressure.shouldSplitBulks(); } public void lastItems(List> items, Releasable releasable, ActionListener listener) { @@ -102,25 +122,27 @@ public void lastItems(List> items, Releasable releasable, Act errorResponse(listener); } else { assert bulkRequest != null; - internalAddItems(items, releasable); + if (internalAddItems(items, releasable)) { + client.bulk(bulkRequest, new ActionListener<>() { - client.bulk(bulkRequest, new ActionListener<>() { + private final boolean isFirstRequest = incrementalRequestSubmitted == false; - private final boolean isFirstRequest = incrementalRequestSubmitted == false; - - @Override - public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); - listener.onResponse(combineResponses()); - } + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + listener.onResponse(combineResponses()); + } - @Override - public void onFailure(Exception e) { - handleBulkFailure(isFirstRequest, e); - errorResponse(listener); - } - }); + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + errorResponse(listener); + } + }); + } else { + errorResponse(listener); + } } } @@ -160,9 +182,22 @@ private void addItemLevelFailures(List> items) { responses.add(new BulkResponse(bulkItemResponses, 0, 0)); } - private void internalAddItems(List> items, Releasable releasable) { - bulkRequest.add(items); - releasables.add(releasable); + private boolean internalAddItems(List> items, Releasable releasable) { + try { + bulkRequest.add(items); + releasables.add(releasable); + releasables.add( + indexingPressure.markCoordinatingOperationStarted( + items.size(), + items.stream().mapToLong(Accountable::ramBytesUsed).sum(), + false + ) + ); + return true; + } catch (EsRejectedExecutionException e) { + handleBulkFailure(incrementalRequestSubmitted == false, e); + return false; + } } private void createNewBulkRequest(BulkRequest.IncrementalState incrementalState) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportAbstractBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportAbstractBulkAction.java index d306299645d64..78652081c9f0d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportAbstractBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportAbstractBulkAction.java @@ -112,7 +112,12 @@ protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener {}; + } else { + releasable = indexingPressure.markCoordinatingOperationStarted(indexingOps, indexingBytes, isOnlySystem); + } final ActionListener releasingListener = ActionListener.runBefore(listener, releasable::close); final Executor executor = isOnlySystem ? systemWriteExecutor : writeExecutor; ensureClusterStateThenForkAndExecute(task, bulkRequest, executor, releasingListener); diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 18aaaf414101b..8471dbf36e885 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -560,6 +560,7 @@ public void apply(Settings value, Settings current, Settings previous) { FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, IndexingPressure.MAX_INDEXING_BYTES, + IndexingPressure.SPLIT_BULK_THRESHOLD, ShardLimitValidator.SETTING_CLUSTER_MAX_SHARDS_PER_NODE_FROZEN, DataTier.ENFORCE_DEFAULT_TIER_PREFERENCE_SETTING, CoordinationDiagnosticsService.IDENTITY_CHANGES_THRESHOLD_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexingPressure.java b/server/src/main/java/org/elasticsearch/index/IndexingPressure.java index 70300222883d2..14f8b92db3eaa 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexingPressure.java +++ b/server/src/main/java/org/elasticsearch/index/IndexingPressure.java @@ -30,6 +30,12 @@ public class IndexingPressure { Setting.Property.NodeScope ); + public static final Setting SPLIT_BULK_THRESHOLD = Setting.memorySizeSetting( + "indexing_pressure.memory.split_bulk_threshold", + "8.5%", + Setting.Property.NodeScope + ); + private static final Logger logger = LogManager.getLogger(IndexingPressure.class); private final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0); @@ -57,10 +63,12 @@ public class IndexingPressure { private final AtomicLong primaryDocumentRejections = new AtomicLong(0); private final long primaryAndCoordinatingLimits; + private final long splitBulkThreshold; private final long replicaLimits; public IndexingPressure(Settings settings) { this.primaryAndCoordinatingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); + this.splitBulkThreshold = SPLIT_BULK_THRESHOLD.get(settings).getBytes(); this.replicaLimits = (long) (this.primaryAndCoordinatingLimits * 1.5); } @@ -204,6 +212,10 @@ public Releasable markReplicaOperationStarted(int operations, long bytes, boolea }); } + public boolean shouldSplitBulks() { + return currentCombinedCoordinatingAndPrimaryBytes.get() >= splitBulkThreshold; + } + public IndexingPressureStats stats() { return new IndexingPressureStats( totalCombinedCoordinatingAndPrimaryBytes.get(), diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index 3f8644dda140c..b634d506aa319 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -980,7 +980,7 @@ private void construct( ); final HttpServerTransport httpServerTransport = serviceProvider.newHttpTransport(pluginsService, networkModule); final IndexingPressure indexingLimits = new IndexingPressure(settings); - final IncrementalBulkService incrementalBulkService = new IncrementalBulkService(client); + final IncrementalBulkService incrementalBulkService = new IncrementalBulkService(client, indexingLimits); SnapshotsService snapshotsService = new SnapshotsService( settings, diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index a8f269f265dc4..92e480aff3bc9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -125,6 +125,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexingPressure; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; import org.elasticsearch.index.MockEngineFactoryPlugin; @@ -2103,6 +2104,9 @@ protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { TransportSearchAction.DEFAULT_PRE_FILTER_SHARD_SIZE.getKey(), randomFrom(1, 2, SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE) ); + if (randomBoolean()) { + builder.put(IndexingPressure.SPLIT_BULK_THRESHOLD.getKey(), randomFrom("256B", "1KB", "64KB")); + } return builder.build(); } From a03fb12b0921b48ed0dc4b1b81d21b49c8443ac1 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 30 Aug 2024 12:06:04 -0600 Subject: [PATCH 06/14] Incremental bulk integration with rest layer (#112154) Integrate the incremental bulks into RestBulkAction --- .../test/rest/RequestsWithoutContentIT.java | 1 + .../netty4/Netty4HttpRequestSizeLimitIT.java | 3 + .../http/netty4/Netty4HttpAggregator.java | 6 +- .../netty4/Netty4HttpServerTransport.java | 18 +- .../Netty4HttpServerTransportTests.java | 4 +- .../http/IncrementalBulkRestIT.java | 117 +++++++++ .../elasticsearch/action/ActionModule.java | 8 +- .../action/bulk/BulkRequestParser.java | 58 ++++- .../action/bulk/IncrementalBulkService.java | 137 ++++++++--- .../common/settings/ClusterSettings.java | 2 + .../elasticsearch/node/NodeConstruction.java | 13 +- .../rest/action/document/RestBulkAction.java | 229 +++++++++++++++--- .../action/ActionModuleTests.java | 16 +- .../AbstractHttpServerTransportTests.java | 4 +- .../action/document/RestBulkActionTests.java | 150 +++++++++++- .../test/rest/FakeRestRequest.java | 5 + .../xpack/security/SecurityTests.java | 4 +- 17 files changed, 675 insertions(+), 100 deletions(-) create mode 100644 qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java diff --git a/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java b/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java index c95c4c1d198f2..83cb4fb7180ef 100644 --- a/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java +++ b/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java @@ -26,6 +26,7 @@ public void testIndexMissingBody() throws IOException { assertResponseException(responseException, "request body is required"); } + @AwaitsFix(bugUrl = "need to decide how to handle this scenario") public void testBulkMissingBody() throws IOException { ResponseException responseException = expectThrows( ResponseException.class, diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java index d9cfe009718b7..d4e5746a6159f 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java @@ -21,6 +21,7 @@ import org.elasticsearch.core.Tuple; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -52,6 +53,8 @@ protected boolean addMockHttpTransport() { protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal, otherSettings)) + // TODO: We do not currently support in flight circuit breaker limits for bulk. However, IndexingPressure applies + .put(RestBulkAction.INCREMENTAL_BULK.getKey(), false) .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), LIMIT) .build(); } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java index bab01f047ee9e..3c9e684ef4279 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpAggregator.java @@ -36,10 +36,6 @@ public class Netty4HttpAggregator extends HttpObjectAggregator { private boolean aggregating = true; private boolean ignoreContentAfterContinueResponse = false; - public Netty4HttpAggregator(int maxContentLength) { - this(maxContentLength, IGNORE_TEST); - } - public Netty4HttpAggregator(int maxContentLength, Predicate decider) { super(maxContentLength); this.decider = decider; @@ -50,7 +46,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception assert msg instanceof HttpObject; if (msg instanceof HttpRequest request) { var preReq = HttpHeadersAuthenticatorUtils.asHttpPreRequest(request); - aggregating = decider.test(preReq); + aggregating = decider.test(preReq) && IGNORE_TEST.test(preReq); } if (aggregating || msg instanceof FullHttpRequest) { super.channelRead(ctx, msg); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java index 129c3b9c3d33c..c6e7fa3517771 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java @@ -37,6 +37,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.common.network.CloseableChannel; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.ThreadWatchdog; @@ -97,6 +98,7 @@ public class Netty4HttpServerTransport extends AbstractHttpServerTransport { private final TLSConfig tlsConfig; private final AcceptChannelHandler.AcceptPredicate acceptChannelPredicate; private final HttpValidator httpValidator; + private final IncrementalBulkService.Enabled enabled; private final ThreadWatchdog threadWatchdog; private final int readTimeoutMillis; @@ -135,6 +137,7 @@ public Netty4HttpServerTransport( this.acceptChannelPredicate = acceptChannelPredicate; this.httpValidator = httpValidator; this.threadWatchdog = networkService.getThreadWatchdog(); + this.enabled = new IncrementalBulkService.Enabled(clusterSettings); this.pipeliningMaxEvents = SETTING_PIPELINING_MAX_EVENTS.get(settings); @@ -280,7 +283,7 @@ public void onException(HttpChannel channel, Exception cause) { } public ChannelHandler configureServerChannelHandler() { - return new HttpChannelHandler(this, handlingSettings, tlsConfig, acceptChannelPredicate, httpValidator); + return new HttpChannelHandler(this, handlingSettings, tlsConfig, acceptChannelPredicate, httpValidator, enabled); } static final AttributeKey HTTP_CHANNEL_KEY = AttributeKey.newInstance("es-http-channel"); @@ -293,19 +296,22 @@ protected static class HttpChannelHandler extends ChannelInitializer { private final TLSConfig tlsConfig; private final BiPredicate acceptChannelPredicate; private final HttpValidator httpValidator; + private final IncrementalBulkService.Enabled enabled; protected HttpChannelHandler( final Netty4HttpServerTransport transport, final HttpHandlingSettings handlingSettings, final TLSConfig tlsConfig, @Nullable final BiPredicate acceptChannelPredicate, - @Nullable final HttpValidator httpValidator + @Nullable final HttpValidator httpValidator, + IncrementalBulkService.Enabled enabled ) { this.transport = transport; this.handlingSettings = handlingSettings; this.tlsConfig = tlsConfig; this.acceptChannelPredicate = acceptChannelPredicate; this.httpValidator = httpValidator; + this.enabled = enabled; } @Override @@ -366,7 +372,13 @@ protected HttpMessage createMessage(String[] initialLine) throws Exception { ); } // combines the HTTP message pieces into a single full HTTP request (with headers and body) - final HttpObjectAggregator aggregator = new Netty4HttpAggregator(handlingSettings.maxContentLength()); + final HttpObjectAggregator aggregator = new Netty4HttpAggregator( + handlingSettings.maxContentLength(), + httpPreRequest -> enabled.get() == false + || (httpPreRequest.uri().contains("_bulk") == false + || httpPreRequest.uri().contains("_bulk_update") + || httpPreRequest.uri().contains("/_xpack/monitoring/_bulk")) + ); aggregator.setMaxCumulationBufferComponents(transport.maxCompositeBufferComponents); ch.pipeline() .addLast("decoder_compress", new HttpContentDecompressor()) // this handles request body decompression diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java index fae40c03b1b8d..3fd5cc44a3403 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java @@ -46,6 +46,7 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ElasticsearchWrapperException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.client.Request; @@ -419,7 +420,8 @@ public ChannelHandler configureServerChannelHandler() { handlingSettings, TLSConfig.noTLS(), null, - randomFrom((httpPreRequest, channel, listener) -> listener.onResponse(null), null) + randomFrom((httpPreRequest, channel, listener) -> listener.onResponse(null), null), + new IncrementalBulkService.Enabled(clusterSettings) ) { @Override protected void initChannel(Channel ch) throws Exception { diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java new file mode 100644 index 0000000000000..fcb0c1044d22f --- /dev/null +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java @@ -0,0 +1,117 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.xcontent.json.JsonXContent; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.rest.RestStatus.OK; +import static org.hamcrest.Matchers.equalTo; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, supportsDedicatedMasters = false, numDataNodes = 2, numClientNodes = 0) +public class IncrementalBulkRestIT extends HttpSmokeTestCase { + + @SuppressWarnings("unchecked") + public void testIncrementalBulk() throws IOException { + Request createRequest = new Request("PUT", "/index_name"); + createRequest.setJsonEntity(""" + { + "settings": { + "index": { + "number_of_shards": 1, + "number_of_replicas": 1, + "write.wait_for_active_shards": 2 + } + } + }"""); + final Response indexCreatedResponse = getRestClient().performRequest(createRequest); + assertThat(indexCreatedResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + + Request firstBulkRequest = new Request("POST", "/index_name/_bulk"); + + // index documents for the rollup job + String bulkBody = "{\"index\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n" + + "{\"field\":1}\n" + + "{\"index\":{\"_index\":\"index_name\",\"_id\":\"2\"}}\n" + + "{\"field\":1}\n" + + "\r\n"; + + firstBulkRequest.setJsonEntity(bulkBody); + + final Response indexSuccessFul = getRestClient().performRequest(firstBulkRequest); + assertThat(indexSuccessFul.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + + Request bulkRequest = new Request("POST", "/index_name/_bulk"); + + // index documents for the rollup job + final StringBuilder bulk = new StringBuilder(); + bulk.append("{\"delete\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n"); + int updates = 0; + for (int i = 0; i < 1000; i++) { + bulk.append("{\"index\":{\"_index\":\"index_name\"}}\n"); + bulk.append("{\"field\":").append(i).append("}\n"); + if (randomBoolean() && randomBoolean() && randomBoolean() && randomBoolean()) { + ++updates; + bulk.append("{\"update\":{\"_index\":\"index_name\",\"_id\":\"2\"}}\n"); + bulk.append("{\"doc\":{\"field\":").append(i).append("}}\n"); + } + } + bulk.append("\r\n"); + + bulkRequest.setJsonEntity(bulk.toString()); + + final Response bulkResponse = getRestClient().performRequest(bulkRequest); + assertThat(bulkResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + Map responseMap = XContentHelper.convertToMap( + JsonXContent.jsonXContent, + bulkResponse.getEntity().getContent(), + true + ); + + assertFalse((Boolean) responseMap.get("errors")); + assertThat(((List) responseMap.get("items")).size(), equalTo(1001 + updates)); + } + + public void testIncrementalMalformed() throws IOException { + Request createRequest = new Request("PUT", "/index_name"); + createRequest.setJsonEntity(""" + { + "settings": { + "index": { + "number_of_shards": 1, + "number_of_replicas": 1, + "write.wait_for_active_shards": 2 + } + } + }"""); + final Response indexCreatedResponse = getRestClient().performRequest(createRequest); + assertThat(indexCreatedResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + + Request bulkRequest = new Request("POST", "/index_name/_bulk"); + + // index documents for the rollup job + final StringBuilder bulk = new StringBuilder(); + bulk.append("{\"index\":{\"_index\":\"index_name\"}}\n"); + bulk.append("{\"field\":1}\n"); + bulk.append("{}\n"); + bulk.append("\r\n"); + + bulkRequest.setJsonEntity(bulk.toString()); + + expectThrows(ResponseException.class, () -> getRestClient().performRequest(bulkRequest)); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index 6c736b47bc94c..2d72f5d71ccda 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -160,6 +160,7 @@ import org.elasticsearch.action.admin.indices.template.put.TransportPutIndexTemplateAction; import org.elasticsearch.action.admin.indices.validate.query.TransportValidateQueryAction; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.bulk.SimulateBulkAction; import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.bulk.TransportShardBulkAction; @@ -448,6 +449,7 @@ public class ActionModule extends AbstractModule { private final List actionPlugins; private final Map> actions; private final ActionFilters actionFilters; + private final IncrementalBulkService bulkService; private final AutoCreateIndex autoCreateIndex; private final DestructiveOperations destructiveOperations; private final RestController restController; @@ -476,7 +478,8 @@ public ActionModule( ClusterService clusterService, RerouteService rerouteService, List> reservedStateHandlers, - RestExtension restExtension + RestExtension restExtension, + IncrementalBulkService bulkService ) { this.settings = settings; this.indexNameExpressionResolver = indexNameExpressionResolver; @@ -488,6 +491,7 @@ public ActionModule( this.threadPool = threadPool; actions = setupActions(actionPlugins); actionFilters = setupActionFilters(actionPlugins); + this.bulkService = bulkService; autoCreateIndex = new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver, systemIndices); destructiveOperations = new DestructiveOperations(settings, clusterSettings); Set headers = Stream.concat( @@ -928,7 +932,7 @@ public void initRestHandlers(Supplier nodesInCluster, Predicate< registerHandler.accept(new RestCountAction()); registerHandler.accept(new RestTermVectorsAction()); registerHandler.accept(new RestMultiTermVectorsAction()); - registerHandler.accept(new RestBulkAction(settings)); + registerHandler.accept(new RestBulkAction(settings, bulkService)); registerHandler.accept(new RestUpdateAction()); registerHandler.accept(new RestSearchAction(restController.getSearchUsageHolder(), clusterSupportsFeature)); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index e94bfff69d3d1..c27e3d319d7ca 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -86,13 +86,13 @@ public BulkRequestParser(boolean deprecateOrErrorOnType, RestApiVersion restApiV .withRestApiVersion(restApiVersion); } - private static int findNextMarker(byte marker, int from, BytesReference data) { + private static int findNextMarker(byte marker, int from, BytesReference data, boolean isIncremental) { final int res = data.indexOf(marker, from); if (res != -1) { assert res >= 0; return res; } - if (from != data.length()) { + if (from != data.length() && isIncremental == false) { throw new IllegalArgumentException("The bulk request must be terminated by a newline [\\n]"); } return res; @@ -137,18 +137,57 @@ public void parse( Consumer updateRequestConsumer, Consumer deleteRequestConsumer ) throws IOException { - XContent xContent = xContentType.xContent(); - int line = 0; - int from = 0; - byte marker = xContent.bulkSeparator(); // Bulk requests can contain a lot of repeated strings for the index, pipeline and routing parameters. This map is used to // deduplicate duplicate strings parsed for these parameters. While it does not prevent instantiating the duplicate strings, it // reduces their lifetime to the lifetime of this parse call instead of the lifetime of the full bulk request. final Map stringDeduplicator = new HashMap<>(); + + incrementalParse( + data, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer, + false, + stringDeduplicator + ); + } + + public int incrementalParse( + BytesReference data, + String defaultIndex, + String defaultRouting, + FetchSourceContext defaultFetchSourceContext, + String defaultPipeline, + Boolean defaultRequireAlias, + Boolean defaultRequireDataStream, + Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer, + boolean isIncremental, + Map stringDeduplicator + ) throws IOException { + XContent xContent = xContentType.xContent(); + byte marker = xContent.bulkSeparator(); boolean typesDeprecationLogged = false; + int line = 0; + int from = 0; + int consumed = 0; + while (true) { - int nextMarker = findNextMarker(marker, from, data); + int nextMarker = findNextMarker(marker, from, data, isIncremental); if (nextMarker == -1) { break; } @@ -333,8 +372,9 @@ public void parse( .setIfSeqNo(ifSeqNo) .setIfPrimaryTerm(ifPrimaryTerm) ); + consumed = from; } else { - nextMarker = findNextMarker(marker, from, data); + nextMarker = findNextMarker(marker, from, data, isIncremental); if (nextMarker == -1) { break; } @@ -407,9 +447,11 @@ public void parse( } // move pointers from = nextMarker + 1; + consumed = from; } } } + return isIncremental ? consumed : from; } @UpdateForV9 diff --git a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java index 529de5ee8406f..8ecb0a198fe10 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java @@ -14,25 +14,56 @@ import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexingPressure; +import org.elasticsearch.rest.action.document.RestBulkAction; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; public class IncrementalBulkService { private final Client client; private final IndexingPressure indexingPressure; + private final ThreadContext threadContext; + private final Supplier enabled; - public IncrementalBulkService(Client client, IndexingPressure indexingPressure) { + public IncrementalBulkService(Client client, IndexingPressure indexingPressure, ThreadContext threadContext) { + this(client, indexingPressure, threadContext, new Enabled()); + } + + public IncrementalBulkService( + Client client, + IndexingPressure indexingPressure, + ThreadContext threadContext, + ClusterSettings clusterSettings + ) { + this(client, indexingPressure, threadContext, new Enabled(clusterSettings)); + } + + public IncrementalBulkService( + Client client, + IndexingPressure indexingPressure, + ThreadContext threadContext, + Supplier enabled + ) { this.client = client; this.indexingPressure = indexingPressure; + this.threadContext = threadContext; + this.enabled = enabled; + } + + public boolean incrementalBulkEnabled() { + return enabled.get(); } public Handler newBulkRequest() { @@ -40,14 +71,32 @@ public Handler newBulkRequest() { } public Handler newBulkRequest(@Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh) { - return new Handler(client, indexingPressure, waitForActiveShards, timeout, refresh); + return new Handler(client, threadContext, indexingPressure, waitForActiveShards, timeout, refresh); } - public static class Handler { + public static class Enabled implements Supplier { + + private final AtomicBoolean incrementalBulksEnabled = new AtomicBoolean(true); + + public Enabled() {} + + public Enabled(ClusterSettings clusterSettings) { + incrementalBulksEnabled.set(clusterSettings.get(RestBulkAction.INCREMENTAL_BULK)); + clusterSettings.addSettingsUpdateConsumer(RestBulkAction.INCREMENTAL_BULK, incrementalBulksEnabled::set); + } + + @Override + public Boolean get() { + return incrementalBulksEnabled.get(); + } + } + + public static class Handler implements Releasable { public static final BulkRequest.IncrementalState EMPTY_STATE = new BulkRequest.IncrementalState(Collections.emptyMap(), true); private final Client client; + private final ThreadContext threadContext; private final IndexingPressure indexingPressure; private final ActiveShardCount waitForActiveShards; private final TimeValue timeout; @@ -57,17 +106,21 @@ public static class Handler { private final ArrayList responses = new ArrayList<>(2); private boolean globalFailure = false; private boolean incrementalRequestSubmitted = false; + private ThreadContext.StoredContext requestContext; private Exception bulkActionLevelFailure = null; private BulkRequest bulkRequest = null; - private Handler( + protected Handler( Client client, + ThreadContext threadContext, IndexingPressure indexingPressure, @Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh ) { this.client = client; + this.threadContext = threadContext; + this.requestContext = threadContext.newStoredContext(); this.indexingPressure = indexingPressure; this.waitForActiveShards = waitForActiveShards != null ? ActiveShardCount.parseString(waitForActiveShards) : null; this.timeout = timeout; @@ -85,30 +138,34 @@ public void addItems(List> items, Releasable releasable, Runn if (shouldBackOff()) { final boolean isFirstRequest = incrementalRequestSubmitted == false; incrementalRequestSubmitted = true; - - client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { - - @Override - public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); - createNewBulkRequest( - new BulkRequest.IncrementalState(bulkResponse.getIncrementalState().shardLevelFailures(), true) - ); - } - - @Override - public void onFailure(Exception e) { - handleBulkFailure(isFirstRequest, e); - } - }, nextItems)); + try (ThreadContext.StoredContext ignored = threadContext.stashContext()) { + requestContext.restore(); + client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { + + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + createNewBulkRequest( + new BulkRequest.IncrementalState(bulkResponse.getIncrementalState().shardLevelFailures(), true) + ); + } + + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + } + }, () -> { + requestContext = threadContext.newStoredContext(); + nextItems.run(); + })); + } } else { nextItems.run(); } } else { nextItems.run(); } - } } @@ -123,23 +180,26 @@ public void lastItems(List> items, Releasable releasable, Act } else { assert bulkRequest != null; if (internalAddItems(items, releasable)) { - client.bulk(bulkRequest, new ActionListener<>() { + try (ThreadContext.StoredContext ignored = threadContext.stashContext()) { + requestContext.restore(); + client.bulk(bulkRequest, new ActionListener<>() { - private final boolean isFirstRequest = incrementalRequestSubmitted == false; + private final boolean isFirstRequest = incrementalRequestSubmitted == false; - @Override - public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); - listener.onResponse(combineResponses()); - } + @Override + public void onResponse(BulkResponse bulkResponse) { + responses.add(bulkResponse); + releaseCurrentReferences(); + listener.onResponse(combineResponses()); + } - @Override - public void onFailure(Exception e) { - handleBulkFailure(isFirstRequest, e); - errorResponse(listener); - } - }); + @Override + public void onFailure(Exception e) { + handleBulkFailure(isFirstRequest, e); + errorResponse(listener); + } + }); + } } else { errorResponse(listener); } @@ -240,5 +300,10 @@ private BulkResponse combineResponses() { return new BulkResponse(bulkItemResponses, tookInMillis, ingestTookInMillis); } + + @Override + public void close() { + // TODO: Implement + } } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 8471dbf36e885..35c0900b0dc2b 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -113,6 +113,7 @@ import org.elasticsearch.readiness.ReadinessService; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchService; @@ -242,6 +243,7 @@ public void apply(Settings value, Settings current, Settings previous) { Metadata.SETTING_READ_ONLY_SETTING, Metadata.SETTING_READ_ONLY_ALLOW_DELETE_SETTING, ShardLimitValidator.SETTING_CLUSTER_MAX_SHARDS_PER_NODE, + RestBulkAction.INCREMENTAL_BULK, RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index b634d506aa319..6560d6d6849f9 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -891,6 +891,14 @@ private void construct( .map(TerminationHandlerProvider::handler); terminationHandler = getSinglePlugin(terminationHandlers, TerminationHandler.class).orElse(null); + final IndexingPressure indexingLimits = new IndexingPressure(settings); + final IncrementalBulkService incrementalBulkService = new IncrementalBulkService( + client, + indexingLimits, + threadPool.getThreadContext(), + clusterService.getClusterSettings() + ); + ActionModule actionModule = new ActionModule( settings, clusterModule.getIndexNameExpressionResolver(), @@ -916,7 +924,8 @@ private void construct( metadataCreateIndexService, dataStreamGlobalRetentionSettings ), - pluginsService.loadSingletonServiceProvider(RestExtension.class, RestExtension::allowAll) + pluginsService.loadSingletonServiceProvider(RestExtension.class, RestExtension::allowAll), + incrementalBulkService ); modules.add(actionModule); @@ -979,8 +988,6 @@ private void construct( SearchExecutionStatsCollector.makeWrapper(responseCollectorService) ); final HttpServerTransport httpServerTransport = serviceProvider.newHttpTransport(pluginsService, networkModule); - final IndexingPressure indexingLimits = new IndexingPressure(settings); - final IncrementalBulkService incrementalBulkService = new IncrementalBulkService(client, indexingLimits); SnapshotsService snapshotsService = new SnapshotsService( settings, diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 74009401f02c9..53cd97d15bb82 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -11,21 +11,38 @@ import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkRequestParser; import org.elasticsearch.action.bulk.BulkShardRequest; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.CompositeBytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.Scope; import org.elasticsearch.rest.ServerlessScope; import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener; +import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; @@ -40,12 +57,21 @@ */ @ServerlessScope(Scope.PUBLIC) public class RestBulkAction extends BaseRestHandler { + public static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Specifying types in bulk requests is deprecated."; + public static final Setting INCREMENTAL_BULK = boolSetting( + "rest.incremental_bulk", + true, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); private final boolean allowExplicitIndex; + private final IncrementalBulkService bulkHandler; - public RestBulkAction(Settings settings) { + public RestBulkAction(Settings settings, IncrementalBulkService bulkHandler) { this.allowExplicitIndex = MULTI_ALLOW_EXPLICIT_INDEX.get(settings); + this.bulkHandler = bulkHandler; } @Override @@ -67,38 +93,181 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - if (request.getRestApiVersion() == RestApiVersion.V_7 && request.hasParam("type")) { - request.param("type"); + if (bulkHandler.incrementalBulkEnabled() == false) { + if (request.getRestApiVersion() == RestApiVersion.V_7 && request.hasParam("type")) { + request.param("type"); + } + BulkRequest bulkRequest = new BulkRequest(); + String defaultIndex = request.param("index"); + String defaultRouting = request.param("routing"); + FetchSourceContext defaultFetchSourceContext = FetchSourceContext.parseFromRestRequest(request); + String defaultPipeline = request.param("pipeline"); + boolean defaultListExecutedPipelines = request.paramAsBoolean("list_executed_pipelines", false); + String waitForActiveShards = request.param("wait_for_active_shards"); + if (waitForActiveShards != null) { + bulkRequest.waitForActiveShards(ActiveShardCount.parseString(waitForActiveShards)); + } + Boolean defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false); + boolean defaultRequireDataStream = request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false); + bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); + bulkRequest.setRefreshPolicy(request.param("refresh")); + bulkRequest.add( + request.requiredContent(), + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + request.getXContentType(), + request.getRestApiVersion() + ); + + return channel -> client.bulk(bulkRequest, new RestRefCountedChunkedToXContentListener<>(channel)); + } else { + if (request.getRestApiVersion() == RestApiVersion.V_7 && request.hasParam("type")) { + request.param("type"); + } + + String waitForActiveShards = request.param("wait_for_active_shards"); + TimeValue timeout = request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT); + String refresh = request.param("refresh"); + return new ChunkHandler(allowExplicitIndex, request, () -> bulkHandler.newBulkRequest(waitForActiveShards, timeout, refresh)); } - BulkRequest bulkRequest = new BulkRequest(); - String defaultIndex = request.param("index"); - String defaultRouting = request.param("routing"); - FetchSourceContext defaultFetchSourceContext = FetchSourceContext.parseFromRestRequest(request); - String defaultPipeline = request.param("pipeline"); - boolean defaultListExecutedPipelines = request.paramAsBoolean("list_executed_pipelines", false); - String waitForActiveShards = request.param("wait_for_active_shards"); - if (waitForActiveShards != null) { - bulkRequest.waitForActiveShards(ActiveShardCount.parseString(waitForActiveShards)); + } + + static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { + + private final boolean allowExplicitIndex; + private final RestRequest request; + + private final Map stringDeduplicator = new HashMap<>(); + private final String defaultIndex; + private final String defaultRouting; + private final FetchSourceContext defaultFetchSourceContext; + private final String defaultPipeline; + private final boolean defaultListExecutedPipelines; + private final Boolean defaultRequireAlias; + private final boolean defaultRequireDataStream; + private final BulkRequestParser parser; + private final Supplier handlerSupplier; + private IncrementalBulkService.Handler handler; + + private volatile RestChannel restChannel; + private boolean isException; + private final ArrayDeque unParsedChunks = new ArrayDeque<>(4); + private final ArrayList> items = new ArrayList<>(4); + + ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { + this.allowExplicitIndex = allowExplicitIndex; + this.request = request; + this.defaultIndex = request.param("index"); + this.defaultRouting = request.param("routing"); + this.defaultFetchSourceContext = FetchSourceContext.parseFromRestRequest(request); + this.defaultPipeline = request.param("pipeline"); + this.defaultListExecutedPipelines = request.paramAsBoolean("list_executed_pipelines", false); + this.defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false); + this.defaultRequireDataStream = request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false); + // TODO: Fix type deprecation logging + this.parser = new BulkRequestParser(false, request.getRestApiVersion()); + this.handlerSupplier = handlerSupplier; } - Boolean defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false); - boolean defaultRequireDataStream = request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false); - bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); - bulkRequest.setRefreshPolicy(request.param("refresh")); - bulkRequest.add( - request.requiredContent(), - defaultIndex, - defaultRouting, - defaultFetchSourceContext, - defaultPipeline, - defaultRequireAlias, - defaultRequireDataStream, - defaultListExecutedPipelines, - allowExplicitIndex, - request.getXContentType(), - request.getRestApiVersion() - ); - return channel -> client.bulk(bulkRequest, new RestRefCountedChunkedToXContentListener<>(channel)); + @Override + public void accept(RestChannel restChannel) { + this.restChannel = restChannel; + this.handler = handlerSupplier.get(); + request.contentStream().next(); + } + + @Override + public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boolean isLast) { + assert handler != null; + assert channel == restChannel; + if (isException) { + chunk.close(); + return; + } + + final BytesReference data; + int bytesConsumed; + try { + unParsedChunks.add(chunk); + + if (unParsedChunks.size() > 1) { + data = CompositeBytesReference.of(unParsedChunks.toArray(new ReleasableBytesReference[0])); + } else { + data = chunk; + } + + // TODO: Check that the behavior here vs. globalRouting, globalPipeline, globalRequireAlias, globalRequireDatsStream in + // BulkRequest#add is fine + bytesConsumed = parser.incrementalParse( + data, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + request.getXContentType(), + (request, type) -> items.add(request), + items::add, + items::add, + isLast == false, + stringDeduplicator + ); + + } catch (Exception e) { + // TODO: This needs to be better + Releasables.close(handler); + Releasables.close(unParsedChunks); + unParsedChunks.clear(); + new RestToXContentListener<>(channel).onFailure(e); + isException = true; + return; + } + + final ArrayList releasables = accountParsing(bytesConsumed); + if (isLast) { + assert unParsedChunks.isEmpty(); + assert channel != null; + ArrayList> toPass = new ArrayList<>(items); + items.clear(); + handler.lastItems(toPass, () -> Releasables.close(releasables), new RestRefCountedChunkedToXContentListener<>(channel)); + } else if (items.isEmpty() == false) { + ArrayList> toPass = new ArrayList<>(items); + items.clear(); + handler.addItems(toPass, () -> Releasables.close(releasables), () -> request.contentStream().next()); + } else { + assert releasables.isEmpty(); + request.contentStream().next(); + } + } + + @Override + public void close() { + RequestBodyChunkConsumer.super.close(); + } + + private ArrayList accountParsing(int bytesConsumed) { + ArrayList releasables = new ArrayList<>(unParsedChunks.size()); + while (bytesConsumed > 0) { + ReleasableBytesReference reference = unParsedChunks.removeFirst(); + releasables.add(reference); + if (bytesConsumed >= reference.length()) { + bytesConsumed -= reference.length(); + } else { + unParsedChunks.addFirst(reference.retainedSlice(bytesConsumed, reference.length() - bytesConsumed)); + bytesConsumed = 0; + } + } + return releasables; + } } @Override diff --git a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java index 4ed493a94e20e..871062a687429 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java @@ -10,6 +10,7 @@ package org.elasticsearch.action; import org.elasticsearch.action.admin.cluster.node.info.TransportNodesInfoAction; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.client.internal.node.NodeClient; @@ -129,7 +130,8 @@ public void testSetupRestHandlerContainsKnownBuiltin() { mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ); actionModule.initRestHandlers(null, null); // At this point the easiest way to confirm that a handler is loaded is to try to register another one on top of it and to fail @@ -193,7 +195,8 @@ public String getName() { mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ); Exception e = expectThrows(IllegalArgumentException.class, () -> actionModule.initRestHandlers(null, null)); assertThat(e.getMessage(), startsWith("Cannot replace existing handler for [/_nodes] for method: GET")); @@ -250,7 +253,8 @@ public List getRestHandlers( mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ); actionModule.initRestHandlers(null, null); // At this point the easiest way to confirm that a handler is loaded is to try to register another one on top of it and to fail @@ -300,7 +304,8 @@ public void test3rdPartyHandlerIsNotInstalled() { mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ) ); assertThat( @@ -341,7 +346,8 @@ public void test3rdPartyRestControllerIsNotInstalled() { mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ) ); assertThat( diff --git a/server/src/test/java/org/elasticsearch/http/AbstractHttpServerTransportTests.java b/server/src/test/java/org/elasticsearch/http/AbstractHttpServerTransportTests.java index c842dbd294b65..981eae9d60694 100644 --- a/server/src/test/java/org/elasticsearch/http/AbstractHttpServerTransportTests.java +++ b/server/src/test/java/org/elasticsearch/http/AbstractHttpServerTransportTests.java @@ -15,6 +15,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionModule; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; @@ -1177,7 +1178,8 @@ public Collection getRestHeaders() { mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ); } diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 772ff0efb1218..44fd1e721f8d2 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -11,23 +11,37 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.IndexingPressure; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.client.NoOpNodeClient; +import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.xcontent.XContentType; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.mockito.Mockito.mock; @@ -51,7 +65,10 @@ public void bulk(BulkRequest request, ActionListener listener) { }; final Map params = new HashMap<>(); params.put("pipeline", "timestamps"); - new RestBulkAction(settings(IndexVersion.current()).build()).handleRequest( + new RestBulkAction( + settings(IndexVersion.current()).build(), + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY), () -> false) + ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk").withParams(params).withContent(new BytesArray(""" {"index":{"_id":"1"}} {"field1":"val1"} @@ -83,7 +100,15 @@ public void bulk(BulkRequest request, ActionListener listener) { }; Map params = new HashMap<>(); { - new RestBulkAction(settings(IndexVersion.current()).build()).handleRequest( + new RestBulkAction( + settings(IndexVersion.current()).build(), + new IncrementalBulkService( + mock(Client.class), + mock(IndexingPressure.class), + new ThreadContext(Settings.EMPTY), + () -> false + ) + ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) .withContent(new BytesArray(""" @@ -104,7 +129,15 @@ public void bulk(BulkRequest request, ActionListener listener) { { params.put("list_executed_pipelines", "true"); bulkCalled.set(false); - new RestBulkAction(settings(IndexVersion.current()).build()).handleRequest( + new RestBulkAction( + settings(IndexVersion.current()).build(), + new IncrementalBulkService( + mock(Client.class), + mock(IndexingPressure.class), + new ThreadContext(Settings.EMPTY), + () -> false + ) + ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) .withContent(new BytesArray(""" @@ -124,7 +157,15 @@ public void bulk(BulkRequest request, ActionListener listener) { } { bulkCalled.set(false); - new RestBulkAction(settings(IndexVersion.current()).build()).handleRequest( + new RestBulkAction( + settings(IndexVersion.current()).build(), + new IncrementalBulkService( + mock(Client.class), + mock(IndexingPressure.class), + new ThreadContext(Settings.EMPTY), + () -> false + ) + ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) .withContent(new BytesArray(""" @@ -145,7 +186,15 @@ public void bulk(BulkRequest request, ActionListener listener) { { params.remove("list_executed_pipelines"); bulkCalled.set(false); - new RestBulkAction(settings(IndexVersion.current()).build()).handleRequest( + new RestBulkAction( + settings(IndexVersion.current()).build(), + new IncrementalBulkService( + mock(Client.class), + mock(IndexingPressure.class), + new ThreadContext(Settings.EMPTY), + () -> false + ) + ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) .withContent(new BytesArray(""" @@ -165,4 +214,95 @@ public void bulk(BulkRequest request, ActionListener listener) { } } } + + public void testIncrementalParsing() { + ArrayList> docs = new ArrayList<>(); + AtomicBoolean isLast = new AtomicBoolean(false); + AtomicBoolean next = new AtomicBoolean(false); + + FakeRestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") + .withMethod(RestRequest.Method.POST) + .withBody(new HttpBody.Stream() { + @Override + public void close() {} + + @Override + public ChunkHandler handler() { + return null; + } + + @Override + public void setHandler(ChunkHandler chunkHandler) {} + + @Override + public void next() { + next.set(true); + } + }) + .withHeaders(Map.of("Content-Type", Collections.singletonList("application/json"))) + .build(); + FakeRestChannel channel = new FakeRestChannel(request, false, 1); + + RestBulkAction.ChunkHandler chunkHandler = new RestBulkAction.ChunkHandler( + true, + request, + () -> new IncrementalBulkService.Handler(null, new ThreadContext(Settings.EMPTY), null, null, null, null) { + + @Override + public void addItems(List> items, Releasable releasable, Runnable nextItems) { + releasable.close(); + docs.addAll(items); + } + + @Override + public void lastItems(List> items, Releasable releasable, ActionListener listener) { + releasable.close(); + docs.addAll(items); + isLast.set(true); + } + } + ); + + chunkHandler.accept(channel); + ReleasableBytesReference r1 = new ReleasableBytesReference(new BytesArray("{\"index\":{\"_index\":\"index_name\"}}\n"), () -> {}); + chunkHandler.handleChunk(channel, r1, false); + assertThat(docs, empty()); + assertTrue(next.get()); + next.set(false); + assertFalse(isLast.get()); + + ReleasableBytesReference r2 = new ReleasableBytesReference(new BytesArray("{\"field\":1}"), () -> {}); + chunkHandler.handleChunk(channel, r2, false); + assertThat(docs, empty()); + assertTrue(next.get()); + next.set(false); + assertFalse(isLast.get()); + assertTrue(r1.hasReferences()); + assertTrue(r2.hasReferences()); + + ReleasableBytesReference r3 = new ReleasableBytesReference(new BytesArray("\n{\"delete\":"), () -> {}); + chunkHandler.handleChunk(channel, r3, false); + assertThat(docs, hasSize(1)); + assertFalse(next.get()); + assertFalse(isLast.get()); + assertFalse(r1.hasReferences()); + assertFalse(r2.hasReferences()); + assertTrue(r3.hasReferences()); + + ReleasableBytesReference r4 = new ReleasableBytesReference(new BytesArray("{\"_index\":\"test\",\"_id\":\"2\"}}"), () -> {}); + chunkHandler.handleChunk(channel, r4, false); + assertThat(docs, hasSize(1)); + assertTrue(next.get()); + next.set(false); + assertFalse(isLast.get()); + + ReleasableBytesReference r5 = new ReleasableBytesReference(new BytesArray("\n"), () -> {}); + chunkHandler.handleChunk(channel, r5, true); + assertThat(docs, hasSize(2)); + assertFalse(next.get()); + assertTrue(isLast.get()); + assertFalse(r3.hasReferences()); + assertFalse(r4.hasReferences()); + assertFalse(r5.hasReferences()); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java index 0595e6cbb1c97..9ddcf39d24d98 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestRequest.java @@ -229,6 +229,11 @@ public Builder withContent(BytesReference contentBytes, XContentType xContentTyp return this; } + public Builder withBody(HttpBody body) { + this.content = body; + return this; + } + public Builder withPath(String path) { this.path = path; return this; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index a07a7a3a5dd27..8d580f10e5137 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -822,7 +823,8 @@ public void testSecurityRestHandlerInterceptorCanBeInstalled() throws IllegalAcc mock(ClusterService.class), null, List.of(), - RestExtension.allowAll() + RestExtension.allowAll(), + new IncrementalBulkService(null, null, new ThreadContext(Settings.EMPTY)) ); actionModule.initRestHandlers(null, null); From 95b42a7129af5d8ba9ce85d31ecf07e7569140a8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 5 Sep 2024 10:45:51 -0600 Subject: [PATCH 07/14] Ensure incremental bulk setting is set atomically (#112479) Currently the rest.incremental_bulk is read in two different places. This means that it will be employed in two steps introducing unpredictable behavior. This commit ensures that it is only read in a single place. --- .../netty4/Netty4HttpRequestSizeLimitIT.java | 4 +- .../http/IncrementalBulkRestIT.java | 101 +++++++++++++----- .../action/bulk/IncrementalBulkService.java | 52 +++++---- .../common/settings/ClusterSettings.java | 4 +- .../elasticsearch/node/NodeConstruction.java | 3 +- .../rest/action/document/RestBulkAction.java | 10 +- .../action/document/RestBulkActionTests.java | 30 +----- 7 files changed, 112 insertions(+), 92 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java index d4e5746a6159f..fcd45e9f9f47e 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java @@ -14,6 +14,7 @@ import io.netty.util.ReferenceCounted; import org.elasticsearch.ESNetty4IntegTestCase; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -21,7 +22,6 @@ import org.elasticsearch.core.Tuple; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; -import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -54,7 +54,7 @@ protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal, otherSettings)) // TODO: We do not currently support in flight circuit breaker limits for bulk. However, IndexingPressure applies - .put(RestBulkAction.INCREMENTAL_BULK.getKey(), false) + .put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), false) .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), LIMIT) .build(); } diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java index fcb0c1044d22f..4b3b6fbb2a719 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java @@ -8,9 +8,11 @@ package org.elasticsearch.http; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.xcontent.json.JsonXContent; @@ -25,7 +27,6 @@ @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, supportsDedicatedMasters = false, numDataNodes = 2, numClientNodes = 0) public class IncrementalBulkRestIT extends HttpSmokeTestCase { - @SuppressWarnings("unchecked") public void testIncrementalBulk() throws IOException { Request createRequest = new Request("PUT", "/index_name"); createRequest.setJsonEntity(""" @@ -55,35 +56,52 @@ public void testIncrementalBulk() throws IOException { final Response indexSuccessFul = getRestClient().performRequest(firstBulkRequest); assertThat(indexSuccessFul.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); - Request bulkRequest = new Request("POST", "/index_name/_bulk"); + sendLargeBulk(); + } + + public void testBulkWithIncrementalDisabled() throws IOException { + Request createRequest = new Request("PUT", "/index_name"); + createRequest.setJsonEntity(""" + { + "settings": { + "index": { + "number_of_shards": 1, + "number_of_replicas": 1, + "write.wait_for_active_shards": 2 + } + } + }"""); + final Response indexCreatedResponse = getRestClient().performRequest(createRequest); + assertThat(indexCreatedResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + + Request firstBulkRequest = new Request("POST", "/index_name/_bulk"); // index documents for the rollup job - final StringBuilder bulk = new StringBuilder(); - bulk.append("{\"delete\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n"); - int updates = 0; - for (int i = 0; i < 1000; i++) { - bulk.append("{\"index\":{\"_index\":\"index_name\"}}\n"); - bulk.append("{\"field\":").append(i).append("}\n"); - if (randomBoolean() && randomBoolean() && randomBoolean() && randomBoolean()) { - ++updates; - bulk.append("{\"update\":{\"_index\":\"index_name\",\"_id\":\"2\"}}\n"); - bulk.append("{\"doc\":{\"field\":").append(i).append("}}\n"); - } - } - bulk.append("\r\n"); + String bulkBody = "{\"index\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n" + + "{\"field\":1}\n" + + "{\"index\":{\"_index\":\"index_name\",\"_id\":\"2\"}}\n" + + "{\"field\":1}\n" + + "\r\n"; - bulkRequest.setJsonEntity(bulk.toString()); + firstBulkRequest.setJsonEntity(bulkBody); - final Response bulkResponse = getRestClient().performRequest(bulkRequest); - assertThat(bulkResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); - Map responseMap = XContentHelper.convertToMap( - JsonXContent.jsonXContent, - bulkResponse.getEntity().getContent(), - true - ); + final Response indexSuccessFul = getRestClient().performRequest(firstBulkRequest); + assertThat(indexSuccessFul.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); - assertFalse((Boolean) responseMap.get("errors")); - assertThat(((List) responseMap.get("items")).size(), equalTo(1001 + updates)); + clusterAdmin().prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), false).build()) + .get(); + + internalCluster().getInstances(IncrementalBulkService.class).forEach(i -> i.setForTests(false)); + + try { + sendLargeBulk(); + } finally { + internalCluster().getInstances(IncrementalBulkService.class).forEach(i -> i.setForTests(true)); + clusterAdmin().prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), (String) null).build()) + .get(); + } } public void testIncrementalMalformed() throws IOException { @@ -114,4 +132,37 @@ public void testIncrementalMalformed() throws IOException { expectThrows(ResponseException.class, () -> getRestClient().performRequest(bulkRequest)); } + + @SuppressWarnings("unchecked") + private static void sendLargeBulk() throws IOException { + Request bulkRequest = new Request("POST", "/index_name/_bulk"); + + // index documents for the rollup job + final StringBuilder bulk = new StringBuilder(); + bulk.append("{\"delete\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n"); + int updates = 0; + for (int i = 0; i < 1000; i++) { + bulk.append("{\"index\":{\"_index\":\"index_name\"}}\n"); + bulk.append("{\"field\":").append(i).append("}\n"); + if (randomBoolean() && randomBoolean() && randomBoolean() && randomBoolean()) { + ++updates; + bulk.append("{\"update\":{\"_index\":\"index_name\",\"_id\":\"2\"}}\n"); + bulk.append("{\"doc\":{\"field\":").append(i).append("}}\n"); + } + } + bulk.append("\r\n"); + + bulkRequest.setJsonEntity(bulk.toString()); + + final Response bulkResponse = getRestClient().performRequest(bulkRequest); + assertThat(bulkResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); + Map responseMap = XContentHelper.convertToMap( + JsonXContent.jsonXContent, + bulkResponse.getEntity().getContent(), + true + ); + + assertFalse((Boolean) responseMap.get("errors")); + assertThat(((List) responseMap.get("items")).size(), equalTo(1001 + updates)); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java index 8ecb0a198fe10..f9e23e3bfe02d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.Nullable; @@ -22,7 +23,6 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexingPressure; -import org.elasticsearch.rest.action.document.RestBulkAction; import java.util.ArrayList; import java.util.Collections; @@ -30,50 +30,48 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; +import static org.elasticsearch.common.settings.Setting.boolSetting; + public class IncrementalBulkService { + public static final Setting INCREMENTAL_BULK = boolSetting( + "rest.incremental_bulk", + true, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); private final Client client; + private final AtomicBoolean enabledForTests = new AtomicBoolean(true); private final IndexingPressure indexingPressure; private final ThreadContext threadContext; - private final Supplier enabled; public IncrementalBulkService(Client client, IndexingPressure indexingPressure, ThreadContext threadContext) { - this(client, indexingPressure, threadContext, new Enabled()); - } - - public IncrementalBulkService( - Client client, - IndexingPressure indexingPressure, - ThreadContext threadContext, - ClusterSettings clusterSettings - ) { - this(client, indexingPressure, threadContext, new Enabled(clusterSettings)); - } - - public IncrementalBulkService( - Client client, - IndexingPressure indexingPressure, - ThreadContext threadContext, - Supplier enabled - ) { this.client = client; this.indexingPressure = indexingPressure; this.threadContext = threadContext; - this.enabled = enabled; - } - - public boolean incrementalBulkEnabled() { - return enabled.get(); } public Handler newBulkRequest() { + ensureEnabled(); return newBulkRequest(null, null, null); } public Handler newBulkRequest(@Nullable String waitForActiveShards, @Nullable TimeValue timeout, @Nullable String refresh) { + ensureEnabled(); return new Handler(client, threadContext, indexingPressure, waitForActiveShards, timeout, refresh); } + private void ensureEnabled() { + if (enabledForTests.get() == false) { + throw new AssertionError("Unexpected incremental bulk request"); + } + } + + // This method only exists to tests that the feature flag works. Remove once we no longer need the flag. + public void setForTests(boolean value) { + enabledForTests.set(value); + } + public static class Enabled implements Supplier { private final AtomicBoolean incrementalBulksEnabled = new AtomicBoolean(true); @@ -81,8 +79,8 @@ public static class Enabled implements Supplier { public Enabled() {} public Enabled(ClusterSettings clusterSettings) { - incrementalBulksEnabled.set(clusterSettings.get(RestBulkAction.INCREMENTAL_BULK)); - clusterSettings.addSettingsUpdateConsumer(RestBulkAction.INCREMENTAL_BULK, incrementalBulksEnabled::set); + incrementalBulksEnabled.set(clusterSettings.get(INCREMENTAL_BULK)); + clusterSettings.addSettingsUpdateConsumer(INCREMENTAL_BULK, incrementalBulksEnabled::set); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 35c0900b0dc2b..2ab0318490f7a 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; +import org.elasticsearch.action.bulk.IncrementalBulkService; import org.elasticsearch.action.bulk.WriteAckDelay; import org.elasticsearch.action.datastreams.autosharding.DataStreamAutoShardingService; import org.elasticsearch.action.ingest.SimulatePipelineTransportAction; @@ -113,7 +114,6 @@ import org.elasticsearch.readiness.ReadinessService; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.rest.BaseRestHandler; -import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchService; @@ -243,7 +243,7 @@ public void apply(Settings value, Settings current, Settings previous) { Metadata.SETTING_READ_ONLY_SETTING, Metadata.SETTING_READ_ONLY_ALLOW_DELETE_SETTING, ShardLimitValidator.SETTING_CLUSTER_MAX_SHARDS_PER_NODE, - RestBulkAction.INCREMENTAL_BULK, + IncrementalBulkService.INCREMENTAL_BULK, RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index 6560d6d6849f9..c4816b440f568 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -895,8 +895,7 @@ private void construct( final IncrementalBulkService incrementalBulkService = new IncrementalBulkService( client, indexingLimits, - threadPool.getThreadContext(), - clusterService.getClusterSettings() + threadPool.getThreadContext() ); ActionModule actionModule = new ActionModule( diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 53cd97d15bb82..9439102073df4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -19,7 +19,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.CompositeBytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; -import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -42,7 +41,6 @@ import java.util.Map; import java.util.function.Supplier; -import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; @@ -59,12 +57,6 @@ public class RestBulkAction extends BaseRestHandler { public static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Specifying types in bulk requests is deprecated."; - public static final Setting INCREMENTAL_BULK = boolSetting( - "rest.incremental_bulk", - true, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); private final boolean allowExplicitIndex; private final IncrementalBulkService bulkHandler; @@ -93,7 +85,7 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - if (bulkHandler.incrementalBulkEnabled() == false) { + if (request.isStreamedContent() == false) { if (request.getRestApiVersion() == RestApiVersion.V_7 && request.hasParam("type")) { request.param("type"); } diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 44fd1e721f8d2..3bc39f46bb153 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -67,7 +67,7 @@ public void bulk(BulkRequest request, ActionListener listener) { params.put("pipeline", "timestamps"); new RestBulkAction( settings(IndexVersion.current()).build(), - new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY), () -> false) + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY)) ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk").withParams(params).withContent(new BytesArray(""" {"index":{"_id":"1"}} @@ -102,12 +102,7 @@ public void bulk(BulkRequest request, ActionListener listener) { { new RestBulkAction( settings(IndexVersion.current()).build(), - new IncrementalBulkService( - mock(Client.class), - mock(IndexingPressure.class), - new ThreadContext(Settings.EMPTY), - () -> false - ) + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY)) ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) @@ -131,12 +126,7 @@ public void bulk(BulkRequest request, ActionListener listener) { bulkCalled.set(false); new RestBulkAction( settings(IndexVersion.current()).build(), - new IncrementalBulkService( - mock(Client.class), - mock(IndexingPressure.class), - new ThreadContext(Settings.EMPTY), - () -> false - ) + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY)) ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) @@ -159,12 +149,7 @@ public void bulk(BulkRequest request, ActionListener listener) { bulkCalled.set(false); new RestBulkAction( settings(IndexVersion.current()).build(), - new IncrementalBulkService( - mock(Client.class), - mock(IndexingPressure.class), - new ThreadContext(Settings.EMPTY), - () -> false - ) + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY)) ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) @@ -188,12 +173,7 @@ public void bulk(BulkRequest request, ActionListener listener) { bulkCalled.set(false); new RestBulkAction( settings(IndexVersion.current()).build(), - new IncrementalBulkService( - mock(Client.class), - mock(IndexingPressure.class), - new ThreadContext(Settings.EMPTY), - () -> false - ) + new IncrementalBulkService(mock(Client.class), mock(IndexingPressure.class), new ThreadContext(Settings.EMPTY)) ).handleRequest( new FakeRestRequest.Builder(xContentRegistry()).withPath("my_index/_bulk") .withParams(params) From ce2d648d8eadd444c1c3ebfc65510f02fe0d7ae1 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 6 Sep 2024 12:34:50 -0600 Subject: [PATCH 08/14] Reduce autoread changes in header validator (#112608) The header validator is very aggressive about adjusting autoread on the belief it is the only place where autoread is tweaked. However, with stream backpressure, we should only change it when we are starting or finishing header validation. --- .../netty4/Netty4HttpHeaderValidator.java | 17 +++++------ .../Netty4HttpHeaderValidatorTests.java | 30 +++++++++++++++++++ 2 files changed, 38 insertions(+), 9 deletions(-) diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidator.java index 88b458fd1c416..95a68cb52bbdb 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidator.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidator.java @@ -61,6 +61,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception pending.add(ReferenceCountUtil.retain(httpObject)); requestStart(ctx); assert state == QUEUEING_DATA; + assert ctx.channel().config().isAutoRead() == false; break; case QUEUEING_DATA: pending.add(ReferenceCountUtil.retain(httpObject)); @@ -77,14 +78,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (httpObject instanceof LastHttpContent) { state = WAITING_TO_START; } - // fall-through + ReferenceCountUtil.release(httpObject); + break; case DROPPING_DATA_PERMANENTLY: assert pending.isEmpty(); ReferenceCountUtil.release(httpObject); // consume without enqueuing + ctx.channel().config().setAutoRead(false); break; } - - setAutoReadForState(ctx, state); } private void requestStart(ChannelHandlerContext ctx) { @@ -105,6 +106,7 @@ private void requestStart(ChannelHandlerContext ctx) { } state = QUEUEING_DATA; + ctx.channel().config().setAutoRead(false); if (httpRequest == null) { // this looks like a malformed request and will forward without validation @@ -150,6 +152,7 @@ private void forwardFullRequest(ChannelHandlerContext ctx) { assert ctx.channel().config().isAutoRead() == false; assert state == QUEUEING_DATA; + ctx.channel().config().setAutoRead(true); boolean fullRequestForwarded = forwardData(ctx, pending); assert fullRequestForwarded || pending.isEmpty(); @@ -161,7 +164,6 @@ private void forwardFullRequest(ChannelHandlerContext ctx) { } assert state == WAITING_TO_START || state == QUEUEING_DATA || state == FORWARDING_DATA_UNTIL_NEXT_REQUEST; - setAutoReadForState(ctx, state); } private void forwardRequestWithDecoderExceptionAndNoContent(ChannelHandlerContext ctx, Exception e) { @@ -177,6 +179,8 @@ private void forwardRequestWithDecoderExceptionAndNoContent(ChannelHandlerContex messageToForward = toReplace.replace(Unpooled.EMPTY_BUFFER); } messageToForward.setDecoderResult(DecoderResult.failure(e)); + + ctx.channel().config().setAutoRead(true); ctx.fireChannelRead(messageToForward); assert fullRequestDropped || pending.isEmpty(); @@ -188,7 +192,6 @@ private void forwardRequestWithDecoderExceptionAndNoContent(ChannelHandlerContex } assert state == WAITING_TO_START || state == QUEUEING_DATA || state == DROPPING_DATA_UNTIL_NEXT_REQUEST; - setAutoReadForState(ctx, state); } @Override @@ -244,10 +247,6 @@ private static void maybeResizePendingDown(int largeSize, ArrayDeque } } - private static void setAutoReadForState(ChannelHandlerContext ctx, State state) { - ctx.channel().config().setAutoRead((state == QUEUEING_DATA || state == DROPPING_DATA_PERMANENTLY) == false); - } - enum State { WAITING_TO_START, QUEUEING_DATA, diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidatorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidatorTests.java index c2d52ac761034..1c0b434105f28 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidatorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpHeaderValidatorTests.java @@ -117,6 +117,36 @@ public void testValidationPausesAndResumesData() { assertThat(netty4HttpHeaderValidator.getState(), equalTo(QUEUEING_DATA)); } + public void testValidatorDoesNotTweakAutoReadAfterValidationComplete() { + assertTrue(channel.config().isAutoRead()); + assertThat(netty4HttpHeaderValidator.getState(), equalTo(WAITING_TO_START)); + + final DefaultHttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/uri"); + DefaultHttpContent content = new DefaultHttpContent(Unpooled.buffer(4)); + channel.writeInbound(request); + channel.writeInbound(content); + + assertThat(header.get(), sameInstance(request)); + // channel is paused + assertThat(channel.readInbound(), nullValue()); + assertFalse(channel.config().isAutoRead()); + + // channel is resumed + listener.get().onResponse(null); + channel.runPendingTasks(); + + assertTrue(channel.config().isAutoRead()); + assertThat(netty4HttpHeaderValidator.getState(), equalTo(FORWARDING_DATA_UNTIL_NEXT_REQUEST)); + assertThat(channel.readInbound(), sameInstance(request)); + assertThat(channel.readInbound(), sameInstance(content)); + assertThat(channel.readInbound(), nullValue()); + assertThat(content.refCnt(), equalTo(1)); + channel.config().setAutoRead(false); + + channel.writeOutbound(new DefaultHttpContent(Unpooled.buffer(4))); + assertFalse(channel.config().isAutoRead()); + } + public void testContentForwardedAfterValidation() { assertTrue(channel.config().isAutoRead()); assertThat(netty4HttpHeaderValidator.getState(), equalTo(WAITING_TO_START)); From 0d55dc6de4acdd28a3da5a9681e4a2a3c22fc5aa Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Mon, 9 Sep 2024 10:42:20 -0700 Subject: [PATCH 09/14] fix leaking listener (#112629) --- .../Netty4IncrementalRequestHandlingIT.java | 13 ++- .../netty4/Netty4HttpRequestBodyStream.java | 96 ++++++++++--------- .../Netty4HttpRequestBodyStreamTests.java | 23 +++-- 3 files changed, 69 insertions(+), 63 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index c85f845887203..e5b132e22dbf4 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -153,7 +153,7 @@ public void testReceiveAllChunks() throws Exception { } } - // ensures that all queued chunks are released when connection closed + // ensures that all received chunks are released when connection closed public void testClientConnectionCloseMidStream() throws Exception { try (var ctx = setupClientCtx()) { var opaqueId = opaqueId(0); @@ -164,18 +164,18 @@ public void testClientConnectionCloseMidStream() throws Exception { // await stream handler is ready and request full content var handler = ctx.awaitRestChannelAccepted(opaqueId); - assertBusy(() -> assertEquals(1, handler.stream.chunkQueue().size())); + assertBusy(() -> assertNotNull(handler.stream.buf())); // enable auto-read to receive channel close event handler.stream.channel().config().setAutoRead(true); // terminate connection and wait resources are released ctx.clientChannel.close(); - assertBusy(() -> assertEquals(0, handler.stream.chunkQueue().size())); + assertBusy(() -> assertNull(handler.stream.buf())); } } - // ensures that all queued chunks are released when server decides to close connection + // ensures that all recieved chunks are released when server decides to close connection public void testServerCloseConnectionMidStream() throws Exception { try (var ctx = setupClientCtx()) { var opaqueId = opaqueId(0); @@ -186,11 +186,11 @@ public void testServerCloseConnectionMidStream() throws Exception { // await stream handler is ready and request full content var handler = ctx.awaitRestChannelAccepted(opaqueId); - assertBusy(() -> assertEquals(1, handler.stream.chunkQueue().size())); + assertBusy(() -> assertNotNull(handler.stream.buf())); // terminate connection on server and wait resources are released handler.channel.request().getHttpChannel().close(); - assertBusy(() -> assertEquals(0, handler.stream.chunkQueue().size())); + assertBusy(() -> assertNull(handler.stream.buf())); } } @@ -470,7 +470,6 @@ static class ServerRequestHandler implements BaseRestHandler.RequestBodyChunkCon final BlockingDeque recvChunks = new LinkedBlockingDeque<>(); final Netty4HttpRequestBodyStream stream; RestChannel channel; - boolean recvLast = false; ServerRequestHandler(String opaqueId, Netty4HttpRequestBodyStream stream) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index a7783626e6b80..66fa0a1b6ee4c 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -9,34 +9,36 @@ package org.elasticsearch.http.netty4; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; import io.netty.channel.Channel; +import io.netty.channel.ChannelFutureListener; import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.LastHttpContent; import org.elasticsearch.http.HttpBody; import org.elasticsearch.transport.netty4.Netty4Utils; -import java.util.ArrayDeque; -import java.util.Queue; - /** * Netty based implementation of {@link HttpBody.Stream}. * This implementation utilize {@link io.netty.channel.ChannelConfig#setAutoRead(boolean)} * to prevent entire payload buffering. But sometimes upstream can send few chunks of data despite - * autoRead=off. In this case chunks will be queued until downstream calls {@link Stream#next()} + * autoRead=off. In this case chunks will be buffered until downstream calls {@link Stream#next()} */ public class Netty4HttpRequestBodyStream implements HttpBody.Stream { private final Channel channel; - private final Queue chunkQueue = new ArrayDeque<>(); - private boolean requested = false; + private final ChannelFutureListener closeListener = future -> doClose(); + private ByteBuf buf; private boolean hasLast = false; + private boolean requested = false; private boolean closing = false; private HttpBody.ChunkHandler handler; + public Netty4HttpRequestBodyStream(Channel channel) { this.channel = channel; - channel.closeFuture().addListener((f) -> doClose()); + Netty4Utils.addListener(channel.closeFuture(), closeListener); channel.config().setAutoRead(false); } @@ -50,41 +52,49 @@ public void setHandler(ChunkHandler chunkHandler) { this.handler = chunkHandler; } - private void sendQueuedOrRead() { - assert channel.eventLoop().inEventLoop(); - requested = true; - var chunk = chunkQueue.poll(); - if (chunk == null) { - channel.read(); - } else { - sendChunk(chunk); - } - } - @Override public void next() { + assert closing == false : "cannot request next chunk on closing stream"; assert handler != null : "handler must be set before requesting next chunk"; - if (channel.eventLoop().inEventLoop()) { - sendQueuedOrRead(); - } else { - channel.eventLoop().submit(this::sendQueuedOrRead); - } + channel.eventLoop().submit(() -> { + requested = true; + if (buf == null) { + channel.read(); + } else { + send(); + } + }); } public void handleNettyContent(HttpContent httpContent) { + assert hasLast == false : "receive http content on completed stream"; + hasLast = httpContent instanceof LastHttpContent; if (closing) { httpContent.release(); - return; - } - assert handler != null : "handler must be set before processing http content"; - if (requested && chunkQueue.isEmpty()) { - sendChunk(httpContent); } else { - chunkQueue.add(httpContent); + addChunk(httpContent.content()); + if (requested) { + send(); + } } - if (httpContent instanceof LastHttpContent) { - hasLast = true; + if (hasLast) { channel.config().setAutoRead(true); + channel.closeFuture().removeListener(closeListener); + } + } + + // adds chunk to current buffer, will allocate composite buffer when need to hold more than 1 chunk + private void addChunk(ByteBuf chunk) { + assert chunk != null; + if (buf == null) { + buf = chunk; + } else if (buf instanceof CompositeByteBuf comp) { + comp.addComponent(true, chunk); + } else { + var comp = channel.alloc().compositeBuffer(); + comp.addComponent(true, buf); + comp.addComponent(true, chunk); + buf = comp; } } @@ -94,8 +104,8 @@ Channel channel() { } // visible for test - Queue chunkQueue() { - return chunkQueue; + ByteBuf buf() { + return buf; } // visible for test @@ -103,18 +113,13 @@ boolean hasLast() { return hasLast; } - private void sendChunk(HttpContent httpContent) { + private void send() { assert requested; + assert handler != null : "must set handler before receiving next chunk"; + var bytesRef = Netty4Utils.toReleasableBytesReference(buf); requested = false; - var bytesRef = Netty4Utils.toReleasableBytesReference(httpContent.content()); - var isLast = httpContent instanceof LastHttpContent; - handler.onNext(bytesRef, isLast); - } - - private void releaseQueuedChunks() { - while (chunkQueue.isEmpty() == false) { - chunkQueue.poll().release(); - } + buf = null; + handler.onNext(bytesRef, hasLast); } @Override @@ -128,7 +133,10 @@ public void close() { private void doClose() { closing = true; - releaseQueuedChunks(); + if (buf != null) { + buf.release(); + buf = null; + } channel.config().setAutoRead(true); } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java index 3448fd15d79d6..0f35de483dc82 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStreamTests.java @@ -53,33 +53,31 @@ public void testEnqueueChunksBeforeRequest() { for (int i = 0; i < totalChunks; i++) { channel.writeInbound(randomContent(1024)); } - assertEquals(totalChunks, stream.chunkQueue().size()); + assertEquals(totalChunks * 1024, stream.buf().readableBytes()); } - // ensures all queued chunks can be flushed downstream - public void testFlushQueued() { + // ensures all received chunks can be flushed downstream + public void testFlushAllReceivedChunks() { var chunks = new ArrayList(); var totalBytes = new AtomicInteger(); stream.setHandler((chunk, isLast) -> { chunks.add(chunk); totalBytes.addAndGet(chunk.length()); }); - // enqueue chunks + var chunkSize = 1024; var totalChunks = randomIntBetween(1, 100); for (int i = 0; i < totalChunks; i++) { channel.writeInbound(randomContent(chunkSize)); } - // consume all chunks - for (var i = 0; i < totalChunks; i++) { - stream.next(); - } - assertEquals(totalChunks, chunks.size()); + stream.next(); + channel.runPendingTasks(); + assertEquals("should receive all chunks as single composite", 1, chunks.size()); assertEquals(chunkSize * totalChunks, totalBytes.get()); } - // ensures that we read from channel when chunks queue is empty - // and pass next chunk downstream without queuing + // ensures that we read from channel when no current chunks available + // and pass next chunk downstream without holding public void testReadFromChannel() { var gotChunks = new ArrayList(); var gotLast = new AtomicBoolean(false); @@ -96,8 +94,9 @@ public void testReadFromChannel() { channel.writeInbound(randomLastContent(chunkSize)); for (int i = 0; i < totalChunks; i++) { - assertEquals("should not enqueue chunks", 0, stream.chunkQueue().size()); + assertNull("should not enqueue chunks", stream.buf()); stream.next(); + channel.runPendingTasks(); assertEquals("each next() should produce single chunk", i + 1, gotChunks.size()); } assertTrue("should receive last content", gotLast.get()); From 2dbbd7dd451be0f6f8b0d827dcb15798ce818b20 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 11 Sep 2024 16:13:11 -0600 Subject: [PATCH 10/14] Ensure http content copied for safe buffers (#112767) Currently, unless a rest handler specifies that it handles "unsafe" buffers, we must copy the http buffers in releaseAndCopy. Unfortuantely, the original content was slipping through in the initial stream PR. This less to memory corruption on index and update requests which depend on buffers being copied. --- .../org/elasticsearch/http/netty4/Netty4HttpRequest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java index 4f1bf0f01478b..b04da46a2d7d7 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequest.java @@ -128,6 +128,12 @@ public HttpRequest releaseAndCopy() { } try { final ByteBuf copiedContent = Unpooled.copiedBuffer(request.content()); + HttpBody newContent; + if (content.isStream()) { + newContent = content; + } else { + newContent = Netty4Utils.fullHttpBodyFrom(copiedContent); + } return new Netty4HttpRequest( sequence, new DefaultFullHttpRequest( @@ -140,7 +146,7 @@ public HttpRequest releaseAndCopy() { ), new AtomicBoolean(false), false, - content + newContent ); } finally { release(); From 58e3a393924440148755b506989ebc212067e7f3 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 11 Sep 2024 19:20:15 -0600 Subject: [PATCH 11/14] Ensure partial bulks released if channel closes (#112724) Currently, the entire close pipeline is not hooked up in case of a channel close while a request is being buffered or executed. This commit resolves the issue by adding a connection to a stream closure. --- .../Netty4IncrementalRequestHandlingIT.java | 24 +++++++++-- .../netty4/Netty4HttpRequestBodyStream.java | 3 ++ .../action/bulk/IncrementalBulkIT.java | 41 ++++++++++++++++--- .../action/bulk/IncrementalBulkService.java | 38 +++++++++++------ .../java/org/elasticsearch/http/HttpBody.java | 5 ++- .../elasticsearch/rest/BaseRestHandler.java | 20 ++++++++- .../rest/action/document/RestBulkAction.java | 23 +++++++---- 7 files changed, 122 insertions(+), 32 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index e5b132e22dbf4..36ff091f20843 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -111,9 +111,11 @@ public void testEmptyContent() throws Exception { assertTrue(recvChunk.isLast); assertEquals(0, recvChunk.chunk.length()); recvChunk.chunk.close(); + assertFalse(handler.streamClosed); // send response to process following request handler.sendResponse(new RestResponse(RestStatus.OK, "")); + assertBusy(() -> assertTrue(handler.streamClosed)); } assertBusy(() -> assertEquals("should receive all server responses", totalRequests, ctx.clientRespQueue.size())); } @@ -146,14 +148,16 @@ public void testReceiveAllChunks() throws Exception { } } + assertFalse(handler.streamClosed); assertEquals("sent and received payloads are not the same", sendData, recvData); handler.sendResponse(new RestResponse(RestStatus.OK, "")); + assertBusy(() -> assertTrue(handler.streamClosed)); } assertBusy(() -> assertEquals("should receive all server responses", totalRequests, ctx.clientRespQueue.size())); } } - // ensures that all received chunks are released when connection closed + // ensures that all received chunks are released when connection closed and handler notified public void testClientConnectionCloseMidStream() throws Exception { try (var ctx = setupClientCtx()) { var opaqueId = opaqueId(0); @@ -168,10 +172,14 @@ public void testClientConnectionCloseMidStream() throws Exception { // enable auto-read to receive channel close event handler.stream.channel().config().setAutoRead(true); + assertFalse(handler.streamClosed); // terminate connection and wait resources are released ctx.clientChannel.close(); - assertBusy(() -> assertNull(handler.stream.buf())); + assertBusy(() -> { + assertNull(handler.stream.buf()); + assertTrue(handler.streamClosed); + }); } } @@ -187,10 +195,14 @@ public void testServerCloseConnectionMidStream() throws Exception { // await stream handler is ready and request full content var handler = ctx.awaitRestChannelAccepted(opaqueId); assertBusy(() -> assertNotNull(handler.stream.buf())); + assertFalse(handler.streamClosed); // terminate connection on server and wait resources are released handler.channel.request().getHttpChannel().close(); - assertBusy(() -> assertNull(handler.stream.buf())); + assertBusy(() -> { + assertNull(handler.stream.buf()); + assertTrue(handler.streamClosed); + }); } } @@ -471,6 +483,7 @@ static class ServerRequestHandler implements BaseRestHandler.RequestBodyChunkCon final Netty4HttpRequestBodyStream stream; RestChannel channel; boolean recvLast = false; + volatile boolean streamClosed = false; ServerRequestHandler(String opaqueId, Netty4HttpRequestBodyStream stream) { this.opaqueId = opaqueId; @@ -488,6 +501,11 @@ public void accept(RestChannel channel) throws Exception { channelAccepted.onResponse(null); } + @Override + public void streamClose() { + streamClosed = true; + } + void sendResponse(RestResponse response) { channel.sendResponse(response); } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index 66fa0a1b6ee4c..7cc0b50e0c7ec 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -133,6 +133,9 @@ public void close() { private void doClose() { closing = true; + if (handler != null) { + handler.close(); + } if (buf != null) { buf.release(); buf = null; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java index 1bae6a3e75078..d7a5d4e2ac973 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/IncrementalBulkIT.java @@ -90,6 +90,29 @@ public void testSingleBulkRequest() { assertFalse(refCounted.hasReferences()); } + public void testBufferedResourcesReleasedOnClose() { + String index = "test"; + createIndex(index); + + String nodeName = internalCluster().getRandomNodeName(); + IncrementalBulkService incrementalBulkService = internalCluster().getInstance(IncrementalBulkService.class, nodeName); + IndexingPressure indexingPressure = internalCluster().getInstance(IndexingPressure.class, nodeName); + + IncrementalBulkService.Handler handler = incrementalBulkService.newBulkRequest(); + IndexRequest indexRequest = indexRequest(index); + + AbstractRefCounted refCounted = AbstractRefCounted.of(() -> {}); + handler.addItems(List.of(indexRequest), refCounted::decRef, () -> {}); + + assertTrue(refCounted.hasReferences()); + assertThat(indexingPressure.stats().getCurrentCoordinatingBytes(), greaterThan(0L)); + + handler.close(); + + assertFalse(refCounted.hasReferences()); + assertThat(indexingPressure.stats().getCurrentCoordinatingBytes(), equalTo(0L)); + } + public void testIndexingPressureRejection() { String index = "test"; createIndex(index); @@ -303,14 +326,20 @@ public void testShortCircuitShardLevelFailure() throws Exception { String secondShardNode = findShard(resolveIndex(index), 1); IndexingPressure primaryPressure = internalCluster().getInstance(IndexingPressure.class, node); long memoryLimit = primaryPressure.stats().getMemoryLimit(); + long primaryRejections = primaryPressure.stats().getPrimaryRejections(); try (Releasable releasable = primaryPressure.markPrimaryOperationStarted(10, memoryLimit, false)) { - while (nextRequested.get()) { - nextRequested.set(false); - refCounted.incRef(); - handler.addItems(List.of(indexRequest(index)), refCounted::decRef, () -> nextRequested.set(true)); + while (primaryPressure.stats().getPrimaryRejections() == primaryRejections) { + while (nextRequested.get()) { + nextRequested.set(false); + refCounted.incRef(); + List> requests = new ArrayList<>(); + for (int i = 0; i < 20; ++i) { + requests.add(indexRequest(index)); + } + handler.addItems(requests, refCounted::decRef, () -> nextRequested.set(true)); + } + assertBusy(() -> assertTrue(nextRequested.get())); } - - assertBusy(() -> assertTrue(nextRequested.get())); } while (nextRequested.get()) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java index f9e23e3bfe02d..7185c4d76265e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java @@ -102,6 +102,7 @@ public static class Handler implements Releasable { private final ArrayList releasables = new ArrayList<>(4); private final ArrayList responses = new ArrayList<>(2); + private boolean closed = false; private boolean globalFailure = false; private boolean incrementalRequestSubmitted = false; private ThreadContext.StoredContext requestContext; @@ -127,6 +128,7 @@ protected Handler( } public void addItems(List> items, Releasable releasable, Runnable nextItems) { + assert closed == false; if (bulkActionLevelFailure != null) { shortCircuitDueToTopLevelFailure(items, releasable); nextItems.run(); @@ -138,12 +140,13 @@ public void addItems(List> items, Releasable releasable, Runn incrementalRequestSubmitted = true; try (ThreadContext.StoredContext ignored = threadContext.stashContext()) { requestContext.restore(); + final ArrayList toRelease = new ArrayList<>(releasables); + releasables.clear(); client.bulk(bulkRequest, ActionListener.runAfter(new ActionListener<>() { @Override public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); + handleBulkSuccess(bulkResponse); createNewBulkRequest( new BulkRequest.IncrementalState(bulkResponse.getIncrementalState().shardLevelFailures(), true) ); @@ -155,6 +158,7 @@ public void onFailure(Exception e) { } }, () -> { requestContext = threadContext.newStoredContext(); + toRelease.forEach(Releasable::close); nextItems.run(); })); } @@ -180,14 +184,15 @@ public void lastItems(List> items, Releasable releasable, Act if (internalAddItems(items, releasable)) { try (ThreadContext.StoredContext ignored = threadContext.stashContext()) { requestContext.restore(); - client.bulk(bulkRequest, new ActionListener<>() { + final ArrayList toRelease = new ArrayList<>(releasables); + releasables.clear(); + client.bulk(bulkRequest, ActionListener.runBefore(new ActionListener<>() { private final boolean isFirstRequest = incrementalRequestSubmitted == false; @Override public void onResponse(BulkResponse bulkResponse) { - responses.add(bulkResponse); - releaseCurrentReferences(); + handleBulkSuccess(bulkResponse); listener.onResponse(combineResponses()); } @@ -196,7 +201,7 @@ public void onFailure(Exception e) { handleBulkFailure(isFirstRequest, e); errorResponse(listener); } - }); + }, () -> toRelease.forEach(Releasable::close))); } } else { errorResponse(listener); @@ -204,6 +209,13 @@ public void onFailure(Exception e) { } } + @Override + public void close() { + closed = true; + releasables.forEach(Releasable::close); + releasables.clear(); + } + private void shortCircuitDueToTopLevelFailure(List> items, Releasable releasable) { assert releasables.isEmpty(); assert bulkRequest == null; @@ -221,12 +233,17 @@ private void errorResponse(ActionListener listener) { } } + private void handleBulkSuccess(BulkResponse bulkResponse) { + responses.add(bulkResponse); + bulkRequest = null; + } + private void handleBulkFailure(boolean isFirstRequest, Exception e) { assert bulkActionLevelFailure == null; globalFailure = isFirstRequest; bulkActionLevelFailure = e; addItemLevelFailures(bulkRequest.requests()); - releaseCurrentReferences(); + bulkRequest = null; } private void addItemLevelFailures(List> items) { @@ -254,6 +271,8 @@ private boolean internalAddItems(List> items, Releasable rele return true; } catch (EsRejectedExecutionException e) { handleBulkFailure(incrementalRequestSubmitted == false, e); + releasables.forEach(Releasable::close); + releasables.clear(); return false; } } @@ -298,10 +317,5 @@ private BulkResponse combineResponses() { return new BulkResponse(bulkItemResponses, tookInMillis, ingestTookInMillis); } - - @Override - public void close() { - // TODO: Implement - } } } diff --git a/server/src/main/java/org/elasticsearch/http/HttpBody.java b/server/src/main/java/org/elasticsearch/http/HttpBody.java index f8761fac7a67f..40fd2fd64ec87 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpBody.java +++ b/server/src/main/java/org/elasticsearch/http/HttpBody.java @@ -100,8 +100,11 @@ non-sealed interface Stream extends HttpBody { } @FunctionalInterface - interface ChunkHandler { + interface ChunkHandler extends Releasable { void onNext(ReleasableBytesReference chunk, boolean isLast); + + @Override + default void close() {} } record ByteRefHttpBody(BytesReference bytes) implements Full {} diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 227c1edc0867e..5f12a2bdd6783 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -20,6 +20,7 @@ import org.elasticsearch.core.Releasable; import org.elasticsearch.core.RestApiVersion; import org.elasticsearch.core.Tuple; +import org.elasticsearch.http.HttpBody; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.rest.action.admin.cluster.RestNodesUsageAction; @@ -127,7 +128,17 @@ public final void handleRequest(RestRequest request, RestChannel channel, NodeCl if (request.isStreamedContent()) { assert action instanceof RequestBodyChunkConsumer; var chunkConsumer = (RequestBodyChunkConsumer) action; - request.contentStream().setHandler((chunk, isLast) -> chunkConsumer.handleChunk(channel, chunk, isLast)); + request.contentStream().setHandler(new HttpBody.ChunkHandler() { + @Override + public void onNext(ReleasableBytesReference chunk, boolean isLast) { + chunkConsumer.handleChunk(channel, chunk, isLast); + } + + @Override + public void close() { + chunkConsumer.streamClose(); + } + }); } usageCount.increment(); @@ -189,6 +200,13 @@ default void close() {} public interface RequestBodyChunkConsumer extends RestChannelConsumer { void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boolean isLast); + + /** + * Called when the stream closes. This could happen prior to the completion of the request if the underlying channel was closed. + * Implementors should do their best to clean up resources and early terminate request processing if it is triggered before a + * response is generated. + */ + default void streamClose() {} } /** diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 9439102073df4..8766f275a5621 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -32,6 +32,7 @@ import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.transport.Transports; import java.io.IOException; import java.util.ArrayDeque; @@ -148,7 +149,7 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { private IncrementalBulkService.Handler handler; private volatile RestChannel restChannel; - private boolean isException; + private boolean shortCircuited; private final ArrayDeque unParsedChunks = new ArrayDeque<>(4); private final ArrayList> items = new ArrayList<>(4); @@ -178,7 +179,7 @@ public void accept(RestChannel restChannel) { public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boolean isLast) { assert handler != null; assert channel == restChannel; - if (isException) { + if (shortCircuited) { chunk.close(); return; } @@ -215,12 +216,8 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo ); } catch (Exception e) { - // TODO: This needs to be better - Releasables.close(handler); - Releasables.close(unParsedChunks); - unParsedChunks.clear(); + shortCircuit(); new RestToXContentListener<>(channel).onFailure(e); - isException = true; return; } @@ -242,8 +239,16 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo } @Override - public void close() { - RequestBodyChunkConsumer.super.close(); + public void streamClose() { + assert Transports.assertTransportThread(); + shortCircuit(); + } + + private void shortCircuit() { + shortCircuited = true; + Releasables.close(handler); + Releasables.close(unParsedChunks); + unParsedChunks.clear(); } private ArrayList accountParsing(int bytesConsumed) { From dce8a0bfd35e2e3adb1ceffd11d01b0195cfa8f7 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Thu, 12 Sep 2024 10:13:52 -0700 Subject: [PATCH 12/14] merge main --- .../Netty4IncrementalRequestHandlingIT.java | 137 ++++++++++++++++-- .../netty4/Netty4HttpRequestBodyStream.java | 17 +++ .../http/IncrementalBulkRestIT.java | 17 +-- .../java/org/elasticsearch/http/HttpBody.java | 7 + .../http/HttpClientStatsTracker.java | 2 + .../org/elasticsearch/http/HttpTracer.java | 57 +++++++- .../action/document/RestBulkActionTests.java | 3 + 7 files changed, 213 insertions(+), 27 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java index 36ff091f20843..2b9c77b17bced 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4IncrementalRequestHandlingIT.java @@ -37,6 +37,7 @@ import io.netty.handler.stream.ChunkedStream; import io.netty.handler.stream.ChunkedWriteHandler; +import org.apache.logging.log4j.Level; import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.client.internal.node.NodeClient; @@ -52,6 +53,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.http.HttpBodyTracer; import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpTransportSettings; @@ -66,6 +68,8 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.MockLog; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.transport.netty4.Netty4Utils; import java.util.Collection; @@ -75,6 +79,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; @@ -210,10 +215,12 @@ public void testServerCloseConnectionMidStream() throws Exception { public void testClientBackpressure() throws Exception { try (var ctx = setupClientCtx()) { var opaqueId = opaqueId(0); - var payloadSize = MBytes(50); + var payloadSize = maxContentLength(); + var totalParts = 10; + var partSize = payloadSize / totalParts; ctx.clientChannel.writeAndFlush(httpRequest(opaqueId, payloadSize)); - for (int i = 0; i < 5; i++) { - ctx.clientChannel.writeAndFlush(randomContent(MBytes(10), false)); + for (int i = 0; i < totalParts; i++) { + ctx.clientChannel.writeAndFlush(randomContent(partSize, false)); } assertFalse( "should not flush last content immediately", @@ -222,16 +229,15 @@ public void testClientBackpressure() throws Exception { var handler = ctx.awaitRestChannelAccepted(opaqueId); - // Read buffers for socket and channel usually within few MBytes range all together. - // This test assumes that buffers will not exceed 10 MBytes, in other words there should - // be less than 10 MBytes in fly between http client's socket and rest handler. This - // loop ensures that reading 10 MBytes of content on server side should free almost - // same size in client's channel write buffer. - for (int mb = 0; mb <= 50; mb += 10) { - var minBufSize = payloadSize - MBytes(10 + mb); - var maxBufSize = payloadSize - MBytes(mb); + // some data flushes from channel into OS buffer and won't be visible here, usually 4-8Mb + var osBufferOffset = MBytes(10); + + // incrementally read data on server side and ensure client side buffer drains accordingly + for (int readBytes = 0; readBytes <= payloadSize; readBytes += partSize) { + var minBufSize = Math.max(payloadSize - readBytes - osBufferOffset, 0); + var maxBufSize = Math.max(payloadSize - readBytes, 0); // it is hard to tell that client's channel is no logger flushing data - // it might take a few busy-iterations before channel buffer flush to kernel + // it might take a few busy-iterations before channel buffer flush to OS // and bytesBeforeWritable will stop changing assertBusy(() -> { var bufSize = ctx.clientChannel.bytesBeforeWritable(); @@ -240,7 +246,7 @@ public void testClientBackpressure() throws Exception { bufSize >= minBufSize && bufSize <= maxBufSize ); }); - handler.readBytes(MBytes(10)); + handler.readBytes(partSize); } assertTrue(handler.stream.hasLast()); } @@ -351,6 +357,107 @@ public void testBadRequestReleaseQueuedChunks() throws Exception { } } + private static long transportStatsRequestBytesSize(Ctx ctx) { + var httpTransport = internalCluster().getInstance(HttpServerTransport.class, ctx.nodeName); + var stats = httpTransport.stats().clientStats(); + var bytes = 0L; + for (var s : stats) { + bytes += s.requestSizeBytes(); + } + return bytes; + } + + /** + * ensures that {@link org.elasticsearch.http.HttpClientStatsTracker} counts streamed content bytes + */ + public void testHttpClientStats() throws Exception { + try (var ctx = setupClientCtx()) { + // need to offset starting point, since we reuse cluster and other tests already sent some data + var totalBytesSent = transportStatsRequestBytesSize(ctx); + + for (var reqNo = 0; reqNo < randomIntBetween(2, 10); reqNo++) { + var id = opaqueId(reqNo); + var contentSize = randomIntBetween(0, maxContentLength()); + totalBytesSent += contentSize; + ctx.clientChannel.writeAndFlush(httpRequest(id, contentSize)); + ctx.clientChannel.writeAndFlush(randomContent(contentSize, true)); + var handler = ctx.awaitRestChannelAccepted(id); + handler.readAllBytes(); + handler.sendResponse(new RestResponse(RestStatus.OK, "")); + assertEquals(totalBytesSent, transportStatsRequestBytesSize(ctx)); + } + } + } + + /** + * ensures that we log parts of http body and final line + */ + @TestLogging( + reason = "testing TRACE logging", + value = "org.elasticsearch.http.HttpTracer:TRACE,org.elasticsearch.http.HttpBodyTracer:TRACE" + ) + public void testHttpBodyLogging() throws Exception { + assertHttpBodyLogging((ctx) -> () -> { + try { + var req = fullHttpRequest(opaqueId(0), randomByteBuf(8 * 1024)); + ctx.clientChannel.writeAndFlush(req); + var handler = ctx.awaitRestChannelAccepted(opaqueId(0)); + handler.readAllBytes(); + } catch (Exception e) { + fail(e); + } + }); + } + + /** + * ensures that we log some parts of body and final line when connection is closed in the middle + */ + @TestLogging( + reason = "testing TRACE logging", + value = "org.elasticsearch.http.HttpTracer:TRACE,org.elasticsearch.http.HttpBodyTracer:TRACE" + ) + public void testHttpBodyLoggingChannelClose() throws Exception { + assertHttpBodyLogging((ctx) -> () -> { + try { + var req = httpRequest(opaqueId(0), 2 * 8192); + var halfContent = randomContent(8192, false); + ctx.clientChannel.writeAndFlush(req); + ctx.clientChannel.writeAndFlush(halfContent); + var handler = ctx.awaitRestChannelAccepted(opaqueId(0)); + handler.readBytes(8192); + ctx.clientChannel.close(); + handler.stream.next(); + assertBusy(() -> assertTrue(handler.streamClosed)); + } catch (Exception e) { + fail(e); + } + }); + } + + // asserts that we emit at least one logging event for a part and last line + // http body should be large enough to split across multiple lines, > 4kb + private void assertHttpBodyLogging(Function test) throws Exception { + try (var ctx = setupClientCtx()) { + MockLog.assertThatLogger( + test.apply(ctx), + HttpBodyTracer.class, + new MockLog.SeenEventExpectation( + "request part", + HttpBodyTracer.class.getCanonicalName(), + Level.TRACE, + "* request body [part *]*" + ), + new MockLog.SeenEventExpectation( + "request end", + HttpBodyTracer.class.getCanonicalName(), + Level.TRACE, + "* request body (gzip compressed, base64-encoded, and split into * parts on preceding log lines; for details see " + + "https://www.elastic.co/guide/en/elasticsearch/reference/master/modules-network.html#http-rest-request-tracer)" + ) + ); + } + } + private int maxContentLength() { return HttpHandlingSettings.fromSettings(internalCluster().getInstance(Settings.class)).maxContentLength(); } @@ -403,6 +510,10 @@ static HttpContent randomContent(int size, boolean isLast) { } } + static ByteBuf randomByteBuf(int size) { + return Unpooled.wrappedBuffer(randomByteArrayOfLength(size)); + } + Ctx setupClientCtx() throws Exception { var nodeName = internalCluster().getRandomNodeName(); var clientRespQueue = new LinkedBlockingDeque<>(16); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index 7cc0b50e0c7ec..b06dfd6d9a4f8 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -16,9 +16,13 @@ import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.LastHttpContent; +import org.elasticsearch.core.Releasables; import org.elasticsearch.http.HttpBody; import org.elasticsearch.transport.netty4.Netty4Utils; +import java.util.ArrayList; +import java.util.List; + /** * Netty based implementation of {@link HttpBody.Stream}. * This implementation utilize {@link io.netty.channel.ChannelConfig#setAutoRead(boolean)} @@ -29,6 +33,7 @@ public class Netty4HttpRequestBodyStream implements HttpBody.Stream { private final Channel channel; private final ChannelFutureListener closeListener = future -> doClose(); + private final List tracingHandlers = new ArrayList<>(4); private ByteBuf buf; private boolean hasLast = false; private boolean requested = false; @@ -52,6 +57,12 @@ public void setHandler(ChunkHandler chunkHandler) { this.handler = chunkHandler; } + @Override + public void addTracingHandler(ChunkHandler chunkHandler) { + assert tracingHandlers.contains(chunkHandler) == false; + tracingHandlers.add(chunkHandler); + } + @Override public void next() { assert closing == false : "cannot request next chunk on closing stream"; @@ -119,6 +130,9 @@ private void send() { var bytesRef = Netty4Utils.toReleasableBytesReference(buf); requested = false; buf = null; + for (var tracer : tracingHandlers) { + tracer.onNext(bytesRef, hasLast); + } handler.onNext(bytesRef, hasLast); } @@ -133,6 +147,9 @@ public void close() { private void doClose() { closing = true; + for (var tracer : tracingHandlers) { + Releasables.closeExpectNoException(tracer); + } if (handler != null) { handler.close(); } diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java index 4b3b6fbb2a719..a05c3e510c0f8 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java @@ -1,9 +1,10 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.http; @@ -88,9 +89,7 @@ public void testBulkWithIncrementalDisabled() throws IOException { final Response indexSuccessFul = getRestClient().performRequest(firstBulkRequest); assertThat(indexSuccessFul.getStatusLine().getStatusCode(), equalTo(OK.getStatus())); - clusterAdmin().prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), false).build()) - .get(); + updateClusterSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), false)); internalCluster().getInstances(IncrementalBulkService.class).forEach(i -> i.setForTests(false)); @@ -98,9 +97,7 @@ public void testBulkWithIncrementalDisabled() throws IOException { sendLargeBulk(); } finally { internalCluster().getInstances(IncrementalBulkService.class).forEach(i -> i.setForTests(true)); - clusterAdmin().prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), (String) null).build()) - .get(); + updateClusterSettings(Settings.builder().put(IncrementalBulkService.INCREMENTAL_BULK.getKey(), (String) null)); } } diff --git a/server/src/main/java/org/elasticsearch/http/HttpBody.java b/server/src/main/java/org/elasticsearch/http/HttpBody.java index 40fd2fd64ec87..a10487502ed3c 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpBody.java +++ b/server/src/main/java/org/elasticsearch/http/HttpBody.java @@ -73,6 +73,13 @@ non-sealed interface Stream extends HttpBody { @Nullable ChunkHandler handler(); + /** + * Adds tracing chunk handler. Tracing handler will be invoked before main handler, and + * should never release or call for next chunk. It should be used for monitoring and + * logging purposes. + */ + void addTracingHandler(ChunkHandler chunkHandler); + /** * Sets handler that can handle next chunk */ diff --git a/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java index 9097610dc7722..59e45242e46c5 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpClientStatsTracker.java @@ -229,6 +229,8 @@ synchronized void update(HttpRequest httpRequest, HttpChannel httpChannel, long requestCount += 1; if (httpRequest.body().isFull()) { requestSizeBytes += httpRequest.body().asFull().bytes().length(); + } else { + httpRequest.body().asStream().addTracingHandler((chunk, last) -> requestSizeBytes += chunk.length()); } } diff --git a/server/src/main/java/org/elasticsearch/http/HttpTracer.java b/server/src/main/java/org/elasticsearch/http/HttpTracer.java index 81c406b3545ed..3d8360e6ee3fa 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpTracer.java +++ b/server/src/main/java/org/elasticsearch/http/HttpTracer.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Nullable; @@ -21,6 +22,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.io.OutputStream; import java.util.List; @@ -78,10 +80,10 @@ HttpTracer maybeLogRequest(RestRequest restRequest, @Nullable Exception e) { e ); if (isBodyTracerEnabled()) { - try (var stream = HttpBodyTracer.getBodyOutputStream(restRequest.getRequestId(), HttpBodyTracer.Type.REQUEST)) { - restRequest.content().writeTo(stream); - } catch (Exception e2) { - assert false : e2; // no real IO here + if (restRequest.isFullContent()) { + logFullContent(restRequest); + } else { + logStreamContent(restRequest); } } @@ -90,6 +92,53 @@ HttpTracer maybeLogRequest(RestRequest restRequest, @Nullable Exception e) { return null; } + private void logFullContent(RestRequest restRequest) { + try (var stream = HttpBodyTracer.getBodyOutputStream(restRequest.getRequestId(), HttpBodyTracer.Type.REQUEST)) { + restRequest.content().writeTo(stream); + } catch (Exception e2) { + assert false : e2; // no real IO here + } + } + + private void logStreamContent(RestRequest restRequest) { + restRequest.contentStream().addTracingHandler(new LoggingChunkHandler(restRequest)); + } + + private static class LoggingChunkHandler implements HttpBody.ChunkHandler { + private final OutputStream stream; + private volatile boolean closed = false; + + LoggingChunkHandler(RestRequest request) { + stream = HttpBodyTracer.getBodyOutputStream(request.getRequestId(), HttpBodyTracer.Type.REQUEST); + } + + @Override + public void onNext(ReleasableBytesReference chunk, boolean isLast) { + try { + chunk.writeTo(stream); + } catch (IOException e) { + assert false : e; // no real IO + } finally { + if (isLast) { + this.close(); + } + } + } + + @Override + public void close() { + if (closed) { + return; + } + try { + closed = true; + stream.close(); + } catch (IOException e) { + assert false : e; // no real IO + } + } + } + boolean isBodyTracerEnabled() { return HttpBodyTracer.isEnabled(); } diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 3bc39f46bb153..d3cd6dd9ca420 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -211,6 +211,9 @@ public ChunkHandler handler() { return null; } + @Override + public void addTracingHandler(ChunkHandler chunkHandler) {} + @Override public void setHandler(ChunkHandler chunkHandler) {} From 92daeeba119bc3f9dbde31f49924b1ec2c834397 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 16 Sep 2024 17:45:35 -0600 Subject: [PATCH 13/14] Properly handle empty incremental bulk requests (#112974) This commit ensures we properly throw exceptions when an empty bulk request is received with the incremental handling enabled. --- .../test/rest/RequestsWithoutContentIT.java | 8 +- .../http/IncrementalBulkRestIT.java | 22 +++++ .../rest/action/document/RestBulkAction.java | 83 +++++++++++-------- 3 files changed, 74 insertions(+), 39 deletions(-) diff --git a/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java b/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java index 83cb4fb7180ef..8732110bb1937 100644 --- a/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java +++ b/distribution/archives/integ-test-zip/src/javaRestTest/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java @@ -26,12 +26,10 @@ public void testIndexMissingBody() throws IOException { assertResponseException(responseException, "request body is required"); } - @AwaitsFix(bugUrl = "need to decide how to handle this scenario") public void testBulkMissingBody() throws IOException { - ResponseException responseException = expectThrows( - ResponseException.class, - () -> client().performRequest(new Request(randomBoolean() ? "POST" : "PUT", "/_bulk")) - ); + Request request = new Request(randomBoolean() ? "POST" : "PUT", "/_bulk"); + request.setJsonEntity(""); + ResponseException responseException = expectThrows(ResponseException.class, () -> client().performRequest(request)); assertResponseException(responseException, "request body is required"); } diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java index a05c3e510c0f8..08026e0435f33 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/IncrementalBulkRestIT.java @@ -23,11 +23,33 @@ import java.util.Map; import static org.elasticsearch.rest.RestStatus.OK; +import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.Matchers.equalTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, supportsDedicatedMasters = false, numDataNodes = 2, numClientNodes = 0) public class IncrementalBulkRestIT extends HttpSmokeTestCase { + public void testBulkMissingBody() throws IOException { + Request request = new Request(randomBoolean() ? "POST" : "PUT", "/_bulk"); + request.setJsonEntity(""); + ResponseException responseException = expectThrows(ResponseException.class, () -> getRestClient().performRequest(request)); + assertEquals(400, responseException.getResponse().getStatusLine().getStatusCode()); + assertThat(responseException.getMessage(), containsString("request body is required")); + } + + public void testBulkRequestBodyImproperlyTerminated() throws IOException { + Request request = new Request(randomBoolean() ? "POST" : "PUT", "/_bulk"); + // missing final line of the bulk body. cannot process + request.setJsonEntity( + "{\"index\":{\"_index\":\"index_name\",\"_id\":\"1\"}}\n" + + "{\"field\":1}\n" + + "{\"index\":{\"_index\":\"index_name\",\"_id\":\"2\"}" + ); + ResponseException responseException = expectThrows(ResponseException.class, () -> getRestClient().performRequest(request)); + assertEquals(400, responseException.getResponse().getStatusLine().getStatusCode()); + assertThat(responseException.getMessage(), containsString("could not parse bulk request body")); + } + public void testIncrementalBulk() throws IOException { Request createRequest = new Request("PUT", "/index_name"); createRequest.setJsonEntity(""" diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 8766f275a5621..ff87bb834f3e1 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -9,6 +9,7 @@ package org.elasticsearch.rest.action.document; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestParser; @@ -150,6 +151,7 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { private volatile RestChannel restChannel; private boolean shortCircuited; + private int bytesParsed = 0; private final ArrayDeque unParsedChunks = new ArrayDeque<>(4); private final ArrayList> items = new ArrayList<>(4); @@ -186,48 +188,61 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo final BytesReference data; int bytesConsumed; - try { - unParsedChunks.add(chunk); + if (chunk.length() == 0) { + chunk.close(); + bytesConsumed = 0; + } else { + try { + unParsedChunks.add(chunk); - if (unParsedChunks.size() > 1) { - data = CompositeBytesReference.of(unParsedChunks.toArray(new ReleasableBytesReference[0])); - } else { - data = chunk; - } + if (unParsedChunks.size() > 1) { + data = CompositeBytesReference.of(unParsedChunks.toArray(new ReleasableBytesReference[0])); + } else { + data = chunk; + } - // TODO: Check that the behavior here vs. globalRouting, globalPipeline, globalRequireAlias, globalRequireDatsStream in - // BulkRequest#add is fine - bytesConsumed = parser.incrementalParse( - data, - defaultIndex, - defaultRouting, - defaultFetchSourceContext, - defaultPipeline, - defaultRequireAlias, - defaultRequireDataStream, - defaultListExecutedPipelines, - allowExplicitIndex, - request.getXContentType(), - (request, type) -> items.add(request), - items::add, - items::add, - isLast == false, - stringDeduplicator - ); + // TODO: Check that the behavior here vs. globalRouting, globalPipeline, globalRequireAlias, globalRequireDatsStream in + // BulkRequest#add is fine + bytesConsumed = parser.incrementalParse( + data, + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + request.getXContentType(), + (request, type) -> items.add(request), + items::add, + items::add, + isLast == false, + stringDeduplicator + ); + bytesParsed += bytesConsumed; - } catch (Exception e) { - shortCircuit(); - new RestToXContentListener<>(channel).onFailure(e); - return; + } catch (Exception e) { + shortCircuit(); + new RestToXContentListener<>(channel).onFailure( + new ElasticsearchParseException("could not parse bulk request body", e) + ); + return; + } } final ArrayList releasables = accountParsing(bytesConsumed); if (isLast) { assert unParsedChunks.isEmpty(); - assert channel != null; - ArrayList> toPass = new ArrayList<>(items); - items.clear(); - handler.lastItems(toPass, () -> Releasables.close(releasables), new RestRefCountedChunkedToXContentListener<>(channel)); + if (bytesParsed == 0) { + shortCircuit(); + new RestToXContentListener<>(channel).onFailure(new ElasticsearchParseException("request body is required")); + } else { + assert channel != null; + ArrayList> toPass = new ArrayList<>(items); + items.clear(); + handler.lastItems(toPass, () -> Releasables.close(releasables), new RestRefCountedChunkedToXContentListener<>(channel)); + } } else if (items.isEmpty() == false) { ArrayList> toPass = new ArrayList<>(items); items.clear(); From 529d349a250145419d1094c49951e302c60b0959 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 18 Sep 2024 13:55:08 -0600 Subject: [PATCH 14/14] Fix spotless in netty stream class Spotless broke during a rebase. Fixing in this commit. --- .../elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java | 1 - .../main/java/org/elasticsearch/action/bulk/BulkOperation.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java index b06dfd6d9a4f8..96f7deea978d9 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpRequestBodyStream.java @@ -40,7 +40,6 @@ public class Netty4HttpRequestBodyStream implements HttpBody.Stream { private boolean closing = false; private HttpBody.ChunkHandler handler; - public Netty4HttpRequestBodyStream(Channel channel) { this.channel = channel; Netty4Utils.addListener(channel.closeFuture(), closeListener); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java index 350967684f00d..13229fbf65fef 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java @@ -438,7 +438,7 @@ private void discardRedirectsAndFinish(Exception exception) { private void executeBulkShardRequest(BulkShardRequest bulkShardRequest, Releasable releaseOnFinish) { ShardId shardId = bulkShardRequest.shardId(); - // Short circuit the shark level request with the existing shard failure. + // Short circuit the shard level request with the existing shard failure. if (shortCircuitShardFailures.containsKey(shardId)) { handleShardFailure(bulkShardRequest, clusterService.state(), shortCircuitShardFailures.get(shardId)); releaseOnFinish.close();