From 8cc388020c210065b4ab19090823610a424fc91d Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 28 Jul 2017 22:12:08 -0500 Subject: [PATCH 01/35] WIPD --- .../http/netty4/Netty4HttpChannel.java | 4 +- .../http/netty4/Netty4HttpRequest.java | 2 +- .../nio/http/NioHttpNettyAdaptor.java | 126 +++++++ .../nio/http/NioHttpRequestHandler.java | 87 +++++ .../transport/nio/http/NioHttpTransport.java | 318 ++++++++++++++++++ test/framework/build.gradle | 7 + .../transport/nio/NioSelectors.java | 92 +++++ .../transport/nio/NioShutdown.java | 4 +- .../transport/nio/NioTransport.java | 40 +-- 9 files changed, 644 insertions(+), 36 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java create mode 100644 test/framework/src/main/java/org/elasticsearch/transport/nio/NioSelectors.java diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java index 12db47908d1f3..be2411d33fab9 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java @@ -53,7 +53,7 @@ import java.util.Map; import java.util.Set; -final class Netty4HttpChannel extends AbstractRestChannel { +public final class Netty4HttpChannel extends AbstractRestChannel { private final Netty4HttpServerTransport transport; private final Channel channel; @@ -69,7 +69,7 @@ final class Netty4HttpChannel extends AbstractRestChannel { * @param detailedErrorsEnabled true iff error messages should include stack traces. * @param threadContext the thread context for the channel */ - Netty4HttpChannel( + public Netty4HttpChannel( final Netty4HttpServerTransport transport, final Netty4HttpRequest request, final HttpPipelinedRequest pipelinedRequest, 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 f3099db08e992..c1b5632aadf21 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 @@ -45,7 +45,7 @@ public class Netty4HttpRequest extends RestRequest { private final Channel channel; private final BytesReference content; - Netty4HttpRequest(NamedXContentRegistry xContentRegistry, FullHttpRequest request, Channel channel) { + public Netty4HttpRequest(NamedXContentRegistry xContentRegistry, FullHttpRequest request, Channel channel) { super(xContentRegistry, request.uri(), new HttpHeadersMap(request.headers())); this.request = request; this.channel = channel; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java new file mode 100644 index 0000000000000..614c7c1ded281 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java @@ -0,0 +1,126 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.transport.nio.http; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.http.HttpContentCompressor; +import io.netty.handler.codec.http.HttpContentDecompressor; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponseEncoder; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; +import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; +import org.elasticsearch.http.netty4.pipelining.HttpPipeliningHandler; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION_LEVEL; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; + +public class NioHttpNettyAdaptor { + + + private final NioHttpTransport transport; + private final Netty4CorsConfig config; + private final int maxContentLength; + private final NioHttpRequestHandler requestHandler; + private final Netty4CorsConfig corsConfig; + private final boolean compression; + private final int compressionLevel; + private final boolean detailedErrorsEnabled; + private final boolean corsEnabled; + private boolean pipelining; + private final int pipeliningMaxEvents; + private final int maxChunkSize; + private final int maxHeaderSize; + private final int maxInitialLineLength; + + protected NioHttpNettyAdaptor(NioHttpTransport transport, NamedXContentRegistry xContentRegistry, ThreadContext threadContext, + Settings settings, Netty4CorsConfig config, int maxContentLength) { + this.transport = transport; + this.config = config; + this.maxContentLength = maxContentLength; + this.detailedErrorsEnabled = SETTING_HTTP_DETAILED_ERRORS_ENABLED.get(settings); + + this.requestHandler = new NioHttpRequestHandler(transport, xContentRegistry, detailedErrorsEnabled, threadContext, pipelining); + + this.compression = SETTING_HTTP_COMPRESSION.get(settings); + this.compressionLevel = SETTING_HTTP_COMPRESSION_LEVEL.get(settings); + this.pipelining = SETTING_PIPELINING.get(settings); + this.pipeliningMaxEvents = SETTING_PIPELINING_MAX_EVENTS.get(settings); + this.corsConfig = config; + this.maxChunkSize = Math.toIntExact(SETTING_HTTP_MAX_CHUNK_SIZE.get(settings).getBytes()); + this.maxHeaderSize = Math.toIntExact(SETTING_HTTP_MAX_HEADER_SIZE.get(settings).getBytes()); + this.maxInitialLineLength = Math.toIntExact(SETTING_HTTP_MAX_INITIAL_LINE_LENGTH.get(settings).getBytes()); + this.corsEnabled = SETTING_CORS_ENABLED.get(settings); + } + + protected void initChannel(NioSocketChannel channel) throws Exception { + pipelining = false; + + EmbeddedChannel ch = new ESNettyChannel(channel); + + final HttpRequestDecoder decoder = new HttpRequestDecoder(maxInitialLineLength, maxHeaderSize, maxChunkSize); + ch.pipeline().addLast(decoder); + ch.pipeline().addLast(new HttpContentDecompressor()); + ch.pipeline().addLast(new HttpResponseEncoder()); + decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR); + final HttpObjectAggregator aggregator = new HttpObjectAggregator(maxContentLength); +// if (maxCompositeBufferComponents != -1) { +// aggregator.setMaxCumulationBufferComponents(maxCompositeBufferComponents); +// } + ch.pipeline().addLast(aggregator); + if (compression) { + ch.pipeline().addLast("encoder_compress", new HttpContentCompressor(compressionLevel)); + } + if (corsConfig.isCorsSupportEnabled()) { + ch.pipeline().addLast("cors", new Netty4CorsHandler(config)); + } + if (pipelining) { + ch.pipeline().addLast("pipelining", new HttpPipeliningHandler(pipeliningMaxEvents)); + } + ch.pipeline().addLast("handler", requestHandler); + } + + public static class ESNettyChannel extends EmbeddedChannel { + + private final NioSocketChannel nioSocketChannel; + + private ESNettyChannel(NioSocketChannel nioSocketChannel) { + this.nioSocketChannel = nioSocketChannel; + } + + public NioSocketChannel getNioSocketChannel() { + return nioSocketChannel; + } + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java new file mode 100644 index 0000000000000..785c5a6514b38 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java @@ -0,0 +1,87 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.http; + +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.http.netty4.Netty4HttpChannel; +import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; + +public class NioHttpRequestHandler extends SimpleChannelInboundHandler { + + private final NioHttpTransport transport; + private final NamedXContentRegistry xContentRegistry; + private final ThreadContext threadContext; + private final boolean detailedErrorsEnabled; + private final boolean httpPipeliningEnabled; + + public NioHttpRequestHandler(NioHttpTransport transport, NamedXContentRegistry xContentRegistry, boolean detailedErrorsEnabled, + ThreadContext threadContext, boolean httpPipeliningEnabled) { + this.transport = transport; + this.detailedErrorsEnabled = detailedErrorsEnabled; + this.threadContext = threadContext; + this.httpPipeliningEnabled = httpPipeliningEnabled; + this.xContentRegistry = xContentRegistry; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Exception { + final FullHttpRequest request; + final HttpPipelinedRequest pipelinedRequest; + if (this.httpPipeliningEnabled && msg instanceof HttpPipelinedRequest) { + pipelinedRequest = (HttpPipelinedRequest) msg; + request = (FullHttpRequest) pipelinedRequest.last(); + } else { + pipelinedRequest = null; + request = (FullHttpRequest) msg; + } + + final FullHttpRequest copy = + new DefaultFullHttpRequest( + request.protocolVersion(), + request.method(), + request.uri(), + Unpooled.copiedBuffer(request.content()), + request.headers(), + request.trailingHeaders()); + final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, ctx.channel()); + final Netty4HttpChannel channel = new Netty4HttpChannel(null, httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext); + + if (request.decoderResult().isSuccess()) { + transport.dispatchRequest(httpRequest, channel); + } else { + assert request.decoderResult().isFailure(); + transport.dispatchBadRequest(httpRequest, channel, request.decoderResult().cause()); + } + + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + NioHttpNettyAdaptor.ESNettyChannel channel = (NioHttpNettyAdaptor.ESNettyChannel) ctx.channel(); + transport.exceptionCaught(channel.getNioSocketChannel(), cause); + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java new file mode 100644 index 0000000000000..ba6ed9f6229f7 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java @@ -0,0 +1,318 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.http; + +import com.carrotsearch.hppc.IntHashSet; +import com.carrotsearch.hppc.IntSet; +import io.netty.handler.timeout.ReadTimeoutException; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Supplier; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.network.NetworkService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.BoundTransportAddress; +import org.elasticsearch.common.transport.NetworkExceptionHelper; +import org.elasticsearch.common.transport.PortsRange; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.http.BindHttpException; +import org.elasticsearch.http.HttpInfo; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.http.HttpStats; +import org.elasticsearch.http.netty4.Netty4HttpChannel; +import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.BindTransportException; +import org.elasticsearch.transport.nio.AcceptingSelector; +import org.elasticsearch.transport.nio.NioSelectors; +import org.elasticsearch.transport.nio.NioShutdown; +import org.elasticsearch.transport.nio.NioTransport; +import org.elasticsearch.transport.nio.OpenChannels; +import org.elasticsearch.transport.nio.SocketEventHandler; +import org.elasticsearch.transport.nio.SocketSelector; +import org.elasticsearch.transport.nio.channel.ChannelFactory; +import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import static org.elasticsearch.common.settings.Setting.boolSetting; +import static org.elasticsearch.common.settings.Setting.intSetting; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_BIND_HOST; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PORT; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT; +import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX; +import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_WORKER_THREAD_NAME_PREFIX; + +public class NioHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport { + + public static final Setting NIO_HTTP_WORKER_COUNT = + new Setting<>("transport.nio.http.worker_count", + (s) -> Integer.toString(EsExecutors.numberOfProcessors(s) * 2), + (s) -> Setting.parseInt(s, 1, "transport.nio.worker_count"), Setting.Property.NodeScope); + + public static final Setting NIO_HTTP_ACCEPTOR_COUNT = + intSetting("transport.nio.http.acceptor_count", 1, 1, Setting.Property.NodeScope); + + public static final Setting SETTING_HTTP_TCP_NO_DELAY = + boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = + boolSetting("http.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_REUSE_ADDRESS = + boolSetting("http.tcp.reuse_address", NetworkService.TCP_REUSE_ADDRESS, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_SEND_BUFFER_SIZE = + Setting.byteSizeSetting("http.tcp.send_buffer_size", NetworkService.TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = + Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); + + private final PortsRange port; + private final NetworkService networkService; + private final BigArrays bigArrays; + private final ThreadPool threadPool; + private final NamedXContentRegistry xContentRegistry; + private final Dispatcher dispatcher; + private final String[] publishHosts; + private final String bindHosts[]; + private OpenChannels openChannels; + private final ByteSizeValue maxContentLength; + private ArrayList socketSelectors; + private ArrayList acceptors; + private BoundTransportAddress boundAddress; + private int acceptorNumber; + + public NioHttpTransport(Settings settings, NetworkService networkService, BigArrays bigArrays, ThreadPool threadPool, + NamedXContentRegistry xContentRegistry, Dispatcher dispatcher) { + super(settings); + this.networkService = networkService; + this.bigArrays = bigArrays; + this.threadPool = threadPool; + this.xContentRegistry = xContentRegistry; + this.dispatcher = dispatcher; + List httpBindHost = SETTING_HTTP_BIND_HOST.get(settings); + this.bindHosts = (httpBindHost.isEmpty() ? NetworkService.GLOBAL_NETWORK_BINDHOST_SETTING.get(settings) : httpBindHost) + .toArray(Strings.EMPTY_ARRAY); + this.port = SETTING_HTTP_PORT.get(settings); + // we can't make the network.publish_host a fallback since we already fall back to http.host hence the extra conditional here + List httpPublishHost = SETTING_HTTP_PUBLISH_HOST.get(settings); + this.publishHosts = (httpPublishHost.isEmpty() ? NetworkService.GLOBAL_NETWORK_PUBLISHHOST_SETTING.get(settings) : httpPublishHost) + .toArray(Strings.EMPTY_ARRAY); + maxContentLength = SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings); + } + + @Override + protected void doStart() { + boolean success = false; + try { + this.openChannels = new OpenChannels(logger); + + socketSelectors = NioSelectors.socketSelectors(settings, () -> new SocketEventHandler(logger, this::exceptionCaught), + NioTransport.NIO_WORKER_COUNT.get(settings), TRANSPORT_WORKER_THREAD_NAME_PREFIX); + + acceptors = NioSelectors.acceptingSelectors(logger, settings, openChannels, socketSelectors, + NioTransport.NIO_ACCEPTOR_COUNT.get(settings), TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); + + this.boundAddress = createBoundHttpAddress(); + if (logger.isInfoEnabled()) { + logger.info("{}", boundAddress); + } + success = true; + } finally { + if (success == false) { + doStop(); // otherwise we leak threads since we never moved to started + } + } + } + + @Override + protected void doStop() { + NioShutdown nioShutdown = new NioShutdown(logger); + nioShutdown.orderlyShutdown(openChannels, null, acceptors, socketSelectors); + } + + @Override + protected void doClose() throws IOException { + } + + @Override + public BoundTransportAddress boundAddress() { + return this.boundAddress; + } + + @Override + public HttpInfo info() { + BoundTransportAddress boundTransportAddress = boundAddress(); + if (boundTransportAddress == null) { + return null; + } + return new HttpInfo(boundTransportAddress, maxContentLength.getBytes()); + } + + @Override + public HttpStats stats() { + long serverChannelsCount = openChannels.serverChannelsCount(); + return new HttpStats(openChannels == null ? 0 : serverChannelsCount, openChannels == null + ? 0 : openChannels.getAcceptedChannels().size() + serverChannelsCount); + } + + public void dispatchRequest(Netty4HttpRequest httpRequest, Netty4HttpChannel channel) { + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + dispatcher.dispatchRequest(httpRequest, channel, threadContext); + } + } + + public void dispatchBadRequest(Netty4HttpRequest httpRequest, Netty4HttpChannel channel, Throwable cause) { + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + dispatcher.dispatchBadRequest(httpRequest, channel, threadContext, cause); + } + } + + protected void exceptionCaught(NioSocketChannel channel, Throwable cause) { + // UH? Read timeout logged as connection? Also no read timeout handler currently + if (cause instanceof ReadTimeoutException) { + if (logger.isTraceEnabled()) { + logger.trace("Connection timeout [{}]", channel.getRemoteAddress()); + } + channel.closeAsync(); + } else { + if (!lifecycle.started()) { + // ignore + return; + } + // TODO: Does channel need to implement toString? + if (!NetworkExceptionHelper.isCloseConnectionException(cause)) { + logger.warn( + (Supplier) () -> new ParameterizedMessage( + "caught exception while handling client http traffic, closing connection {}", channel), + cause); + channel.closeAsync(); + } else { + logger.debug( + (Supplier) () -> new ParameterizedMessage( + "caught exception while handling client http traffic, closing connection {}", channel), + cause); + channel.closeAsync(); + } + } + } + + private BoundTransportAddress createBoundHttpAddress() { + // Bind and start to accept incoming connections. + InetAddress hostAddresses[]; + try { + hostAddresses = networkService.resolveBindHostAddresses(bindHosts); + } catch (IOException e) { + throw new BindHttpException("Failed to resolve host [" + Arrays.toString(bindHosts) + "]", e); + } + + List boundAddresses = new ArrayList<>(hostAddresses.length); + for (InetAddress address : hostAddresses) { + boundAddresses.add(bindAddress(address)); + } + + final InetAddress publishInetAddress; + try { + publishInetAddress = networkService.resolvePublishHostAddresses(publishHosts); + } catch (Exception e) { + throw new BindTransportException("Failed to resolve publish address", e); + } + + final int publishPort = resolvePublishPort(settings, boundAddresses, publishInetAddress); + final InetSocketAddress publishAddress = new InetSocketAddress(publishInetAddress, publishPort); + return new BoundTransportAddress(boundAddresses.toArray(new TransportAddress[0]), new TransportAddress(publishAddress)); + } + + private static int resolvePublishPort(Settings settings, List boundAddresses, InetAddress publishInetAddress) { + int publishPort = SETTING_HTTP_PUBLISH_PORT.get(settings); + + if (publishPort < 0) { + for (TransportAddress boundAddress : boundAddresses) { + InetAddress boundInetAddress = boundAddress.address().getAddress(); + if (boundInetAddress.isAnyLocalAddress() || boundInetAddress.equals(publishInetAddress)) { + publishPort = boundAddress.getPort(); + break; + } + } + } + + // if no matching boundAddress found, check if there is a unique port for all bound addresses + if (publishPort < 0) { + final IntSet ports = new IntHashSet(); + for (TransportAddress boundAddress : boundAddresses) { + ports.add(boundAddress.getPort()); + } + if (ports.size() == 1) { + publishPort = ports.iterator().next().value; + } + } + + if (publishPort < 0) { + throw new BindHttpException("Failed to auto-resolve http publish port, multiple bound addresses " + boundAddresses + + " with distinct ports and none of them matched the publish address (" + publishInetAddress + "). " + + "Please specify a unique port by setting " + SETTING_HTTP_PORT.getKey() + " or " + SETTING_HTTP_PUBLISH_PORT.getKey()); + } + return publishPort; + } + + private TransportAddress bindAddress(final InetAddress hostAddress) { + final AtomicReference lastException = new AtomicReference<>(); + final AtomicReference boundSocket = new AtomicReference<>(); + boolean success = port.iterate(portNumber -> { + try { + synchronized (this) { + AcceptingSelector selector = acceptors.get(++acceptorNumber % NioTransport.NIO_ACCEPTOR_COUNT.get(settings)); + InetSocketAddress address = new InetSocketAddress(hostAddress, portNumber); + ChannelFactory channelFactory = new ChannelFactory(null, null); + NioServerSocketChannel serverChannel = channelFactory.openNioServerSocketChannel("http-server", address, selector); + selector.scheduleForRegistration(serverChannel); + boundSocket.set(serverChannel.getLocalAddress()); + } + } catch (Exception e) { + lastException.set(e); + return false; + } + return true; + }); + if (!success) { + throw new BindHttpException("Failed to bind to [" + port.getPortRangeString() + "]", lastException.get()); + } + + if (logger.isDebugEnabled()) { + logger.debug("Bound http to address {{}}", NetworkAddress.format(boundSocket.get())); + } + return new TransportAddress(boundSocket.get()); + } +} diff --git a/test/framework/build.gradle b/test/framework/build.gradle index cc132a9d71820..161dbea71a47a 100644 --- a/test/framework/build.gradle +++ b/test/framework/build.gradle @@ -32,6 +32,13 @@ dependencies { compile "commons-codec:commons-codec:${versions.commonscodec}" compile "org.elasticsearch:securemock:${versions.securemock}" compile "org.elasticsearch:mocksocket:${versions.mocksocket}" + compile "io.netty:netty-buffer:4.1.13.Final" + compile "io.netty:netty-codec:4.1.13.Final" + compile "io.netty:netty-codec-http:4.1.13.Final" + compile "io.netty:netty-common:4.1.13.Final" + compile "io.netty:netty-handler:4.1.13.Final" + compile "io.netty:netty-resolver:4.1.13.Final" + compile "io.netty:netty-transport:4.1.13.Final" } compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes,-try,-unchecked' diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioSelectors.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioSelectors.java new file mode 100644 index 0000000000000..6c6bf3283d1a2 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioSelectors.java @@ -0,0 +1,92 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio; + +import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.settings.Settings; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.concurrent.ThreadFactory; +import java.util.function.Supplier; + +import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; + +public class NioSelectors { + + public static ArrayList socketSelectors(Settings settings, Supplier eventHandlerSupplier, + int workerCount, String threadNamePrefix) { + ArrayList socketSelectors = new ArrayList<>(workerCount); + + try { + for (int i = 0; i < workerCount; ++i) { + SocketSelector selector = new SocketSelector(eventHandlerSupplier.get()); + socketSelectors.add(selector); + } + } catch (IOException e) { + for (SocketSelector selector : socketSelectors) { + IOUtils.closeWhileHandlingException(selector.rawSelector()); + } + throw new ElasticsearchException(e); + } + + for (SocketSelector selector : socketSelectors) { + if (selector.isRunning() == false) { + ThreadFactory threadFactory = daemonThreadFactory(settings, threadNamePrefix); + threadFactory.newThread(selector::runLoop).start(); + selector.isRunningFuture().actionGet(); + } + } + + return socketSelectors; + } + + public static ArrayList acceptingSelectors(Logger logger, Settings settings, OpenChannels openChannels, + ArrayList socketSelectors, int acceptorCount, + String threadNamePrefix) { + ArrayList acceptors = new ArrayList<>(acceptorCount); + + try { + for (int i = 0; i < acceptorCount; ++i) { + Supplier selectorSupplier = new RoundRobinSelectorSupplier(socketSelectors); + AcceptorEventHandler eventHandler = new AcceptorEventHandler(logger, openChannels, selectorSupplier); + AcceptingSelector acceptor = new AcceptingSelector(eventHandler); + acceptors.add(acceptor); + } + } catch (IOException e) { + for (AcceptingSelector selector : acceptors) { + IOUtils.closeWhileHandlingException(selector.rawSelector()); + } + throw new ElasticsearchException(e); + } + + for (AcceptingSelector acceptor : acceptors) { + if (acceptor.isRunning() == false) { + ThreadFactory threadFactory = daemonThreadFactory(settings, threadNamePrefix); + threadFactory.newThread(acceptor::runLoop).start(); + acceptor.isRunningFuture().actionGet(); + } + } + + return acceptors; + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java index 8dc87f80f8aa3..57d044069be9e 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioShutdown.java @@ -34,10 +34,10 @@ public NioShutdown(Logger logger) { this.logger = logger; } - void orderlyShutdown(OpenChannels openChannels, NioClient client, ArrayList acceptors, + public void orderlyShutdown(OpenChannels openChannels, NioClient client, ArrayList acceptors, ArrayList socketSelectors) { // Close the client. This ensures that no new send connections will be opened. Client could be null if exception was - // throw on start up + // throw on start up. Additionally, the http transport does not have a client if (client != null) { client.close(); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index a621925140090..4191bc85ce0af 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -45,6 +45,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadFactory; @@ -71,8 +72,8 @@ public class NioTransport extends TcpTransport { private final TcpReadHandler tcpReadHandler = new TcpReadHandler(this); private final ConcurrentMap profileToChannelFactory = newConcurrentMap(); private final OpenChannels openChannels = new OpenChannels(logger); - private final ArrayList acceptors = new ArrayList<>(); - private final ArrayList socketSelectors = new ArrayList<>(); + private ArrayList acceptors; + private ArrayList socketSelectors; private NioClient client; private int acceptorNumber; @@ -163,50 +164,27 @@ protected boolean isOpen(NioChannel channel) { protected void doStart() { boolean success = false; try { - int workerCount = NioTransport.NIO_WORKER_COUNT.get(settings); - for (int i = 0; i < workerCount; ++i) { - SocketSelector selector = new SocketSelector(getSocketEventHandler()); - socketSelectors.add(selector); - } - - for (SocketSelector selector : socketSelectors) { - if (selector.isRunning() == false) { - ThreadFactory threadFactory = daemonThreadFactory(this.settings, TRANSPORT_WORKER_THREAD_NAME_PREFIX); - threadFactory.newThread(selector::runLoop).start(); - selector.isRunningFuture().actionGet(); - } - } + socketSelectors = NioSelectors.socketSelectors(settings, this::getSocketEventHandler, + NioTransport.NIO_WORKER_COUNT.get(settings), TRANSPORT_WORKER_THREAD_NAME_PREFIX); client = createClient(); if (NetworkService.NETWORK_SERVER.get(settings)) { int acceptorCount = NioTransport.NIO_ACCEPTOR_COUNT.get(settings); - for (int i = 0; i < acceptorCount; ++i) { - Supplier selectorSupplier = new RoundRobinSelectorSupplier(socketSelectors); - AcceptorEventHandler eventHandler = new AcceptorEventHandler(logger, openChannels, selectorSupplier); - AcceptingSelector acceptor = new AcceptingSelector(eventHandler); - acceptors.add(acceptor); - } - - for (AcceptingSelector acceptor : acceptors) { - if (acceptor.isRunning() == false) { - ThreadFactory threadFactory = daemonThreadFactory(this.settings, TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); - threadFactory.newThread(acceptor::runLoop).start(); - acceptor.isRunningFuture().actionGet(); - } - } + acceptors = NioSelectors.acceptingSelectors(logger, settings, openChannels, socketSelectors, acceptorCount, + TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); // loop through all profiles and start them up, special handling for default one for (ProfileSettings profileSettings : profileSettings) { profileToChannelFactory.putIfAbsent(profileSettings.profileName, new ChannelFactory(profileSettings, tcpReadHandler)); bindServer(profileSettings); } + } else { + acceptors = new ArrayList<>(); } super.doStart(); success = true; - } catch (IOException e) { - throw new ElasticsearchException(e); } finally { if (success == false) { doStop(); From 7f0cc5a87933953faa8f05d91c1c44914556d8e3 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sat, 29 Jul 2017 14:56:02 -0500 Subject: [PATCH 02/35] WIP --- .../http/nio/HttpReadContext.java | 78 +++++ .../http/nio/HttpWriteContext.java | 55 ++++ .../http/nio/NioHttpChannel.java | 284 ++++++++++++++++++ .../nio}/NioHttpNettyAdaptor.java | 73 +++-- .../http/nio/NioHttpNettyAdaptorTests.java | 60 ++++ .../nio}/NioHttpRequestHandler.java | 31 +- .../http => http/nio}/NioHttpTransport.java | 7 +- .../transport/nio/channel/ReadContext.java | 2 + .../transport/nio/channel/TcpReadContext.java | 3 - 9 files changed, 531 insertions(+), 62 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java rename modules/transport-netty4/src/test/java/org/elasticsearch/{transport/nio/http => http/nio}/NioHttpNettyAdaptor.java (71%) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java rename modules/transport-netty4/src/test/java/org/elasticsearch/{transport/nio/http => http/nio}/NioHttpRequestHandler.java (76%) rename modules/transport-netty4/src/test/java/org/elasticsearch/{transport/nio/http => http/nio}/NioHttpTransport.java (98%) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java new file mode 100644 index 0000000000000..307d747db5470 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -0,0 +1,78 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.transport.nio.NetworkBytesReference; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.channel.ReadContext; + +import java.io.IOException; +import java.util.LinkedList; + +public class HttpReadContext implements ReadContext { + + private final NioSocketChannel channel; + private final EmbeddedChannel nettyReadHandler; + private final LinkedList references = new LinkedList<>(); + + public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor) { + this.channel = channel; + this.nettyReadHandler = adaptor.getAdaptor(channel); + } + + @Override + public int read() throws IOException { + NetworkBytesReference last = references.peekLast(); + if (last == null || last.hasWriteRemaining() == false) { + this.references.add(NetworkBytesReference.wrap(new BytesArray(new byte[DEFAULT_READ_LENGTH]))); + } + + int bytesRead = channel.read(references.getLast()); + + if (bytesRead == -1) { + return bytesRead; + } + + int size = references.size(); + + ByteBuf inboundBytes = toByteBuf(size); + + nettyReadHandler.writeInbound(inboundBytes); + + return bytesRead; + } + + private ByteBuf toByteBuf(int size) { + if (size == 1) { + return Unpooled.wrappedBuffer(references.getFirst().getReadByteBuffer()); + } else { + CompositeByteBuf byteBuf = Unpooled.compositeBuffer(size); + for (NetworkBytesReference reference : references) { + byteBuf.addComponent(Unpooled.wrappedBuffer(reference.getReadByteBuffer())); + } + return byteBuf; + } + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java new file mode 100644 index 0000000000000..bbb3407aabcba --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -0,0 +1,55 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.WriteContext; + +import java.io.IOException; + +public class HttpWriteContext implements WriteContext { + @Override + public void sendMessage(BytesReference reference, ActionListener listener) { + + } + + @Override + public void queueWriteOperations(WriteOperation writeOperation) { + + } + + @Override + public void flushChannel() throws IOException { + + } + + @Override + public boolean hasQueuedWriteOps() { + return false; + } + + @Override + public void clearQueuedWriteOps(Exception e) { + + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java new file mode 100644 index 0000000000000..273debc785965 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java @@ -0,0 +1,284 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelPromise; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpHeaderValues; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.cookie.ServerCookieDecoder; +import io.netty.handler.codec.http.cookie.ServerCookieEncoder; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; +import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; +import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; +import org.elasticsearch.rest.AbstractRestChannel; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.transport.netty4.Netty4Utils; + +import java.util.Collections; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public final class NioHttpChannel extends AbstractRestChannel { + + private final Channel channel; + private final FullHttpRequest nettyRequest; + private final HttpPipelinedRequest pipelinedRequest; + private final ThreadContext threadContext; + private final Netty4CorsConfig corsConfig; + private final boolean resetCookies; + + /** + * @param request The request that is handled by this channel. + * @param pipelinedRequest If HTTP pipelining is enabled provide the corresponding pipelined request. May be null if + * HTTP pipelining is disabled. + * @param detailedErrorsEnabled true iff error messages should include stack traces. + * @param threadContext the thread context for the channel + */ + public NioHttpChannel(Netty4HttpRequest request, HttpPipelinedRequest pipelinedRequest, boolean detailedErrorsEnabled, + ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { + super(request, detailedErrorsEnabled); + this.channel = request.getChannel(); + this.nettyRequest = request.request(); + this.pipelinedRequest = pipelinedRequest; + this.threadContext = threadContext; + this.corsConfig = corsConfig; + this.resetCookies = resetCookies; + } + + @Override + protected BytesStreamOutput newBytesOutput() { + BigArrays bigArrays = BigArrays.NON_RECYCLING_INSTANCE; + return new ReleasableBytesStreamOutput(bigArrays); + } + + @Override + public void sendResponse(RestResponse response) { + // if the response object was created upstream, then use it; + // otherwise, create a new one + ByteBuf buffer = Netty4Utils.toByteBuf(response.content()); + final FullHttpResponse resp; + if (HttpMethod.HEAD.equals(nettyRequest.method())) { + resp = newResponse(Unpooled.EMPTY_BUFFER); + } else { + resp = newResponse(buffer); + } + resp.setStatus(getStatus(response.status())); + + Netty4CorsHandler.setCorsResponseHeaders(nettyRequest, resp, corsConfig); + + String opaque = nettyRequest.headers().get("X-Opaque-Id"); + if (opaque != null) { + setHeaderField(resp, "X-Opaque-Id", opaque); + } + + // Add all custom headers + addCustomHeaders(resp, response.getHeaders()); + addCustomHeaders(resp, threadContext.getResponseHeaders()); + + BytesReference content = response.content(); + boolean releaseContent = content instanceof Releasable; + boolean releaseBytesStreamOutput = bytesOutputOrNull() instanceof ReleasableBytesStreamOutput; + try { + // If our response doesn't specify a content-type header, set one + setHeaderField(resp, HttpHeaderNames.CONTENT_TYPE.toString(), response.contentType(), false); + // If our response has no content-length, calculate and set one + setHeaderField(resp, HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(buffer.readableBytes()), false); + + addCookies(resp); + + final ChannelPromise promise = channel.newPromise(); + + if (releaseContent) { + promise.addListener(f -> ((Releasable) content).close()); + } + + if (releaseBytesStreamOutput) { + promise.addListener(f -> bytesOutputOrNull().close()); + } + + if (isCloseConnection()) { + promise.addListener(ChannelFutureListener.CLOSE); + } + + final Object msg; + if (pipelinedRequest != null) { + msg = pipelinedRequest.createHttpResponse(resp, promise); + } else { + msg = resp; + } + channel.writeAndFlush(msg, promise); + releaseContent = false; + releaseBytesStreamOutput = false; + } finally { + if (releaseContent) { + ((Releasable) content).close(); + } + if (releaseBytesStreamOutput) { + bytesOutputOrNull().close(); + } + if (pipelinedRequest != null) { + pipelinedRequest.release(); + } + } + } + + private void setHeaderField(HttpResponse resp, String headerField, String value) { + setHeaderField(resp, headerField, value, true); + } + + private void setHeaderField(HttpResponse resp, String headerField, String value, boolean override) { + if (override || !resp.headers().contains(headerField)) { + resp.headers().add(headerField, value); + } + } + + private void addCookies(HttpResponse resp) { + if (resetCookies) { + String cookieString = nettyRequest.headers().get(HttpHeaderNames.COOKIE); + if (cookieString != null) { + Set cookies = ServerCookieDecoder.STRICT.decode(cookieString); + if (!cookies.isEmpty()) { + // Reset the cookies if necessary. + resp.headers().set(HttpHeaderNames.SET_COOKIE, ServerCookieEncoder.STRICT.encode(cookies)); + } + } + } + } + + private void addCustomHeaders(HttpResponse response, Map> customHeaders) { + if (customHeaders != null) { + for (Map.Entry> headerEntry : customHeaders.entrySet()) { + for (String headerValue : headerEntry.getValue()) { + setHeaderField(response, headerEntry.getKey(), headerValue); + } + } + } + } + + // Determine if the request protocol version is HTTP 1.0 + private boolean isHttp10() { + return nettyRequest.protocolVersion().equals(HttpVersion.HTTP_1_0); + } + + // Determine if the request connection should be closed on completion. + private boolean isCloseConnection() { + final boolean http10 = isHttp10(); + return HttpHeaderValues.CLOSE.contentEqualsIgnoreCase(nettyRequest.headers().get(HttpHeaderNames.CONNECTION)) || + (http10 && !HttpHeaderValues.KEEP_ALIVE.contentEqualsIgnoreCase(nettyRequest.headers().get(HttpHeaderNames.CONNECTION))); + } + + // Create a new {@link HttpResponse} to transmit the response for the netty request. + private FullHttpResponse newResponse(ByteBuf buffer) { + final boolean http10 = isHttp10(); + final boolean close = isCloseConnection(); + // Build the response object. + final HttpResponseStatus status = HttpResponseStatus.OK; // default to initialize + final FullHttpResponse response; + if (http10) { + response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_0, status, buffer); + if (!close) { + response.headers().add(HttpHeaderNames.CONNECTION, "Keep-Alive"); + } + } else { + response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status, buffer); + } + return response; + } + + private static final HttpResponseStatus TOO_MANY_REQUESTS = new HttpResponseStatus(429, "Too Many Requests"); + + private static Map MAP; + + static { + EnumMap map = new EnumMap<>(RestStatus.class); + map.put(RestStatus.CONTINUE, HttpResponseStatus.CONTINUE); + map.put(RestStatus.SWITCHING_PROTOCOLS, HttpResponseStatus.SWITCHING_PROTOCOLS); + map.put(RestStatus.OK, HttpResponseStatus.OK); + map.put(RestStatus.CREATED, HttpResponseStatus.CREATED); + map.put(RestStatus.ACCEPTED, HttpResponseStatus.ACCEPTED); + map.put(RestStatus.NON_AUTHORITATIVE_INFORMATION, HttpResponseStatus.NON_AUTHORITATIVE_INFORMATION); + map.put(RestStatus.NO_CONTENT, HttpResponseStatus.NO_CONTENT); + map.put(RestStatus.RESET_CONTENT, HttpResponseStatus.RESET_CONTENT); + map.put(RestStatus.PARTIAL_CONTENT, HttpResponseStatus.PARTIAL_CONTENT); + map.put(RestStatus.MULTI_STATUS, HttpResponseStatus.INTERNAL_SERVER_ERROR); // no status for this?? + map.put(RestStatus.MULTIPLE_CHOICES, HttpResponseStatus.MULTIPLE_CHOICES); + map.put(RestStatus.MOVED_PERMANENTLY, HttpResponseStatus.MOVED_PERMANENTLY); + map.put(RestStatus.FOUND, HttpResponseStatus.FOUND); + map.put(RestStatus.SEE_OTHER, HttpResponseStatus.SEE_OTHER); + map.put(RestStatus.NOT_MODIFIED, HttpResponseStatus.NOT_MODIFIED); + map.put(RestStatus.USE_PROXY, HttpResponseStatus.USE_PROXY); + map.put(RestStatus.TEMPORARY_REDIRECT, HttpResponseStatus.TEMPORARY_REDIRECT); + map.put(RestStatus.BAD_REQUEST, HttpResponseStatus.BAD_REQUEST); + map.put(RestStatus.UNAUTHORIZED, HttpResponseStatus.UNAUTHORIZED); + map.put(RestStatus.PAYMENT_REQUIRED, HttpResponseStatus.PAYMENT_REQUIRED); + map.put(RestStatus.FORBIDDEN, HttpResponseStatus.FORBIDDEN); + map.put(RestStatus.NOT_FOUND, HttpResponseStatus.NOT_FOUND); + map.put(RestStatus.METHOD_NOT_ALLOWED, HttpResponseStatus.METHOD_NOT_ALLOWED); + map.put(RestStatus.NOT_ACCEPTABLE, HttpResponseStatus.NOT_ACCEPTABLE); + map.put(RestStatus.PROXY_AUTHENTICATION, HttpResponseStatus.PROXY_AUTHENTICATION_REQUIRED); + map.put(RestStatus.REQUEST_TIMEOUT, HttpResponseStatus.REQUEST_TIMEOUT); + map.put(RestStatus.CONFLICT, HttpResponseStatus.CONFLICT); + map.put(RestStatus.GONE, HttpResponseStatus.GONE); + map.put(RestStatus.LENGTH_REQUIRED, HttpResponseStatus.LENGTH_REQUIRED); + map.put(RestStatus.PRECONDITION_FAILED, HttpResponseStatus.PRECONDITION_FAILED); + map.put(RestStatus.REQUEST_ENTITY_TOO_LARGE, HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE); + map.put(RestStatus.REQUEST_URI_TOO_LONG, HttpResponseStatus.REQUEST_URI_TOO_LONG); + map.put(RestStatus.UNSUPPORTED_MEDIA_TYPE, HttpResponseStatus.UNSUPPORTED_MEDIA_TYPE); + map.put(RestStatus.REQUESTED_RANGE_NOT_SATISFIED, HttpResponseStatus.REQUESTED_RANGE_NOT_SATISFIABLE); + map.put(RestStatus.EXPECTATION_FAILED, HttpResponseStatus.EXPECTATION_FAILED); + map.put(RestStatus.UNPROCESSABLE_ENTITY, HttpResponseStatus.BAD_REQUEST); + map.put(RestStatus.LOCKED, HttpResponseStatus.BAD_REQUEST); + map.put(RestStatus.FAILED_DEPENDENCY, HttpResponseStatus.BAD_REQUEST); + map.put(RestStatus.TOO_MANY_REQUESTS, TOO_MANY_REQUESTS); + map.put(RestStatus.INTERNAL_SERVER_ERROR, HttpResponseStatus.INTERNAL_SERVER_ERROR); + map.put(RestStatus.NOT_IMPLEMENTED, HttpResponseStatus.NOT_IMPLEMENTED); + map.put(RestStatus.BAD_GATEWAY, HttpResponseStatus.BAD_GATEWAY); + map.put(RestStatus.SERVICE_UNAVAILABLE, HttpResponseStatus.SERVICE_UNAVAILABLE); + map.put(RestStatus.GATEWAY_TIMEOUT, HttpResponseStatus.GATEWAY_TIMEOUT); + map.put(RestStatus.HTTP_VERSION_NOT_SUPPORTED, HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED); + MAP = Collections.unmodifiableMap(map); + } + + private static HttpResponseStatus getStatus(RestStatus status) { + return MAP.getOrDefault(status, HttpResponseStatus.INTERNAL_SERVER_ERROR); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java similarity index 71% rename from modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java rename to modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 614c7c1ded281..6191e5dcb0857 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -18,9 +18,12 @@ */ -package org.elasticsearch.transport.nio.http; +package org.elasticsearch.http.nio; import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.codec.http.HttpContentCompressor; @@ -29,17 +32,15 @@ import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; import org.elasticsearch.http.netty4.pipelining.HttpPipeliningHandler; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; +import java.util.function.BiConsumer; + import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION_LEVEL; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; @@ -49,45 +50,34 @@ public class NioHttpNettyAdaptor { - private final NioHttpTransport transport; - private final Netty4CorsConfig config; - private final int maxContentLength; - private final NioHttpRequestHandler requestHandler; + private final BiConsumer exceptionHandler; private final Netty4CorsConfig corsConfig; + private final int maxContentLength; private final boolean compression; private final int compressionLevel; - private final boolean detailedErrorsEnabled; - private final boolean corsEnabled; private boolean pipelining; private final int pipeliningMaxEvents; private final int maxChunkSize; private final int maxHeaderSize; private final int maxInitialLineLength; - protected NioHttpNettyAdaptor(NioHttpTransport transport, NamedXContentRegistry xContentRegistry, ThreadContext threadContext, - Settings settings, Netty4CorsConfig config, int maxContentLength) { - this.transport = transport; - this.config = config; + protected NioHttpNettyAdaptor(Settings settings, BiConsumer exceptionHandler, Netty4CorsConfig config, + int maxContentLength) { + this.exceptionHandler = exceptionHandler; this.maxContentLength = maxContentLength; - this.detailedErrorsEnabled = SETTING_HTTP_DETAILED_ERRORS_ENABLED.get(settings); - - this.requestHandler = new NioHttpRequestHandler(transport, xContentRegistry, detailedErrorsEnabled, threadContext, pipelining); + this.corsConfig = config; this.compression = SETTING_HTTP_COMPRESSION.get(settings); this.compressionLevel = SETTING_HTTP_COMPRESSION_LEVEL.get(settings); this.pipelining = SETTING_PIPELINING.get(settings); this.pipeliningMaxEvents = SETTING_PIPELINING_MAX_EVENTS.get(settings); - this.corsConfig = config; this.maxChunkSize = Math.toIntExact(SETTING_HTTP_MAX_CHUNK_SIZE.get(settings).getBytes()); this.maxHeaderSize = Math.toIntExact(SETTING_HTTP_MAX_HEADER_SIZE.get(settings).getBytes()); this.maxInitialLineLength = Math.toIntExact(SETTING_HTTP_MAX_INITIAL_LINE_LENGTH.get(settings).getBytes()); - this.corsEnabled = SETTING_CORS_ENABLED.get(settings); } - protected void initChannel(NioSocketChannel channel) throws Exception { - pipelining = false; - - EmbeddedChannel ch = new ESNettyChannel(channel); + protected EmbeddedChannel getAdaptor(NioSocketChannel channel) { + EmbeddedChannel ch = new EmbeddedChannel(); final HttpRequestDecoder decoder = new HttpRequestDecoder(maxInitialLineLength, maxHeaderSize, maxChunkSize); ch.pipeline().addLast(decoder); @@ -103,24 +93,29 @@ protected void initChannel(NioSocketChannel channel) throws Exception { ch.pipeline().addLast("encoder_compress", new HttpContentCompressor(compressionLevel)); } if (corsConfig.isCorsSupportEnabled()) { - ch.pipeline().addLast("cors", new Netty4CorsHandler(config)); + ch.pipeline().addLast("cors", new Netty4CorsHandler(corsConfig)); } if (pipelining) { ch.pipeline().addLast("pipelining", new HttpPipeliningHandler(pipeliningMaxEvents)); } - ch.pipeline().addLast("handler", requestHandler); - } - - public static class ESNettyChannel extends EmbeddedChannel { - - private final NioSocketChannel nioSocketChannel; - - private ESNettyChannel(NioSocketChannel nioSocketChannel) { - this.nioSocketChannel = nioSocketChannel; - } - - public NioSocketChannel getNioSocketChannel() { - return nioSocketChannel; - } + ch.pipeline().addLast("read_exception_handler", new ChannelInboundHandlerAdapter() { + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + exceptionHandler.accept(channel, cause); + } + }); + ch.pipeline().addLast("writer", new ChannelOutboundHandlerAdapter() { + @Override + public void flush(ChannelHandlerContext ctx) throws Exception { + super.flush(ctx); + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + super.write(ctx, msg, promise); + } + }); + + return ch; } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java new file mode 100644 index 0000000000000..63aa81f745718 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -0,0 +1,60 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestEncoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +import java.util.Queue; +import java.util.function.BiConsumer; + +import static org.mockito.Mockito.mock; + +public class NioHttpNettyAdaptorTests extends ESTestCase { + + public void testThing() { + NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), + Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); + + HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_0, HttpMethod.GET, "localhost:9090/got/got"); + + EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); + ch.writeOutbound(defaultFullHttpRequest); + Object o = ch.readOutbound(); + int i = 0; + + adaptor.writeInbound(o); + + Object o1 = adaptor.readInbound(); + int j = 0; + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java similarity index 76% rename from modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java rename to modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java index 785c5a6514b38..a11693ee192bc 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java @@ -17,38 +17,41 @@ * under the License. */ -package org.elasticsearch.transport.nio.http; +package org.elasticsearch.http.nio; import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.Channel; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.FullHttpRequest; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.http.netty4.Netty4HttpChannel; import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; -public class NioHttpRequestHandler extends SimpleChannelInboundHandler { +public class NioHttpRequestHandler { private final NioHttpTransport transport; private final NamedXContentRegistry xContentRegistry; private final ThreadContext threadContext; + private final Netty4CorsConfig corsConfig; private final boolean detailedErrorsEnabled; private final boolean httpPipeliningEnabled; + private final boolean resetCookies; public NioHttpRequestHandler(NioHttpTransport transport, NamedXContentRegistry xContentRegistry, boolean detailedErrorsEnabled, - ThreadContext threadContext, boolean httpPipeliningEnabled) { + ThreadContext threadContext, boolean httpPipeliningEnabled, Netty4CorsConfig corsConfig, + boolean resetCookies) { this.transport = transport; this.detailedErrorsEnabled = detailedErrorsEnabled; this.threadContext = threadContext; this.httpPipeliningEnabled = httpPipeliningEnabled; this.xContentRegistry = xContentRegistry; + this.corsConfig = corsConfig; + this.resetCookies = resetCookies; } - @Override - protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Exception { + public void handleMessage(Channel nettyChannel, Object msg) { final FullHttpRequest request; final HttpPipelinedRequest pipelinedRequest; if (this.httpPipeliningEnabled && msg instanceof HttpPipelinedRequest) { @@ -67,8 +70,9 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Except Unpooled.copiedBuffer(request.content()), request.headers(), request.trailingHeaders()); - final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, ctx.channel()); - final Netty4HttpChannel channel = new Netty4HttpChannel(null, httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext); + final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, nettyChannel); + final NioHttpChannel channel = new NioHttpChannel(httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, + resetCookies); if (request.decoderResult().isSuccess()) { transport.dispatchRequest(httpRequest, channel); @@ -76,12 +80,5 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Except assert request.decoderResult().isFailure(); transport.dispatchBadRequest(httpRequest, channel, request.decoderResult().cause()); } - - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - NioHttpNettyAdaptor.ESNettyChannel channel = (NioHttpNettyAdaptor.ESNettyChannel) ctx.channel(); - transport.exceptionCaught(channel.getNioSocketChannel(), cause); } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java similarity index 98% rename from modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java rename to modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java index ba6ed9f6229f7..3f1ac80e74a92 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.transport.nio.http; +package org.elasticsearch.http.nio; import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.IntSet; @@ -45,6 +45,7 @@ import org.elasticsearch.http.HttpStats; import org.elasticsearch.http.netty4.Netty4HttpChannel; import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.rest.RestChannel; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BindTransportException; import org.elasticsearch.transport.nio.AcceptingSelector; @@ -186,14 +187,14 @@ public HttpStats stats() { ? 0 : openChannels.getAcceptedChannels().size() + serverChannelsCount); } - public void dispatchRequest(Netty4HttpRequest httpRequest, Netty4HttpChannel channel) { + public void dispatchRequest(Netty4HttpRequest httpRequest, RestChannel channel) { final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { dispatcher.dispatchRequest(httpRequest, channel, threadContext); } } - public void dispatchBadRequest(Netty4HttpRequest httpRequest, Netty4HttpChannel channel, Throwable cause) { + public void dispatchBadRequest(Netty4HttpRequest httpRequest, RestChannel channel, Throwable cause) { final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { dispatcher.dispatchBadRequest(httpRequest, channel, threadContext, cause); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ReadContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ReadContext.java index 9d2919b19286a..40217bbb55888 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ReadContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ReadContext.java @@ -23,6 +23,8 @@ public interface ReadContext { + int DEFAULT_READ_LENGTH = 1 << 14; + int read() throws IOException; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java index ee56f6f6f9763..2b62cae80172e 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpReadContext.java @@ -26,13 +26,10 @@ import org.elasticsearch.transport.nio.TcpReadHandler; import java.io.IOException; -import java.util.Iterator; import java.util.LinkedList; public class TcpReadContext implements ReadContext { - private static final int DEFAULT_READ_LENGTH = 1 << 14; - private final TcpReadHandler handler; private final NioSocketChannel channel; private final TcpFrameDecoder frameDecoder; From 8603ca7a65272c7faa9fd2ed5cdaf51c767776d5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sat, 29 Jul 2017 23:36:22 -0500 Subject: [PATCH 03/35] WIP --- .../http/nio/NioHttpNettyAdaptorTests.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 63aa81f745718..4149934fca85d 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -19,28 +19,27 @@ package org.elasticsearch.http.nio; -import io.netty.buffer.Unpooled; +import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.DefaultFullHttpRequest; -import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequestEncoder; -import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; +import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import java.util.Queue; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; public class NioHttpNettyAdaptorTests extends ESTestCase { - public void testThing() { + public void testDecodeHttpRequest() { NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); @@ -49,12 +48,17 @@ public void testThing() { EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); ch.writeOutbound(defaultFullHttpRequest); - Object o = ch.readOutbound(); - int i = 0; + ByteBuf buf = ch.readOutbound(); - adaptor.writeInbound(o); + adaptor.writeInbound(buf.slice(0, 5).retainedDuplicate()); - Object o1 = adaptor.readInbound(); - int j = 0; + assertNull(adaptor.readInbound()); + + adaptor.writeInbound(buf.slice(5, buf.writerIndex() - 5).retainedDuplicate()); + HttpPipelinedRequest decodedRequest = adaptor.readInbound(); + + FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); + assertEquals(defaultFullHttpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); + assertEquals(defaultFullHttpRequest.method(), fullHttpRequest.method()); } } From 72ea6a6b26538cf999de92cf945c20b5277a52d5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sun, 30 Jul 2017 17:32:02 -0500 Subject: [PATCH 04/35] WIP --- .../http/nio/HttpReadContext.java | 20 ++++++++-- .../http/nio/NioHttpNettyAdaptor.java | 40 ++++++++++++++----- .../http/nio/NioHttpNettyAdaptorTests.java | 24 ++++++++++- .../http/nio/NioHttpTransport.java | 2 + 4 files changed, 71 insertions(+), 15 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index 307d747db5470..cd71364e1cb47 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -23,7 +23,12 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.http.netty4.Netty4HttpChannel; +import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.ReadContext; @@ -34,12 +39,14 @@ public class HttpReadContext implements ReadContext { private final NioSocketChannel channel; - private final EmbeddedChannel nettyReadHandler; + private final EmbeddedChannel nettyPipelineAdaptor; private final LinkedList references = new LinkedList<>(); + private final NioHttpRequestHandler requestHandler; - public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor) { + public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor, NioHttpRequestHandler requestHandler) { this.channel = channel; - this.nettyReadHandler = adaptor.getAdaptor(channel); + this.requestHandler = requestHandler; + this.nettyPipelineAdaptor = adaptor.getAdaptor(channel); } @Override @@ -59,7 +66,12 @@ public int read() throws IOException { ByteBuf inboundBytes = toByteBuf(size); - nettyReadHandler.writeInbound(inboundBytes); + nettyPipelineAdaptor.writeInbound(inboundBytes); + + Object msg; + while ((msg = nettyPipelineAdaptor.readInbound()) != null) { + requestHandler.handleMessage(nettyPipelineAdaptor, msg); + } return bytesRead; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 6191e5dcb0857..a15d486f13d77 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -17,13 +17,18 @@ * under the License. */ - package org.elasticsearch.http.nio; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.channel.ChannelConfig; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelConfig; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.codec.http.HttpContentCompressor; @@ -31,10 +36,14 @@ import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; import org.elasticsearch.http.netty4.pipelining.HttpPipeliningHandler; +import org.elasticsearch.transport.netty4.ByteBufBytesReferenceTests; +import org.elasticsearch.transport.netty4.Netty4Utils; +import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import java.util.function.BiConsumer; @@ -46,10 +55,10 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; +import static org.elasticsearch.http.netty4.Netty4HttpServerTransport.SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS; public class NioHttpNettyAdaptor { - private final BiConsumer exceptionHandler; private final Netty4CorsConfig corsConfig; private final int maxContentLength; @@ -60,6 +69,7 @@ public class NioHttpNettyAdaptor { private final int maxChunkSize; private final int maxHeaderSize; private final int maxInitialLineLength; + private final int maxCompositeBufferComponents; protected NioHttpNettyAdaptor(Settings settings, BiConsumer exceptionHandler, Netty4CorsConfig config, int maxContentLength) { @@ -74,10 +84,13 @@ protected NioHttpNettyAdaptor(Settings settings, BiConsumer() { + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + } + }); + ctx.write(msg, promise); } }); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 4149934fca85d..fe88f907dbf27 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -22,10 +22,13 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequestEncoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; @@ -44,7 +47,7 @@ public void testDecodeHttpRequest() { Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); - HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_0, HttpMethod.GET, "localhost:9090/got/got"); + HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); ch.writeOutbound(defaultFullHttpRequest); @@ -61,4 +64,23 @@ public void testDecodeHttpRequest() { assertEquals(defaultFullHttpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); assertEquals(defaultFullHttpRequest.method(), fullHttpRequest.method()); } + + public void testEncodeHttpResponse() { + NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), + Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); + + + HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); + ch.writeOutbound(defaultFullHttpRequest); + ByteBuf buf = ch.readOutbound(); + adaptor.writeInbound(buf); + + + HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + + adaptor.writeOutbound(defaultFullHttpResponse); + Object encodedResponse = adaptor.readOutbound(); + } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java index 3f1ac80e74a92..689b0ab092604 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java @@ -98,6 +98,8 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); + + private final PortsRange port; private final NetworkService networkService; private final BigArrays bigArrays; From 3d3f4d0eccd6c87b8db1958effcb21a7d71b5df8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sun, 30 Jul 2017 18:03:18 -0500 Subject: [PATCH 05/35] WIP --- .../http/nio/NioHttpNettyAdaptor.java | 36 ++++++++++--------- .../http/nio/NioHttpNettyAdaptorTests.java | 25 +++++++++---- 2 files changed, 38 insertions(+), 23 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index a15d486f13d77..7c1410966b7f3 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -88,6 +88,7 @@ protected NioHttpNettyAdaptor(Settings settings, BiConsumer() { + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + } + }); + ctx.write(msg, promise); + } + }); ch.pipeline().addLast(new HttpResponseEncoder()); decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR); final HttpObjectAggregator aggregator = new HttpObjectAggregator(maxContentLength); @@ -117,24 +136,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E exceptionHandler.accept(channel, cause); } }); - ch.pipeline().addLast("writer", new ChannelOutboundHandlerAdapter() { - - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - channel.getWriteContext().sendMessage(Netty4Utils.toBytesReference((ByteBuf) msg), new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - promise.setSuccess(); - } - @Override - public void onFailure(Exception e) { - promise.setFailure(e); - } - }); - ctx.write(msg, promise); - } - }); return ch; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index fe88f907dbf27..3ea35bc343569 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -28,17 +28,21 @@ import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequestEncoder; import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseDecoder; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.channel.WriteContext; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class NioHttpNettyAdaptorTests extends ESTestCase { @@ -68,19 +72,28 @@ public void testDecodeHttpRequest() { public void testEncodeHttpResponse() { NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); - EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); - + NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); + when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); + EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(nioSocketChannel); + // Must send a request through pipeline inorder to handle response HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); - EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); - ch.writeOutbound(defaultFullHttpRequest); - ByteBuf buf = ch.readOutbound(); + EmbeddedChannel encodingChannel = new EmbeddedChannel(new HttpRequestEncoder()); + encodingChannel.writeOutbound(defaultFullHttpRequest); + ByteBuf buf = encodingChannel.readOutbound(); adaptor.writeInbound(buf); HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); adaptor.writeOutbound(defaultFullHttpResponse); - Object encodedResponse = adaptor.readOutbound(); + ByteBuf encodedResponse = adaptor.readOutbound(); + + EmbeddedChannel decodingChannel = new EmbeddedChannel(new HttpResponseDecoder()); + decodingChannel.writeInbound(encodedResponse); + HttpResponse response = decodingChannel.readInbound(); + + assertEquals(HttpResponseStatus.OK, response.status()); + assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); } } From ae7743ba141d7479e3ef0cfb35a762695ca9d860 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Aug 2017 11:17:15 -0500 Subject: [PATCH 06/35] WIP --- .../http/nio/NioHttpNettyAdaptor.java | 39 ++++++++++--------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 7c1410966b7f3..5f6df7b41890d 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -36,6 +36,8 @@ import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.FutureListener; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; @@ -96,24 +98,24 @@ protected EmbeddedChannel getAdaptor(NioSocketChannel channel) { final HttpRequestDecoder decoder = new HttpRequestDecoder(maxInitialLineLength, maxHeaderSize, maxChunkSize); ch.pipeline().addLast(decoder); ch.pipeline().addLast(new HttpContentDecompressor()); - ch.pipeline().addLast("writer", new ChannelOutboundHandlerAdapter() { - - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - channel.getWriteContext().sendMessage(Netty4Utils.toBytesReference((ByteBuf) msg), new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - promise.setSuccess(); - } - - @Override - public void onFailure(Exception e) { - promise.setFailure(e); - } - }); - ctx.write(msg, promise); - } - }); +// ch.pipeline().addLast("writer", new ChannelOutboundHandlerAdapter() { +// +// @Override +// public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { +// channel.getWriteContext().sendMessage(Netty4Utils.toBytesReference((ByteBuf) msg), new ActionListener() { +// @Override +// public void onResponse(NioChannel nioChannel) { +// promise.setSuccess(); +// } +// +// @Override +// public void onFailure(Exception e) { +// promise.setFailure(e); +// } +// }); +// ctx.write(msg, promise); +// } +// }); ch.pipeline().addLast(new HttpResponseEncoder()); decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR); final HttpObjectAggregator aggregator = new HttpObjectAggregator(maxContentLength); @@ -137,6 +139,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } }); + ch.closeFuture().addListener((FutureListener) future -> channel.closeAsync()); return ch; } From 85be496ea9deb822f2d3850d4ea645a5e0d24384 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 8 Aug 2017 22:13:37 -0500 Subject: [PATCH 07/35] WIP --- .../http/nio/HttpReadContext.java | 7 +- .../http/nio/HttpWriteContext.java | 3 + ...r.java => NioHttpNettyAdaptorFactory.java} | 69 +++++++++------ .../http/nio/NioHttpNettyAdaptorTests.java | 83 +++++++++++++++++-- 4 files changed, 123 insertions(+), 39 deletions(-) rename modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/{NioHttpNettyAdaptor.java => NioHttpNettyAdaptorFactory.java} (72%) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index cd71364e1cb47..dcd08c6b4bc0f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -23,12 +23,7 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.codec.http.DefaultFullHttpRequest; -import io.netty.handler.codec.http.FullHttpRequest; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.http.netty4.Netty4HttpChannel; -import org.elasticsearch.http.netty4.Netty4HttpRequest; -import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.ReadContext; @@ -43,7 +38,7 @@ public class HttpReadContext implements ReadContext { private final LinkedList references = new LinkedList<>(); private final NioHttpRequestHandler requestHandler; - public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor, NioHttpRequestHandler requestHandler) { + public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptorFactory adaptor, NioHttpRequestHandler requestHandler) { this.channel = channel; this.requestHandler = requestHandler; this.nettyPipelineAdaptor = adaptor.getAdaptor(channel); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java index bbb3407aabcba..29a2927187cf0 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -28,6 +28,9 @@ import java.io.IOException; public class HttpWriteContext implements WriteContext { + + + @Override public void sendMessage(BytesReference reference, ActionListener listener) { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java similarity index 72% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java rename to modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java index 5f6df7b41890d..5fddebe2a40e0 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java @@ -35,10 +35,14 @@ import io.netty.handler.codec.http.HttpContentDecompressor; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; @@ -48,6 +52,7 @@ import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import java.util.LinkedList; import java.util.function.BiConsumer; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION; @@ -59,7 +64,7 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; import static org.elasticsearch.http.netty4.Netty4HttpServerTransport.SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS; -public class NioHttpNettyAdaptor { +public class NioHttpNettyAdaptorFactory { private final BiConsumer exceptionHandler; private final Netty4CorsConfig corsConfig; @@ -73,8 +78,8 @@ public class NioHttpNettyAdaptor { private final int maxInitialLineLength; private final int maxCompositeBufferComponents; - protected NioHttpNettyAdaptor(Settings settings, BiConsumer exceptionHandler, Netty4CorsConfig config, - int maxContentLength) { + protected NioHttpNettyAdaptorFactory(Settings settings, BiConsumer exceptionHandler, + Netty4CorsConfig config, int maxContentLength) { this.exceptionHandler = exceptionHandler; this.maxContentLength = maxContentLength; this.corsConfig = config; @@ -90,34 +95,44 @@ protected NioHttpNettyAdaptor(Settings settings, BiConsumer() { + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + // We should only be using unpooled buffers. So releasing only removes them to ensure they can + // be GCed. + ch.releaseOutbound(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + // We should only be using unpooled buffers. So releasing only removes them to ensure they can + // be GCed. + ch.releaseOutbound(); + } + }); + + // This is a little tricky. The embedded channel will complete the promise once it writes the message + // to its outbound buffer. We do not want to complete the promise until the message is sent. So we + // intercept the promise and pass a different promise back to the rest of the pipeline. + ctx.write(msg, ch.newPromise()); + } + }); ch.pipeline().addLast(new HttpContentDecompressor()); -// ch.pipeline().addLast("writer", new ChannelOutboundHandlerAdapter() { -// -// @Override -// public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { -// channel.getWriteContext().sendMessage(Netty4Utils.toBytesReference((ByteBuf) msg), new ActionListener() { -// @Override -// public void onResponse(NioChannel nioChannel) { -// promise.setSuccess(); -// } -// -// @Override -// public void onFailure(Exception e) { -// promise.setFailure(e); -// } -// }); -// ctx.write(msg, promise); -// } -// }); ch.pipeline().addLast(new HttpResponseEncoder()); - decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR); final HttpObjectAggregator aggregator = new HttpObjectAggregator(maxContentLength); if (maxCompositeBufferComponents != -1) { aggregator.setMaxCumulationBufferComponents(maxCompositeBufferComponents); @@ -138,8 +153,14 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E exceptionHandler.accept(channel, cause); } }); + ch.pipeline().addLast("close_adaptor", new ChannelOutboundHandlerAdapter() { + + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + channel.closeAsync(); + } - ch.closeFuture().addListener((FutureListener) future -> channel.closeAsync()); + }); return ch; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 3ea35bc343569..4764a777b910d 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.DecoderResult; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.FullHttpRequest; @@ -30,26 +31,49 @@ import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseDecoder; import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; +import org.junit.Before; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class NioHttpNettyAdaptorTests extends ESTestCase { + private BiConsumer exceptionHandler; + private NioHttpNettyAdaptorFactory adaptor; + private NioSocketChannel nioSocketChannel; + + @Before + @SuppressWarnings("unchecked") + public void setMocks() { + exceptionHandler = mock(BiConsumer.class); + adaptor = new NioHttpNettyAdaptorFactory(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + nioSocketChannel = mock(NioSocketChannel.class); + + when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); + } + + public void testCloseEmbeddedChannelSchedulesRealChannelForClose() { + NioSocketChannel channel = mock(NioSocketChannel.class); + EmbeddedChannel channelAdaptor = adaptor.getAdaptor(channel); + + channelAdaptor.close(); + verify(channel).closeAsync(); + } + public void testDecodeHttpRequest() { - NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), - Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); - EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(mock(NioSocketChannel.class)); + EmbeddedChannel channelAdaptor = adaptor.getAdaptor(mock(NioSocketChannel.class)); HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); @@ -57,20 +81,61 @@ public void testDecodeHttpRequest() { ch.writeOutbound(defaultFullHttpRequest); ByteBuf buf = ch.readOutbound(); - adaptor.writeInbound(buf.slice(0, 5).retainedDuplicate()); + channelAdaptor.writeInbound(buf.slice(0, 5).retainedDuplicate()); - assertNull(adaptor.readInbound()); + assertNull(channelAdaptor.readInbound()); - adaptor.writeInbound(buf.slice(5, buf.writerIndex() - 5).retainedDuplicate()); - HttpPipelinedRequest decodedRequest = adaptor.readInbound(); + channelAdaptor.writeInbound(buf.slice(5, buf.writerIndex() - 5).retainedDuplicate()); + HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); assertEquals(defaultFullHttpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); assertEquals(defaultFullHttpRequest.method(), fullHttpRequest.method()); } + public void testDecodeHttpRequestError() { + EmbeddedChannel channelAdaptor = adaptor.getAdaptor(mock(NioSocketChannel.class)); + + HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + + EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); + ch.writeOutbound(defaultFullHttpRequest); + ByteBuf buf = ch.readOutbound(); + buf.setByte(0, ' '); + buf.setByte(1, ' '); + buf.setByte(2, ' '); + + channelAdaptor.writeInbound(buf); + + HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); + + FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); + DecoderResult decoderResult = fullHttpRequest.decoderResult(); + assertTrue(decoderResult.isFailure()); + assertTrue(decoderResult.cause() instanceof IllegalArgumentException); + } + + public void testDecodeHttpRequestContentLengthToLong() { + // TODO: need to capture sent message listeners + EmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "localhost:9090/got/got", false); + HttpUtil.setContentLength(httpRequest, 1025); + + EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); + ch.writeOutbound(httpRequest); + ByteBuf buf = ch.readOutbound(); + + channelAdaptor.writeInbound(buf); + + HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); + + assertNull(decodedRequest); + verify(nioSocketChannel, times(0)).closeAsync(); + } + public void testEncodeHttpResponse() { - NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, mock(BiConsumer.class), + NioHttpNettyAdaptorFactory nioHttpNettyAdaptor = new NioHttpNettyAdaptorFactory(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); From e8f8a882cbaa9dc271316472030547047e661ff2 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Aug 2017 10:02:46 -0500 Subject: [PATCH 08/35] WIP --- .../http/nio/HttpReadContext.java | 6 +- .../http/nio/HttpWriteContext.java | 58 ------------------- ...rFactory.java => NioHttpNettyAdaptor.java} | 7 ++- .../http/nio/NioHttpNettyAdaptorTests.java | 21 ++++--- .../http/nio/NioHttpRequestHandler.java | 9 +-- .../http/nio/NioHttpTransport.java | 31 +++++++--- .../transport/nio/NioTransport.java | 12 ++-- .../transport/nio/channel/ChannelFactory.java | 33 ++++++----- .../nio/channel/ChannelFactoryTests.java | 2 +- 9 files changed, 74 insertions(+), 105 deletions(-) delete mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java rename modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/{NioHttpNettyAdaptorFactory.java => NioHttpNettyAdaptor.java} (96%) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index dcd08c6b4bc0f..40e98ac66b60c 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -38,7 +38,7 @@ public class HttpReadContext implements ReadContext { private final LinkedList references = new LinkedList<>(); private final NioHttpRequestHandler requestHandler; - public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptorFactory adaptor, NioHttpRequestHandler requestHandler) { + public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor, NioHttpRequestHandler requestHandler) { this.channel = channel; this.requestHandler = requestHandler; this.nettyPipelineAdaptor = adaptor.getAdaptor(channel); @@ -65,7 +65,7 @@ public int read() throws IOException { Object msg; while ((msg = nettyPipelineAdaptor.readInbound()) != null) { - requestHandler.handleMessage(nettyPipelineAdaptor, msg); + requestHandler.handleMessage(channel, nettyPipelineAdaptor, msg); } return bytesRead; @@ -77,7 +77,7 @@ private ByteBuf toByteBuf(int size) { } else { CompositeByteBuf byteBuf = Unpooled.compositeBuffer(size); for (NetworkBytesReference reference : references) { - byteBuf.addComponent(Unpooled.wrappedBuffer(reference.getReadByteBuffer())); + byteBuf.addComponent(true, Unpooled.wrappedBuffer(reference.getReadByteBuffer())); } return byteBuf; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java deleted file mode 100644 index 29a2927187cf0..0000000000000 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.http.nio; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.transport.nio.WriteOperation; -import org.elasticsearch.transport.nio.channel.NioChannel; -import org.elasticsearch.transport.nio.channel.WriteContext; - -import java.io.IOException; - -public class HttpWriteContext implements WriteContext { - - - - @Override - public void sendMessage(BytesReference reference, ActionListener listener) { - - } - - @Override - public void queueWriteOperations(WriteOperation writeOperation) { - - } - - @Override - public void flushChannel() throws IOException { - - } - - @Override - public boolean hasQueuedWriteOps() { - return false; - } - - @Override - public void clearQueuedWriteOps(Exception e) { - - } -} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java similarity index 96% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java rename to modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 5fddebe2a40e0..21dc107933dab 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorFactory.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -24,6 +24,7 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelOutboundHandlerAdapter; @@ -64,7 +65,7 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; import static org.elasticsearch.http.netty4.Netty4HttpServerTransport.SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS; -public class NioHttpNettyAdaptorFactory { +public class NioHttpNettyAdaptor { private final BiConsumer exceptionHandler; private final Netty4CorsConfig corsConfig; @@ -78,8 +79,8 @@ public class NioHttpNettyAdaptorFactory { private final int maxInitialLineLength; private final int maxCompositeBufferComponents; - protected NioHttpNettyAdaptorFactory(Settings settings, BiConsumer exceptionHandler, - Netty4CorsConfig config, int maxContentLength) { + protected NioHttpNettyAdaptor(Settings settings, BiConsumer exceptionHandler, + Netty4CorsConfig config, int maxContentLength) { this.exceptionHandler = exceptionHandler; this.maxContentLength = maxContentLength; this.corsConfig = config; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 4764a777b910d..4eac175381e55 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.http.nio; import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelFuture; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.DecoderResult; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -37,9 +38,11 @@ import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; import org.junit.Before; +import org.mockito.ArgumentCaptor; import java.util.function.BiConsumer; @@ -51,17 +54,21 @@ public class NioHttpNettyAdaptorTests extends ESTestCase { private BiConsumer exceptionHandler; - private NioHttpNettyAdaptorFactory adaptor; + private NioHttpNettyAdaptor adaptor; private NioSocketChannel nioSocketChannel; + private WriteContext writeContext; + private ArgumentCaptor writeOperation; @Before @SuppressWarnings("unchecked") public void setMocks() { exceptionHandler = mock(BiConsumer.class); - adaptor = new NioHttpNettyAdaptorFactory(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + adaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); nioSocketChannel = mock(NioSocketChannel.class); + writeContext = mock(WriteContext.class); + writeOperation = ArgumentCaptor.forClass(WriteOperation.class); - when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); + when(nioSocketChannel.getWriteContext()).thenReturn(writeContext); } public void testCloseEmbeddedChannelSchedulesRealChannelForClose() { @@ -115,8 +122,7 @@ public void testDecodeHttpRequestError() { assertTrue(decoderResult.cause() instanceof IllegalArgumentException); } - public void testDecodeHttpRequestContentLengthToLong() { - // TODO: need to capture sent message listeners + public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { EmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "localhost:9090/got/got", false); @@ -131,11 +137,12 @@ public void testDecodeHttpRequestContentLengthToLong() { HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); assertNull(decodedRequest); - verify(nioSocketChannel, times(0)).closeAsync(); + + ByteBuf buffer = channelAdaptor.readOutbound(); } public void testEncodeHttpResponse() { - NioHttpNettyAdaptorFactory nioHttpNettyAdaptor = new NioHttpNettyAdaptorFactory(Settings.EMPTY, exceptionHandler, + NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java index a11693ee192bc..07bcea7062744 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java @@ -28,6 +28,7 @@ import org.elasticsearch.http.netty4.Netty4HttpRequest; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; public class NioHttpRequestHandler { @@ -51,7 +52,7 @@ public NioHttpRequestHandler(NioHttpTransport transport, NamedXContentRegistry x this.resetCookies = resetCookies; } - public void handleMessage(Channel nettyChannel, Object msg) { + public void handleMessage(NioSocketChannel channel, Channel nettyChannel, Object msg) { final FullHttpRequest request; final HttpPipelinedRequest pipelinedRequest; if (this.httpPipeliningEnabled && msg instanceof HttpPipelinedRequest) { @@ -71,14 +72,14 @@ public void handleMessage(Channel nettyChannel, Object msg) { request.headers(), request.trailingHeaders()); final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, nettyChannel); - final NioHttpChannel channel = new NioHttpChannel(httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, + final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, resetCookies); if (request.decoderResult().isSuccess()) { - transport.dispatchRequest(httpRequest, channel); + transport.dispatchRequest(httpRequest, httpChannel); } else { assert request.decoderResult().isFailure(); - transport.dispatchBadRequest(httpRequest, channel, request.decoderResult().cause()); + transport.dispatchBadRequest(httpRequest, httpChannel, request.decoderResult().cause()); } } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java index 689b0ab092604..a4162e32b7279 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java @@ -43,7 +43,6 @@ import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpStats; -import org.elasticsearch.http.netty4.Netty4HttpChannel; import org.elasticsearch.http.netty4.Netty4HttpRequest; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.threadpool.ThreadPool; @@ -58,6 +57,7 @@ import org.elasticsearch.transport.nio.channel.ChannelFactory; import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.channel.TcpWriteContext; import java.io.IOException; import java.net.InetAddress; @@ -66,6 +66,7 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.common.settings.Setting.intSetting; @@ -83,10 +84,8 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http new Setting<>("transport.nio.http.worker_count", (s) -> Integer.toString(EsExecutors.numberOfProcessors(s) * 2), (s) -> Setting.parseInt(s, 1, "transport.nio.worker_count"), Setting.Property.NodeScope); - public static final Setting NIO_HTTP_ACCEPTOR_COUNT = intSetting("transport.nio.http.acceptor_count", 1, 1, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_NO_DELAY = boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = @@ -98,8 +97,6 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); - - private final PortsRange port; private final NetworkService networkService; private final BigArrays bigArrays; @@ -108,11 +105,17 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http private final Dispatcher dispatcher; private final String[] publishHosts; private final String bindHosts[]; - private OpenChannels openChannels; + private final Consumer contextSetter; private final ByteSizeValue maxContentLength; + private OpenChannels openChannels; private ArrayList socketSelectors; private ArrayList acceptors; private BoundTransportAddress boundAddress; + private final boolean tcpNoDelay; + private final boolean tcpKeepAlive; + private final boolean tcpReuseAddress; + private final int tcpSendBufferSize; + private final int tcpReceiveBufferSize; private int acceptorNumber; public NioHttpTransport(Settings settings, NetworkService networkService, BigArrays bigArrays, ThreadPool threadPool, @@ -132,6 +135,13 @@ public NioHttpTransport(Settings settings, NetworkService networkService, BigArr this.publishHosts = (httpPublishHost.isEmpty() ? NetworkService.GLOBAL_NETWORK_PUBLISHHOST_SETTING.get(settings) : httpPublishHost) .toArray(Strings.EMPTY_ARRAY); maxContentLength = SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings); + + contextSetter = (c) -> c.setContexts(new HttpReadContext(c, null, null), new TcpWriteContext(c)); + tcpNoDelay = SETTING_HTTP_TCP_NO_DELAY.get(settings); + tcpKeepAlive = SETTING_HTTP_TCP_KEEP_ALIVE.get(settings); + tcpReuseAddress = SETTING_HTTP_TCP_REUSE_ADDRESS.get(settings); + tcpSendBufferSize = Math.toIntExact(SETTING_HTTP_TCP_SEND_BUFFER_SIZE.get(settings).getBytes()); + tcpReceiveBufferSize = Math.toIntExact(SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE.get(settings).getBytes()); } @Override @@ -141,10 +151,10 @@ protected void doStart() { this.openChannels = new OpenChannels(logger); socketSelectors = NioSelectors.socketSelectors(settings, () -> new SocketEventHandler(logger, this::exceptionCaught), - NioTransport.NIO_WORKER_COUNT.get(settings), TRANSPORT_WORKER_THREAD_NAME_PREFIX); + NIO_HTTP_WORKER_COUNT.get(settings), TRANSPORT_WORKER_THREAD_NAME_PREFIX); acceptors = NioSelectors.acceptingSelectors(logger, settings, openChannels, socketSelectors, - NioTransport.NIO_ACCEPTOR_COUNT.get(settings), TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); + NIO_HTTP_ACCEPTOR_COUNT.get(settings), TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); this.boundAddress = createBoundHttpAddress(); if (logger.isInfoEnabled()) { @@ -298,7 +308,10 @@ private TransportAddress bindAddress(final InetAddress hostAddress) { synchronized (this) { AcceptingSelector selector = acceptors.get(++acceptorNumber % NioTransport.NIO_ACCEPTOR_COUNT.get(settings)); InetSocketAddress address = new InetSocketAddress(hostAddress, portNumber); - ChannelFactory channelFactory = new ChannelFactory(null, null); + + ChannelFactory.RawChannelFactory rawChannelFactory = new ChannelFactory.RawChannelFactory(tcpNoDelay, tcpKeepAlive, + tcpReuseAddress, tcpSendBufferSize, tcpReceiveBufferSize); + ChannelFactory channelFactory = new ChannelFactory(rawChannelFactory, contextSetter); NioServerSocketChannel serverChannel = channelFactory.openNioServerSocketChannel("http-server", address, selector); selector.scheduleForRegistration(serverChannel); boundSocket.set(serverChannel.getLocalAddress()); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index 4191bc85ce0af..3c4fb8b5b5ad8 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -41,14 +40,14 @@ import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.channel.TcpReadContext; +import org.elasticsearch.transport.nio.channel.TcpWriteContext; import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ThreadFactory; import java.util.function.Consumer; import java.util.function.Supplier; @@ -69,7 +68,7 @@ public class NioTransport extends TcpTransport { public static final Setting NIO_ACCEPTOR_COUNT = intSetting("transport.nio.acceptor_count", 1, 1, Setting.Property.NodeScope); - private final TcpReadHandler tcpReadHandler = new TcpReadHandler(this); + private final Consumer contextSetter; private final ConcurrentMap profileToChannelFactory = newConcurrentMap(); private final OpenChannels openChannels = new OpenChannels(logger); private ArrayList acceptors; @@ -80,6 +79,7 @@ public class NioTransport extends TcpTransport { public NioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays, NamedWriteableRegistry namedWriteableRegistry, CircuitBreakerService circuitBreakerService) { super("nio", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService); + contextSetter = (c) -> c.setContexts(new TcpReadContext(c, new TcpReadHandler(this)), new TcpWriteContext(c)); } @Override @@ -176,7 +176,7 @@ protected void doStart() { // loop through all profiles and start them up, special handling for default one for (ProfileSettings profileSettings : profileSettings) { - profileToChannelFactory.putIfAbsent(profileSettings.profileName, new ChannelFactory(profileSettings, tcpReadHandler)); + profileToChannelFactory.putIfAbsent(profileSettings.profileName, new ChannelFactory(profileSettings, contextSetter)); bindServer(profileSettings); } } else { @@ -213,7 +213,7 @@ final void exceptionCaught(NioSocketChannel channel, Throwable cause) { private NioClient createClient() { Supplier selectorSupplier = new RoundRobinSelectorSupplier(socketSelectors); - ChannelFactory channelFactory = new ChannelFactory(new ProfileSettings(settings, "default"), tcpReadHandler); + ChannelFactory channelFactory = new ChannelFactory(new ProfileSettings(settings, "default"), contextSetter); return new NioClient(logger, openChannels, selectorSupplier, defaultConnectionProfile.getConnectTimeout(), channelFactory); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java index c25936ce7fc01..83659c647ac01 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java @@ -38,15 +38,19 @@ public class ChannelFactory { - private final TcpReadHandler handler; + private final Consumer contextSetter; private final RawChannelFactory rawChannelFactory; - public ChannelFactory(TcpTransport.ProfileSettings profileSettings, TcpReadHandler handler) { - this(new RawChannelFactory(profileSettings), handler); + public ChannelFactory(TcpTransport.ProfileSettings profileSettings, Consumer contextSetter) { + this(new RawChannelFactory(profileSettings.tcpNoDelay, + profileSettings.tcpKeepAlive, + profileSettings.reuseAddress, + Math.toIntExact(profileSettings.sendBufferSize.getBytes()), + Math.toIntExact(profileSettings.receiveBufferSize.getBytes())), contextSetter); } - ChannelFactory(RawChannelFactory rawChannelFactory, TcpReadHandler handler) { - this.handler = handler; + public ChannelFactory(RawChannelFactory rawChannelFactory, Consumer contextSetter) { + this.contextSetter = contextSetter; this.rawChannelFactory = rawChannelFactory; } @@ -54,7 +58,7 @@ public NioSocketChannel openNioChannel(InetSocketAddress remoteAddress, SocketSe Consumer closeListener) throws IOException { SocketChannel rawChannel = rawChannelFactory.openNioChannel(remoteAddress); NioSocketChannel channel = new NioSocketChannel(NioChannel.CLIENT, rawChannel, selector); - channel.setContexts(new TcpReadContext(channel, handler), new TcpWriteContext(channel)); + contextSetter.accept(channel); channel.getCloseFuture().setListener(closeListener); scheduleChannel(channel, selector); return channel; @@ -64,7 +68,7 @@ public NioSocketChannel acceptNioChannel(NioServerSocketChannel serverChannel, S Consumer closeListener) throws IOException { SocketChannel rawChannel = rawChannelFactory.acceptNioChannel(serverChannel); NioSocketChannel channel = new NioSocketChannel(serverChannel.getProfile(), rawChannel, selector); - channel.setContexts(new TcpReadContext(channel, handler), new TcpWriteContext(channel)); + contextSetter.accept(channel); channel.getCloseFuture().setListener(closeListener); scheduleChannel(channel, selector); return channel; @@ -96,7 +100,7 @@ private void scheduleServerChannel(NioServerSocketChannel channel, AcceptingSele } } - static class RawChannelFactory { + public static class RawChannelFactory { private final boolean tcpNoDelay; private final boolean tcpKeepAlive; @@ -104,12 +108,13 @@ static class RawChannelFactory { private final int tcpSendBufferSize; private final int tcpReceiveBufferSize; - RawChannelFactory(TcpTransport.ProfileSettings profileSettings) { - tcpNoDelay = profileSettings.tcpNoDelay; - tcpKeepAlive = profileSettings.tcpKeepAlive; - tcpReusedAddress = profileSettings.reuseAddress; - tcpSendBufferSize = Math.toIntExact(profileSettings.sendBufferSize.getBytes()); - tcpReceiveBufferSize = Math.toIntExact(profileSettings.receiveBufferSize.getBytes()); + public RawChannelFactory(boolean tcpNoDelay, boolean tcpKeepAlive, boolean tcpReusedAddress, int tcpSendBufferSize, + int tcpReceiveBufferSize) { + this.tcpNoDelay = tcpNoDelay; + this.tcpKeepAlive = tcpKeepAlive; + this.tcpReusedAddress = tcpReusedAddress; + this.tcpSendBufferSize = tcpSendBufferSize; + this.tcpReceiveBufferSize = tcpReceiveBufferSize; } SocketChannel openNioChannel(InetSocketAddress remoteAddress) throws IOException { diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/ChannelFactoryTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/ChannelFactoryTests.java index 8851c37f2012e..6b43dc9ed8087 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/ChannelFactoryTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/ChannelFactoryTests.java @@ -55,7 +55,7 @@ public class ChannelFactoryTests extends ESTestCase { @SuppressWarnings("unchecked") public void setupFactory() throws IOException { rawChannelFactory = mock(ChannelFactory.RawChannelFactory.class); - channelFactory = new ChannelFactory(rawChannelFactory, mock(TcpReadHandler.class)); + channelFactory = new ChannelFactory(rawChannelFactory, mock(Consumer.class)); listener = mock(Consumer.class); socketSelector = mock(SocketSelector.class); acceptingSelector = mock(AcceptingSelector.class); From 959773c9dc6158b5968110eba40c654ef9d1da90 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Aug 2017 23:04:57 -0500 Subject: [PATCH 09/35] WIP --- .../http/nio/ESEmbeddedChannel.java | 60 ++++++++++++++ .../http/nio/HttpReadContext.java | 1 + .../http/nio/HttpWriteContext.java | 55 +++++++++++++ .../http/nio/HttpWriteOperation.java | 79 +++++++++++++++++++ .../http/nio/NioHttpChannel.java | 21 ++++- .../http/nio/NioHttpNettyAdaptor.java | 32 +------- .../http/nio/NioHttpNettyAdaptorTests.java | 10 +-- .../http/nio/NioHttpRequestHandler.java | 2 +- .../transport/nio/ByteWriteOperation.java | 63 +++++++++++++++ .../transport/nio/SocketSelector.java | 10 +-- .../transport/nio/WriteOperation.java | 58 ++------------ .../nio/channel/TcpWriteContext.java | 6 +- .../transport/nio/channel/WriteContext.java | 3 +- .../nio/SocketEventHandlerTests.java | 4 +- .../transport/nio/SocketSelectorTests.java | 14 ++-- .../transport/nio/WriteOperationTests.java | 4 +- .../nio/channel/TcpWriteContextTests.java | 24 +++--- 17 files changed, 325 insertions(+), 121 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java create mode 100644 test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java new file mode 100644 index 0000000000000..d94cd1db21692 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java @@ -0,0 +1,60 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.embedded.EmbeddedChannel; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.transport.netty4.Netty4Utils; + +import java.util.LinkedList; + +public class ESEmbeddedChannel extends EmbeddedChannel { + + private LinkedList> messages = new LinkedList<>(); + + public ESEmbeddedChannel() { + super(); + pipeline().addFirst("promise_captor", new ChannelOutboundHandlerAdapter() { + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + // This is a little tricky. The embedded channel will complete the promise once it writes the message + // to its outbound buffer. We do not want to complete the promise until the message is sent. So we + // intercept the promise and pass a different promise back to the rest of the pipeline. + + try { + BytesReference bytesReference = Netty4Utils.toBytesReference((ByteBuf) msg); + messages.add(new Tuple<>(bytesReference, promise)); + } catch (Exception e) { + promise.setFailure(e); + } + } + }); + } + + public Tuple getMessage() { + return messages.pollFirst(); + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index 40e98ac66b60c..a6d105a7b5315 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -77,6 +77,7 @@ private ByteBuf toByteBuf(int size) { } else { CompositeByteBuf byteBuf = Unpooled.compositeBuffer(size); for (NetworkBytesReference reference : references) { + // TODO: Do I need to increment reader indexes? byteBuf.addComponent(true, Unpooled.wrappedBuffer(reference.getReadByteBuffer())); } return byteBuf; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java new file mode 100644 index 0000000000000..1cd8b09fbf6cd --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -0,0 +1,55 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.WriteContext; + +import java.io.IOException; + +public class HttpWriteContext implements WriteContext { + + @Override + public void sendMessage(Object message, ActionListener listener) { + + } + + @Override + public void queueWriteOperations(WriteOperation writeOperation) { + + } + + @Override + public void flushChannel() throws IOException { + + } + + @Override + public boolean hasQueuedWriteOps() { + return false; + } + + @Override + public void clearQueuedWriteOps(Exception e) { + + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java new file mode 100644 index 0000000000000..76aaf46d0c7cb --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java @@ -0,0 +1,79 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelPromise; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.HttpResponse; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.transport.netty4.Netty4Utils; +import org.elasticsearch.transport.nio.ByteWriteOperation; +import org.elasticsearch.transport.nio.NetworkBytesReference; +import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +public class HttpWriteOperation extends WriteOperation { + + private final ESEmbeddedChannel nettyChannel; + private NetworkBytesReference[] byteReferences; + private HttpResponse httpResponse; + + public HttpWriteOperation(NioSocketChannel channel, ESEmbeddedChannel nettyChannel, Object object, ChannelPromise promise) { + super(channel, new ActionListener() { + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + } + }); + this.nettyChannel = nettyChannel; + if (object instanceof BytesReference) { + byteReferences = ByteWriteOperation.toArray((BytesReference) object); + } else { + httpResponse = (HttpResponse) object; + } + } + + @Override + public NetworkBytesReference[] getByteReferences() { + if (byteReferences == null) { + assert channel.getSelector().isOnCurrentThread() : "must be on selector thread to serialize http response"; + nettyChannel.writeOutbound(httpResponse); + // I do not think this works with pipelined work + Tuple t = nettyChannel.getMessage(); + ChannelPromise serializationPromise = t.v2(); + if (serializationPromise.isDone() && serializationPromise.isSuccess() == false) { + throw new ElasticsearchException(serializationPromise.cause()); + } + byteReferences = ByteWriteOperation.toArray(t.v1()); + } + return byteReferences; + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java index 273debc785965..8bfbec56ca73c 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java @@ -35,6 +35,7 @@ import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http.cookie.ServerCookieDecoder; import io.netty.handler.codec.http.cookie.ServerCookieEncoder; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; @@ -49,7 +50,10 @@ import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.transport.netty4.Netty4Utils; +import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; import java.util.List; @@ -60,6 +64,7 @@ public final class NioHttpChannel extends AbstractRestChannel { private final Channel channel; private final FullHttpRequest nettyRequest; + private final NioSocketChannel nioChannel; private final HttpPipelinedRequest pipelinedRequest; private final ThreadContext threadContext; private final Netty4CorsConfig corsConfig; @@ -67,16 +72,18 @@ public final class NioHttpChannel extends AbstractRestChannel { /** * @param request The request that is handled by this channel. + * @param channel * @param pipelinedRequest If HTTP pipelining is enabled provide the corresponding pipelined request. May be null if * HTTP pipelining is disabled. * @param detailedErrorsEnabled true iff error messages should include stack traces. * @param threadContext the thread context for the channel */ - public NioHttpChannel(Netty4HttpRequest request, HttpPipelinedRequest pipelinedRequest, boolean detailedErrorsEnabled, + public NioHttpChannel(Netty4HttpRequest request, NioSocketChannel channel, HttpPipelinedRequest pipelinedRequest, boolean detailedErrorsEnabled, ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { super(request, detailedErrorsEnabled); this.channel = request.getChannel(); this.nettyRequest = request.request(); + this.nioChannel = channel; this.pipelinedRequest = pipelinedRequest; this.threadContext = threadContext; this.corsConfig = corsConfig; @@ -144,7 +151,17 @@ public void sendResponse(RestResponse response) { } else { msg = resp; } - channel.writeAndFlush(msg, promise); + nioChannel.getWriteContext().sendMessage(msg, new ActionListener() { + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + } + }); releaseContent = false; releaseBytesStreamOutput = false; } finally { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 21dc107933dab..0ed0cf105eb11 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -95,8 +95,8 @@ protected NioHttpNettyAdaptor(Settings settings, BiConsumer() { - @Override - public void onResponse(NioChannel nioChannel) { - promise.setSuccess(); - // We should only be using unpooled buffers. So releasing only removes them to ensure they can - // be GCed. - ch.releaseOutbound(); - } - - @Override - public void onFailure(Exception e) { - promise.setFailure(e); - // We should only be using unpooled buffers. So releasing only removes them to ensure they can - // be GCed. - ch.releaseOutbound(); - } - }); - - // This is a little tricky. The embedded channel will complete the promise once it writes the message - // to its outbound buffer. We do not want to complete the promise until the message is sent. So we - // intercept the promise and pass a different promise back to the rest of the pipeline. - ctx.write(msg, ch.newPromise()); - } - }); ch.pipeline().addLast(new HttpContentDecompressor()); ch.pipeline().addLast(new HttpResponseEncoder()); final HttpObjectAggregator aggregator = new HttpObjectAggregator(maxContentLength); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 4eac175381e55..58f6571f9bc54 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.http.nio; import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelFuture; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.DecoderResult; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -38,7 +37,7 @@ import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; import org.junit.Before; @@ -47,7 +46,6 @@ import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -57,7 +55,7 @@ public class NioHttpNettyAdaptorTests extends ESTestCase { private NioHttpNettyAdaptor adaptor; private NioSocketChannel nioSocketChannel; private WriteContext writeContext; - private ArgumentCaptor writeOperation; + private ArgumentCaptor writeOperation; @Before @SuppressWarnings("unchecked") @@ -66,7 +64,7 @@ public void setMocks() { adaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); nioSocketChannel = mock(NioSocketChannel.class); writeContext = mock(WriteContext.class); - writeOperation = ArgumentCaptor.forClass(WriteOperation.class); + writeOperation = ArgumentCaptor.forClass(ByteWriteOperation.class); when(nioSocketChannel.getWriteContext()).thenReturn(writeContext); } @@ -146,7 +144,7 @@ public void testEncodeHttpResponse() { Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); - EmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(nioSocketChannel); + ESEmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(nioSocketChannel); // Must send a request through pipeline inorder to handle response HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java index 07bcea7062744..bd6f0beea8db9 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java @@ -72,7 +72,7 @@ public void handleMessage(NioSocketChannel channel, Channel nettyChannel, Object request.headers(), request.trailingHeaders()); final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, nettyChannel); - final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, + final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, channel, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, resetCookies); if (request.decoderResult().isSuccess()) { diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java new file mode 100644 index 0000000000000..c3bcbc7fe3b7b --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java @@ -0,0 +1,63 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +import java.io.IOException; +import java.util.ArrayList; + +public class ByteWriteOperation extends WriteOperation { + + private final NetworkBytesReference[] references; + + public ByteWriteOperation(NioSocketChannel channel, BytesReference bytesReference, ActionListener listener) { + super(channel, listener); + this.references = toArray(bytesReference); + } + + @Override + public NetworkBytesReference[] getByteReferences() { + return references; + } + + public static NetworkBytesReference[] toArray(BytesReference reference) { + BytesRefIterator byteRefIterator = reference.iterator(); + BytesRef r; + try { + // Most network messages are composed of three buffers + ArrayList references = new ArrayList<>(3); + while ((r = byteRefIterator.next()) != null) { + references.add(NetworkBytesReference.wrap(new BytesArray(r), r.length, 0)); + } + return references.toArray(new NetworkBytesReference[references.size()]); + + } catch (IOException e) { + // this is really an error since we don't do IO in our bytesreferences + throw new AssertionError("won't happen", e); + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java index b4da075f0fcc9..fe27d0b935bd2 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java @@ -40,7 +40,7 @@ public class SocketSelector extends ESSelector { private final ConcurrentLinkedQueue newChannels = new ConcurrentLinkedQueue<>(); - private final ConcurrentLinkedQueue queuedWrites = new ConcurrentLinkedQueue<>(); + private final ConcurrentLinkedQueue queuedWrites = new ConcurrentLinkedQueue<>(); private final SocketEventHandler eventHandler; public SocketSelector(SocketEventHandler eventHandler) throws IOException { @@ -67,7 +67,7 @@ void doSelect(int timeout) throws IOException, ClosedSelectorException { @Override void cleanup() { - WriteOperation op; + ByteWriteOperation op; while ((op = queuedWrites.poll()) != null) { op.getListener().onFailure(new ClosedSelectorException()); } @@ -92,7 +92,7 @@ public void scheduleForRegistration(NioSocketChannel nioSocketChannel) { * * @param writeOperation to be queued */ - public void queueWrite(WriteOperation writeOperation) { + public void queueWrite(ByteWriteOperation writeOperation) { queuedWrites.offer(writeOperation); if (isOpen() == false) { boolean wasRemoved = queuedWrites.remove(writeOperation); @@ -110,7 +110,7 @@ public void queueWrite(WriteOperation writeOperation) { * * @param writeOperation to be queued in a channel's buffer */ - public void queueWriteInChannelBuffer(WriteOperation writeOperation) { + public void queueWriteInChannelBuffer(ByteWriteOperation writeOperation) { assert isOnCurrentThread() : "Must be on selector thread"; NioSocketChannel channel = writeOperation.getChannel(); WriteContext context = channel.getWriteContext(); @@ -171,7 +171,7 @@ private void handleRead(NioSocketChannel nioSocketChannel) { } private void handleQueuedWrites() { - WriteOperation writeOperation; + ByteWriteOperation writeOperation; while ((writeOperation = queuedWrites.poll()) != null) { if (writeOperation.getChannel().isWritable()) { queueWriteInChannelBuffer(writeOperation); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java index 67ed2447f6383..34bfc507b3ece 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java @@ -1,50 +1,22 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - package org.elasticsearch.transport.nio; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import java.io.IOException; -import java.util.ArrayList; -public class WriteOperation { +public abstract class WriteOperation { - private final NioSocketChannel channel; - private final ActionListener listener; - private final NetworkBytesReference[] references; + protected final NioSocketChannel channel; + protected final ActionListener listener; - public WriteOperation(NioSocketChannel channel, BytesReference bytesReference, ActionListener listener) { + public WriteOperation(NioSocketChannel channel, ActionListener listener) { this.channel = channel; this.listener = listener; - this.references = toArray(bytesReference); } - public NetworkBytesReference[] getByteReferences() { - return references; - } + public abstract NetworkBytesReference[] getByteReferences(); public ActionListener getListener() { return listener; @@ -55,27 +27,11 @@ public NioSocketChannel getChannel() { } public boolean isFullyFlushed() { + NetworkBytesReference[] references = getByteReferences(); return references[references.length - 1].hasReadRemaining() == false; } public int flush() throws IOException { - return channel.write(references); - } - - private static NetworkBytesReference[] toArray(BytesReference reference) { - BytesRefIterator byteRefIterator = reference.iterator(); - BytesRef r; - try { - // Most network messages are composed of three buffers - ArrayList references = new ArrayList<>(3); - while ((r = byteRefIterator.next()) != null) { - references.add(NetworkBytesReference.wrap(new BytesArray(r), r.length, 0)); - } - return references.toArray(new NetworkBytesReference[references.size()]); - - } catch (IOException e) { - // this is really an error since we don't do IO in our bytesreferences - throw new AssertionError("won't happen", e); - } + return channel.write(getByteReferences()); } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java index 03eb652e1aeee..760a760571d4b 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.transport.nio.SocketSelector; +import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.WriteOperation; import java.io.IOException; @@ -38,13 +39,14 @@ public TcpWriteContext(NioSocketChannel channel) { } @Override - public void sendMessage(BytesReference reference, ActionListener listener) { + public void sendMessage(Object byteReference, ActionListener listener) { + BytesReference reference = (BytesReference) byteReference; if (channel.isWritable() == false) { listener.onFailure(new ClosedChannelException()); return; } - WriteOperation writeOperation = new WriteOperation(channel, reference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, reference, listener); SocketSelector selector = channel.getSelector(); if (selector.isOnCurrentThread() == false) { selector.queueWrite(writeOperation); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java index 1a14d279dd2fa..9beb987a8cd14 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java @@ -21,13 +21,14 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.WriteOperation; import java.io.IOException; public interface WriteContext { - void sendMessage(BytesReference reference, ActionListener listener); + void sendMessage(Object message, ActionListener listener); void queueWriteOperations(WriteOperation writeOperation); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java index 3bc5cd083a692..4244c129a15d0 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketEventHandlerTests.java @@ -127,7 +127,7 @@ public void testHandleWriteWithCompleteFlushRemovesOP_WRITEInterest() throws IOE BytesArray bytesArray = new BytesArray(new byte[1]); NetworkBytesReference networkBuffer = NetworkBytesReference.wrap(bytesArray); - channel.getWriteContext().queueWriteOperations(new WriteOperation(channel, networkBuffer, mock(ActionListener.class))); + channel.getWriteContext().queueWriteOperations(new ByteWriteOperation(channel, networkBuffer, mock(ActionListener.class))); when(rawChannel.write(ByteBuffer.wrap(bytesArray.array()))).thenReturn(1); handler.handleWrite(channel); @@ -143,7 +143,7 @@ public void testHandleWriteWithInCompleteFlushLeavesOP_WRITEInterest() throws IO BytesArray bytesArray = new BytesArray(new byte[1]); NetworkBytesReference networkBuffer = NetworkBytesReference.wrap(bytesArray, 1, 0); - channel.getWriteContext().queueWriteOperations(new WriteOperation(channel, networkBuffer, mock(ActionListener.class))); + channel.getWriteContext().queueWriteOperations(new ByteWriteOperation(channel, networkBuffer, mock(ActionListener.class))); when(rawChannel.write(ByteBuffer.wrap(bytesArray.array()))).thenReturn(0); handler.handleWrite(channel); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java index 50ce4a55b2960..7ff6faa2ee17b 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SocketSelectorTests.java @@ -146,13 +146,13 @@ public void testConnectIncompleteWillNotNotify() throws Exception { public void testQueueWriteWhenNotRunning() throws Exception { socketSelector.close(); - socketSelector.queueWrite(new WriteOperation(channel, bufferReference, listener)); + socketSelector.queueWrite(new ByteWriteOperation(channel, bufferReference, listener)); verify(listener).onFailure(any(ClosedSelectorException.class)); } public void testQueueWriteChannelIsNoLongerWritable() throws Exception { - WriteOperation writeOperation = new WriteOperation(channel, bufferReference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, bufferReference, listener); socketSelector.queueWrite(writeOperation); when(channel.isWritable()).thenReturn(false); @@ -165,7 +165,7 @@ public void testQueueWriteChannelIsNoLongerWritable() throws Exception { public void testQueueWriteSelectionKeyThrowsException() throws Exception { SelectionKey selectionKey = mock(SelectionKey.class); - WriteOperation writeOperation = new WriteOperation(channel, bufferReference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, bufferReference, listener); CancelledKeyException cancelledKeyException = new CancelledKeyException(); socketSelector.queueWrite(writeOperation); @@ -179,7 +179,7 @@ public void testQueueWriteSelectionKeyThrowsException() throws Exception { } public void testQueueWriteSuccessful() throws Exception { - WriteOperation writeOperation = new WriteOperation(channel, bufferReference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, bufferReference, listener); socketSelector.queueWrite(writeOperation); assertTrue((selectionKey.interestOps() & SelectionKey.OP_WRITE) == 0); @@ -192,7 +192,7 @@ public void testQueueWriteSuccessful() throws Exception { } public void testQueueDirectlyInChannelBufferSuccessful() throws Exception { - WriteOperation writeOperation = new WriteOperation(channel, bufferReference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, bufferReference, listener); assertTrue((selectionKey.interestOps() & SelectionKey.OP_WRITE) == 0); @@ -206,7 +206,7 @@ public void testQueueDirectlyInChannelBufferSuccessful() throws Exception { public void testQueueDirectlyInChannelBufferSelectionKeyThrowsException() throws Exception { SelectionKey selectionKey = mock(SelectionKey.class); - WriteOperation writeOperation = new WriteOperation(channel, bufferReference, listener); + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, bufferReference, listener); CancelledKeyException cancelledKeyException = new CancelledKeyException(); when(channel.isWritable()).thenReturn(true); @@ -322,7 +322,7 @@ public void testCleanup() throws Exception { socketSelector.doSelect(0); NetworkBytesReference networkBuffer = NetworkBytesReference.wrap(new BytesArray(new byte[1])); - socketSelector.queueWrite(new WriteOperation(mock(NioSocketChannel.class), networkBuffer, listener)); + socketSelector.queueWrite(new ByteWriteOperation(mock(NioSocketChannel.class), networkBuffer, listener)); socketSelector.scheduleForRegistration(unRegisteredChannel); socketSelector.cleanupAndCloseChannels(); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java index d7284491d6421..2f5b23edc5101 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/WriteOperationTests.java @@ -47,7 +47,7 @@ public void setFields() { } public void testFlush() throws IOException { - WriteOperation writeOp = new WriteOperation(channel, new BytesArray(new byte[10]), listener); + ByteWriteOperation writeOp = new ByteWriteOperation(channel, new BytesArray(new byte[10]), listener); when(channel.write(any())).thenAnswer(invocationOnMock -> { @@ -62,7 +62,7 @@ public void testFlush() throws IOException { } public void testPartialFlush() throws IOException { - WriteOperation writeOp = new WriteOperation(channel, new BytesArray(new byte[10]), listener); + ByteWriteOperation writeOp = new ByteWriteOperation(channel, new BytesArray(new byte[10]), listener); when(channel.write(any())).thenAnswer(invocationOnMock -> { NetworkBytesReference[] refs = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java index b4fab855403b9..d2d503188dab7 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpWriteContextTests.java @@ -23,7 +23,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.SocketSelector; -import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.ByteWriteOperation; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -68,7 +68,7 @@ public void testWriteFailsIfChannelNotWritable() throws Exception { public void testSendMessageFromDifferentThreadIsQueuedWithSelector() throws Exception { byte[] bytes = generateBytes(10); BytesArray bytesArray = new BytesArray(bytes); - ArgumentCaptor writeOpCaptor = ArgumentCaptor.forClass(WriteOperation.class); + ArgumentCaptor writeOpCaptor = ArgumentCaptor.forClass(ByteWriteOperation.class); when(selector.isOnCurrentThread()).thenReturn(false); when(channel.isWritable()).thenReturn(true); @@ -76,7 +76,7 @@ public void testSendMessageFromDifferentThreadIsQueuedWithSelector() throws Exce writeContext.sendMessage(bytesArray, listener); verify(selector).queueWrite(writeOpCaptor.capture()); - WriteOperation writeOp = writeOpCaptor.getValue(); + ByteWriteOperation writeOp = writeOpCaptor.getValue(); assertSame(listener, writeOp.getListener()); assertSame(channel, writeOp.getChannel()); @@ -86,14 +86,14 @@ public void testSendMessageFromDifferentThreadIsQueuedWithSelector() throws Exce public void testSendMessageFromSameThreadIsQueuedInChannel() throws Exception { byte[] bytes = generateBytes(10); BytesArray bytesArray = new BytesArray(bytes); - ArgumentCaptor writeOpCaptor = ArgumentCaptor.forClass(WriteOperation.class); + ArgumentCaptor writeOpCaptor = ArgumentCaptor.forClass(ByteWriteOperation.class); when(channel.isWritable()).thenReturn(true); writeContext.sendMessage(bytesArray, listener); verify(selector).queueWriteInChannelBuffer(writeOpCaptor.capture()); - WriteOperation writeOp = writeOpCaptor.getValue(); + ByteWriteOperation writeOp = writeOpCaptor.getValue(); assertSame(listener, writeOp.getListener()); assertSame(channel, writeOp.getChannel()); @@ -103,7 +103,7 @@ public void testSendMessageFromSameThreadIsQueuedInChannel() throws Exception { public void testWriteIsQueuedInChannel() throws Exception { assertFalse(writeContext.hasQueuedWriteOps()); - writeContext.queueWriteOperations(new WriteOperation(channel, new BytesArray(generateBytes(10)), listener)); + writeContext.queueWriteOperations(new ByteWriteOperation(channel, new BytesArray(generateBytes(10)), listener)); assertTrue(writeContext.hasQueuedWriteOps()); } @@ -111,7 +111,7 @@ public void testWriteIsQueuedInChannel() throws Exception { public void testWriteOpsCanBeCleared() throws Exception { assertFalse(writeContext.hasQueuedWriteOps()); - writeContext.queueWriteOperations(new WriteOperation(channel, new BytesArray(generateBytes(10)), listener)); + writeContext.queueWriteOperations(new ByteWriteOperation(channel, new BytesArray(generateBytes(10)), listener)); assertTrue(writeContext.hasQueuedWriteOps()); @@ -126,7 +126,7 @@ public void testWriteOpsCanBeCleared() throws Exception { public void testQueuedWriteIsFlushedInFlushCall() throws Exception { assertFalse(writeContext.hasQueuedWriteOps()); - WriteOperation writeOperation = mock(WriteOperation.class); + ByteWriteOperation writeOperation = mock(ByteWriteOperation.class); writeContext.queueWriteOperations(writeOperation); assertTrue(writeContext.hasQueuedWriteOps()); @@ -143,7 +143,7 @@ public void testQueuedWriteIsFlushedInFlushCall() throws Exception { public void testPartialFlush() throws IOException { assertFalse(writeContext.hasQueuedWriteOps()); - WriteOperation writeOperation = mock(WriteOperation.class); + ByteWriteOperation writeOperation = mock(ByteWriteOperation.class); writeContext.queueWriteOperations(writeOperation); assertTrue(writeContext.hasQueuedWriteOps()); @@ -160,8 +160,8 @@ public void testMultipleWritesPartialFlushes() throws IOException { assertFalse(writeContext.hasQueuedWriteOps()); ActionListener listener2 = mock(ActionListener.class); - WriteOperation writeOperation1 = mock(WriteOperation.class); - WriteOperation writeOperation2 = mock(WriteOperation.class); + ByteWriteOperation writeOperation1 = mock(ByteWriteOperation.class); + ByteWriteOperation writeOperation2 = mock(ByteWriteOperation.class); when(writeOperation1.getListener()).thenReturn(listener); when(writeOperation2.getListener()).thenReturn(listener2); writeContext.queueWriteOperations(writeOperation1); @@ -188,7 +188,7 @@ public void testMultipleWritesPartialFlushes() throws IOException { public void testWhenIOExceptionThrownListenerIsCalled() throws IOException { assertFalse(writeContext.hasQueuedWriteOps()); - WriteOperation writeOperation = mock(WriteOperation.class); + ByteWriteOperation writeOperation = mock(ByteWriteOperation.class); writeContext.queueWriteOperations(writeOperation); assertTrue(writeContext.hasQueuedWriteOps()); From d9a5d80f2eea6ba092ee3a4d576b87a83919690b Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 11 Aug 2017 16:35:06 -0500 Subject: [PATCH 10/35] WIP --- .../http/nio/ESChannelPromise.java | 205 ++++++++++++++++++ .../http/nio/ESEmbeddedChannel.java | 8 + .../http/nio/HttpWriteContext.java | 51 ++++- .../http/nio/HttpWriteOperation.java | 51 +---- .../transport/nio/ByteWriteOperation.java | 10 +- .../transport/nio/SocketSelector.java | 10 +- .../transport/nio/WriteOperation.java | 10 - .../nio/channel/TcpWriteContext.java | 8 +- 8 files changed, 286 insertions(+), 67 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java new file mode 100644 index 0000000000000..527c30583203f --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java @@ -0,0 +1,205 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.transport.nio.channel.NioChannel; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class ESChannelPromise implements ActionListener, ChannelPromise { + + private final ChannelPromise promise; + + public ESChannelPromise(ChannelPromise promise) { + this.promise = promise; + } + + @Override + public void onResponse(NioChannel nioChannel) { + promise.setSuccess(); + } + + @Override + public void onFailure(Exception e) { + promise.setFailure(e); + } + + @Override + public Channel channel() { + return promise.channel(); + } + + @Override + public ChannelPromise setSuccess(Void result) { + return promise.setSuccess(result); + } + + @Override + public boolean trySuccess(Void result) { + return promise.trySuccess(result); + } + + @Override + public ChannelPromise setSuccess() { + return promise.setSuccess(); + } + + @Override + public boolean trySuccess() { + return promise.trySuccess(); + } + + @Override + public ChannelPromise setFailure(Throwable cause) { + return promise.setFailure(cause); + } + + @Override + public boolean tryFailure(Throwable cause) { + return promise.tryFailure(cause); + } + + @Override + public boolean setUncancellable() { + return promise.setUncancellable(); + } + + @Override + public boolean isSuccess() { + return promise.isSuccess(); + } + + @Override + public boolean isCancellable() { + return promise.isCancellable(); + } + + @Override + public Throwable cause() { + return promise.cause(); + } + + @Override + public ChannelPromise addListener(GenericFutureListener> listener) { + return promise.addListener(listener); + } + + @Override + public ChannelPromise addListeners(GenericFutureListener>... listeners) { + return promise.addListeners(listeners); + } + + @Override + public ChannelPromise removeListener(GenericFutureListener> listener) { + return promise.removeListener(listener); + } + + @Override + public ChannelPromise removeListeners(GenericFutureListener>... listeners) { + return promise.removeListeners(listeners); + } + + @Override + public ChannelPromise sync() throws InterruptedException { + return promise.sync(); + } + + @Override + public ChannelPromise syncUninterruptibly() { + return promise.syncUninterruptibly(); + } + + @Override + public ChannelPromise await() throws InterruptedException { + return promise.await(); + } + + @Override + public ChannelPromise awaitUninterruptibly() { + return promise.awaitUninterruptibly(); + } + + @Override + public boolean await(long timeout, TimeUnit unit) throws InterruptedException { + return promise.await(timeout, unit); + } + + @Override + public boolean await(long timeoutMillis) throws InterruptedException { + return promise.await(timeoutMillis); + } + + @Override + public boolean awaitUninterruptibly(long timeout, TimeUnit unit) { + return promise.awaitUninterruptibly(timeout, unit); + } + + @Override + public boolean awaitUninterruptibly(long timeoutMillis) { + return promise.awaitUninterruptibly(timeoutMillis); + } + + @Override + public Void getNow() { + return promise.getNow(); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return promise.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() { + return promise.isCancelled(); + } + + @Override + public boolean isDone() { + return promise.isDone(); + } + + @Override + public Void get() throws InterruptedException, ExecutionException { + return promise.get(); + } + + @Override + public Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return promise.get(timeout, unit); + } + + @Override + public boolean isVoid() { + return promise.isVoid(); + } + + @Override + public ChannelPromise unvoid() { + return promise.unvoid(); + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java index d94cd1db21692..51692a58dc21b 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java @@ -57,4 +57,12 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) public Tuple getMessage() { return messages.pollFirst(); } + + public boolean hasMessages() { + return messages.size() > 0; + } + + public LinkedList> getMessages() { + return messages; + } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java index 1cd8b09fbf6cd..2af4b10d2614f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -19,37 +19,84 @@ package org.elasticsearch.http.nio; +import io.netty.channel.ChannelPromise; +import io.netty.handler.codec.http.HttpResponse; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.transport.nio.ByteWriteOperation; +import org.elasticsearch.transport.nio.SocketSelector; import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioChannel; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; import java.io.IOException; +import java.nio.channels.ClosedChannelException; +import java.util.LinkedList; +import java.util.Queue; public class HttpWriteContext implements WriteContext { + private final NioSocketChannel channel; + private final ESEmbeddedChannel adaptor; + + public HttpWriteContext(NioSocketChannel channel, ESEmbeddedChannel adaptor) { + this.channel = channel; + this.adaptor = adaptor; + } + @Override public void sendMessage(Object message, ActionListener listener) { + if (channel.isWritable() == false) { + listener.onFailure(new ClosedChannelException()); + return; + } + HttpResponse response = (HttpResponse) message; + WriteOperation writeOperation = new HttpWriteOperation(channel, response, listener); + SocketSelector selector = channel.getSelector(); + if (selector.isOnCurrentThread() == false) { + selector.queueWrite(writeOperation); + return; + } + + // TODO: Eval if we will allow writes from sendMessage + selector.queueWriteInChannelBuffer(writeOperation); } @Override public void queueWriteOperations(WriteOperation writeOperation) { + assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to queue writes"; + HttpWriteOperation httpWriteOperation = (HttpWriteOperation) writeOperation; + ESChannelPromise listener = (ESChannelPromise) httpWriteOperation.getListener(); + adaptor.write(httpWriteOperation.getHttpResponse(), listener); } @Override public void flushChannel() throws IOException { + assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; } @Override public boolean hasQueuedWriteOps() { - return false; + assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; + return adaptor.hasMessages(); } @Override public void clearQueuedWriteOps(Exception e) { - + assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; + for (Tuple message : adaptor.getMessages()) { + message.v2().setFailure(e); + BytesReference bytes = message.v1(); + if (bytes instanceof Releasable) { + Releasables.close((Releasable) bytes); + } + } } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java index 76aaf46d0c7cb..c93656066ab6f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java @@ -19,61 +19,22 @@ package org.elasticsearch.http.nio; -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelPromise; -import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.HttpResponse; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.transport.netty4.Netty4Utils; -import org.elasticsearch.transport.nio.ByteWriteOperation; -import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; public class HttpWriteOperation extends WriteOperation { - private final ESEmbeddedChannel nettyChannel; - private NetworkBytesReference[] byteReferences; - private HttpResponse httpResponse; + private final HttpResponse httpResponse; - public HttpWriteOperation(NioSocketChannel channel, ESEmbeddedChannel nettyChannel, Object object, ChannelPromise promise) { - super(channel, new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - promise.setSuccess(); - } - - @Override - public void onFailure(Exception e) { - promise.setFailure(e); - } - }); - this.nettyChannel = nettyChannel; - if (object instanceof BytesReference) { - byteReferences = ByteWriteOperation.toArray((BytesReference) object); - } else { - httpResponse = (HttpResponse) object; - } + public HttpWriteOperation(NioSocketChannel channel, HttpResponse response, ActionListener listener) { + super(channel, listener); + this.httpResponse = response; } - @Override - public NetworkBytesReference[] getByteReferences() { - if (byteReferences == null) { - assert channel.getSelector().isOnCurrentThread() : "must be on selector thread to serialize http response"; - nettyChannel.writeOutbound(httpResponse); - // I do not think this works with pipelined work - Tuple t = nettyChannel.getMessage(); - ChannelPromise serializationPromise = t.v2(); - if (serializationPromise.isDone() && serializationPromise.isSuccess() == false) { - throw new ElasticsearchException(serializationPromise.cause()); - } - byteReferences = ByteWriteOperation.toArray(t.v1()); - } - return byteReferences; + public HttpResponse getHttpResponse() { + return httpResponse; } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java index c3bcbc7fe3b7b..abe03ea7d36c7 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/ByteWriteOperation.java @@ -39,11 +39,19 @@ public ByteWriteOperation(NioSocketChannel channel, BytesReference bytesReferenc this.references = toArray(bytesReference); } - @Override public NetworkBytesReference[] getByteReferences() { return references; } + public boolean isFullyFlushed() { + NetworkBytesReference[] references = getByteReferences(); + return references[references.length - 1].hasReadRemaining() == false; + } + + public int flush() throws IOException { + return channel.write(getByteReferences()); + } + public static NetworkBytesReference[] toArray(BytesReference reference) { BytesRefIterator byteRefIterator = reference.iterator(); BytesRef r; diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java index fe27d0b935bd2..b4da075f0fcc9 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/SocketSelector.java @@ -40,7 +40,7 @@ public class SocketSelector extends ESSelector { private final ConcurrentLinkedQueue newChannels = new ConcurrentLinkedQueue<>(); - private final ConcurrentLinkedQueue queuedWrites = new ConcurrentLinkedQueue<>(); + private final ConcurrentLinkedQueue queuedWrites = new ConcurrentLinkedQueue<>(); private final SocketEventHandler eventHandler; public SocketSelector(SocketEventHandler eventHandler) throws IOException { @@ -67,7 +67,7 @@ void doSelect(int timeout) throws IOException, ClosedSelectorException { @Override void cleanup() { - ByteWriteOperation op; + WriteOperation op; while ((op = queuedWrites.poll()) != null) { op.getListener().onFailure(new ClosedSelectorException()); } @@ -92,7 +92,7 @@ public void scheduleForRegistration(NioSocketChannel nioSocketChannel) { * * @param writeOperation to be queued */ - public void queueWrite(ByteWriteOperation writeOperation) { + public void queueWrite(WriteOperation writeOperation) { queuedWrites.offer(writeOperation); if (isOpen() == false) { boolean wasRemoved = queuedWrites.remove(writeOperation); @@ -110,7 +110,7 @@ public void queueWrite(ByteWriteOperation writeOperation) { * * @param writeOperation to be queued in a channel's buffer */ - public void queueWriteInChannelBuffer(ByteWriteOperation writeOperation) { + public void queueWriteInChannelBuffer(WriteOperation writeOperation) { assert isOnCurrentThread() : "Must be on selector thread"; NioSocketChannel channel = writeOperation.getChannel(); WriteContext context = channel.getWriteContext(); @@ -171,7 +171,7 @@ private void handleRead(NioSocketChannel nioSocketChannel) { } private void handleQueuedWrites() { - ByteWriteOperation writeOperation; + WriteOperation writeOperation; while ((writeOperation = queuedWrites.poll()) != null) { if (writeOperation.getChannel().isWritable()) { queueWriteInChannelBuffer(writeOperation); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java index 34bfc507b3ece..4bd4418826bf6 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java @@ -16,7 +16,6 @@ public WriteOperation(NioSocketChannel channel, ActionListener liste this.listener = listener; } - public abstract NetworkBytesReference[] getByteReferences(); public ActionListener getListener() { return listener; @@ -25,13 +24,4 @@ public ActionListener getListener() { public NioSocketChannel getChannel() { return channel; } - - public boolean isFullyFlushed() { - NetworkBytesReference[] references = getByteReferences(); - return references[references.length - 1].hasReadRemaining() == false; - } - - public int flush() throws IOException { - return channel.write(getByteReferences()); - } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java index 760a760571d4b..79b71595205ee 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java @@ -32,7 +32,7 @@ public class TcpWriteContext implements WriteContext { private final NioSocketChannel channel; - private final LinkedList queued = new LinkedList<>(); + private final LinkedList queued = new LinkedList<>(); public TcpWriteContext(NioSocketChannel channel) { this.channel = channel; @@ -60,7 +60,7 @@ public void sendMessage(Object byteReference, ActionListener listene @Override public void queueWriteOperations(WriteOperation writeOperation) { assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to queue writes"; - queued.add(writeOperation); + queued.add((ByteWriteOperation) writeOperation); } @Override @@ -89,7 +89,7 @@ public void clearQueuedWriteOps(Exception e) { queued.clear(); } - private void singleFlush(WriteOperation headOp) throws IOException { + private void singleFlush(ByteWriteOperation headOp) throws IOException { try { headOp.flush(); } catch (IOException e) { @@ -107,7 +107,7 @@ private void singleFlush(WriteOperation headOp) throws IOException { private void multiFlush() throws IOException { boolean lastOpCompleted = true; while (lastOpCompleted && queued.isEmpty() == false) { - WriteOperation op = queued.pop(); + ByteWriteOperation op = queued.pop(); singleFlush(op); lastOpCompleted = op.isFullyFlushed(); } From 024d51d14b3390efe102dc266577e79cfdbafa90 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sun, 13 Aug 2017 15:39:13 -0500 Subject: [PATCH 11/35] WIP --- .../http/nio/HttpWriteContext.java | 30 +++++++++++++ .../http/nio/NioHttpNettyAdaptorTests.java | 42 ++++++++++++++----- .../nio/channel/TcpWriteContext.java | 13 +----- .../transport/nio/channel/WriteContext.java | 16 +++++++ 4 files changed, 79 insertions(+), 22 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java index 2af4b10d2614f..7597006e4d928 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -42,6 +42,7 @@ public class HttpWriteContext implements WriteContext { private final NioSocketChannel channel; private final ESEmbeddedChannel adaptor; + private ByteWriteOperation partiallyFlushed; public HttpWriteContext(NioSocketChannel channel, ESEmbeddedChannel adaptor) { this.channel = channel; @@ -80,6 +81,33 @@ public void queueWriteOperations(WriteOperation writeOperation) { public void flushChannel() throws IOException { assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; + if (partiallyFlushed != null) { + if (WriteContext.flushOperation(channel, partiallyFlushed)) { + partiallyFlushed = null; + } else { + return; + } + } + + LinkedList> messages = adaptor.getMessages(); + + Tuple message; + boolean lastMessageFullyFlushed = true; + while ((message = messages.pop()) != null && lastMessageFullyFlushed) { + ChannelPromise promise = message.v2(); + ESChannelPromise listener; + if (promise instanceof ESChannelPromise) { + listener = (ESChannelPromise) promise; + } else { + listener = new ESChannelPromise(promise); + } + ByteWriteOperation writeOperation = new ByteWriteOperation(channel, message.v1(), listener); + + if (WriteContext.flushOperation(channel, writeOperation) == false) { + partiallyFlushed = writeOperation; + lastMessageFullyFlushed = false; + } + } } @Override @@ -98,5 +126,7 @@ public void clearQueuedWriteOps(Exception e) { Releasables.close((Releasable) bytes); } } + + adaptor.close().syncUninterruptibly(); } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 58f6571f9bc54..5f57403554fd7 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.http.nio; import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelPromise; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.DecoderResult; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -27,16 +28,21 @@ import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpRequestEncoder; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseDecoder; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.LastHttpContent; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.netty4.Netty4Utils; import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; @@ -57,10 +63,14 @@ public class NioHttpNettyAdaptorTests extends ESTestCase { private WriteContext writeContext; private ArgumentCaptor writeOperation; + private final EmbeddedChannel requestEncoder = new EmbeddedChannel(new HttpRequestEncoder()); + private final EmbeddedChannel responseDecoder = new EmbeddedChannel(new HttpResponseDecoder()); + @Before @SuppressWarnings("unchecked") public void setMocks() { exceptionHandler = mock(BiConsumer.class); + adaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); nioSocketChannel = mock(NioSocketChannel.class); writeContext = mock(WriteContext.class); @@ -146,24 +156,34 @@ public void testEncodeHttpResponse() { when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); ESEmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(nioSocketChannel); - // Must send a request through pipeline inorder to handle response - HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); - EmbeddedChannel encodingChannel = new EmbeddedChannel(new HttpRequestEncoder()); - encodingChannel.writeOutbound(defaultFullHttpRequest); - ByteBuf buf = encodingChannel.readOutbound(); - adaptor.writeInbound(buf); - + prepareAdaptorForResponse(adaptor); HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); adaptor.writeOutbound(defaultFullHttpResponse); - ByteBuf encodedResponse = adaptor.readOutbound(); + Tuple encodedMessage = adaptor.getMessage(); - EmbeddedChannel decodingChannel = new EmbeddedChannel(new HttpResponseDecoder()); - decodingChannel.writeInbound(encodedResponse); - HttpResponse response = decodingChannel.readInbound(); + responseDecoder.writeInbound(Netty4Utils.toByteBuf(encodedMessage.v1())); + HttpResponse response = responseDecoder.readInbound(); assertEquals(HttpResponseStatus.OK, response.status()); assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); } + + private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { + HttpMethod method = HttpMethod.GET; + HttpVersion version = HttpVersion.HTTP_1_1; + String uri = "http://localhost:9090/" + randomAlphaOfLength(8); + + HttpRequest request = new DefaultFullHttpRequest(version, method, uri); + requestEncoder.writeOutbound(request); + ByteBuf buf = requestEncoder.readOutbound(); + adaptor.writeInbound(buf); + HttpPipelinedRequest pipelinedRequest = adaptor.readInbound(); + FullHttpRequest requestParsed = (FullHttpRequest) pipelinedRequest.last(); + assertNotNull(requestParsed); + assertEquals(requestParsed.method(), method); + assertEquals(requestParsed.protocolVersion(), version); + assertEquals(requestParsed.uri(), uri); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java index 79b71595205ee..8b7c37e6394ce 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/TcpWriteContext.java @@ -90,17 +90,8 @@ public void clearQueuedWriteOps(Exception e) { } private void singleFlush(ByteWriteOperation headOp) throws IOException { - try { - headOp.flush(); - } catch (IOException e) { - headOp.getListener().onFailure(e); - throw e; - } - - if (headOp.isFullyFlushed()) { - headOp.getListener().onResponse(channel); - } else { - queued.push(headOp); + if (WriteContext.flushOperation(channel, headOp) == false) { + queued.addFirst(headOp); } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java index 9beb987a8cd14..29d26133927bc 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java @@ -38,4 +38,20 @@ public interface WriteContext { void clearQueuedWriteOps(Exception e); + static boolean flushOperation(NioSocketChannel channel, ByteWriteOperation headOp) throws IOException { + try { + headOp.flush(); + } catch (IOException e) { + headOp.getListener().onFailure(e); + throw e; + } + + if (headOp.isFullyFlushed()) { + headOp.getListener().onResponse(channel); + return true; + } else { + return false; + } + } + } From a3ad4f2d50a9a4d22de3fe69610e7b33beece1c0 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 14 Aug 2017 21:06:12 -0500 Subject: [PATCH 12/35] WIP --- .../http/nio/NioHttpNettyAdaptorTests.java | 102 ++++++++++-------- 1 file changed, 57 insertions(+), 45 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 5f57403554fd7..b6760d0425cdd 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -28,17 +28,17 @@ import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; -import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpRequestEncoder; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseDecoder; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; -import io.netty.handler.codec.http.LastHttpContent; +import org.elasticsearch.client.http.HttpStatus; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.HttpStats; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.test.ESTestCase; @@ -60,11 +60,9 @@ public class NioHttpNettyAdaptorTests extends ESTestCase { private BiConsumer exceptionHandler; private NioHttpNettyAdaptor adaptor; private NioSocketChannel nioSocketChannel; - private WriteContext writeContext; - private ArgumentCaptor writeOperation; - private final EmbeddedChannel requestEncoder = new EmbeddedChannel(new HttpRequestEncoder()); - private final EmbeddedChannel responseDecoder = new EmbeddedChannel(new HttpResponseDecoder()); + private final RequestEncoder requestEncoder = new RequestEncoder(); + private final ResponseDecoder responseDecoder = new ResponseDecoder(); @Before @SuppressWarnings("unchecked") @@ -73,55 +71,48 @@ public void setMocks() { adaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); nioSocketChannel = mock(NioSocketChannel.class); - writeContext = mock(WriteContext.class); - writeOperation = ArgumentCaptor.forClass(ByteWriteOperation.class); - - when(nioSocketChannel.getWriteContext()).thenReturn(writeContext); } - public void testCloseEmbeddedChannelSchedulesRealChannelForClose() { + public void testCloseAdaptorSchedulesRealChannelForClose() { NioSocketChannel channel = mock(NioSocketChannel.class); - EmbeddedChannel channelAdaptor = adaptor.getAdaptor(channel); + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(channel); channelAdaptor.close(); verify(channel).closeAsync(); } - public void testDecodeHttpRequest() { - EmbeddedChannel channelAdaptor = adaptor.getAdaptor(mock(NioSocketChannel.class)); + public void testSuccessfulDecodeHttpRequest() { + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); - EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); - ch.writeOutbound(defaultFullHttpRequest); - ByteBuf buf = ch.readOutbound(); + ByteBuf buf = requestEncoder.encode(httpRequest); + int slicePoint = randomInt(buf.writerIndex() - 1); - channelAdaptor.writeInbound(buf.slice(0, 5).retainedDuplicate()); + ByteBuf slicedBuf = buf.retainedSlice(0, slicePoint); + channelAdaptor.writeInbound(slicedBuf); assertNull(channelAdaptor.readInbound()); - channelAdaptor.writeInbound(buf.slice(5, buf.writerIndex() - 5).retainedDuplicate()); + channelAdaptor.writeInbound(buf.retainedSlice(slicePoint, buf.writerIndex() - slicePoint)); HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); - assertEquals(defaultFullHttpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); - assertEquals(defaultFullHttpRequest.method(), fullHttpRequest.method()); + assertEquals(httpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); + assertEquals(httpRequest.method(), fullHttpRequest.method()); } public void testDecodeHttpRequestError() { - EmbeddedChannel channelAdaptor = adaptor.getAdaptor(mock(NioSocketChannel.class)); + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - HttpRequest defaultFullHttpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); - EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); - ch.writeOutbound(defaultFullHttpRequest); - ByteBuf buf = ch.readOutbound(); + ByteBuf buf = requestEncoder.encode(httpRequest); buf.setByte(0, ' '); buf.setByte(1, ' '); buf.setByte(2, ' '); - channelAdaptor.writeInbound(buf); - + channelAdaptor.writeInbound(buf.retainedDuplicate()); HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); @@ -131,40 +122,41 @@ public void testDecodeHttpRequestError() { } public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { - EmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "localhost:9090/got/got", false); HttpUtil.setContentLength(httpRequest, 1025); - EmbeddedChannel ch = new EmbeddedChannel(new HttpRequestEncoder()); - ch.writeOutbound(httpRequest); - ByteBuf buf = ch.readOutbound(); + ByteBuf buf = requestEncoder.encode(httpRequest); - channelAdaptor.writeInbound(buf); + channelAdaptor.writeInbound(buf.retainedDuplicate()); HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); assertNull(decodedRequest); - ByteBuf buffer = channelAdaptor.readOutbound(); + Tuple message = channelAdaptor.getMessage(); + + assertFalse(message.v2().isDone()); + + HttpResponse response = responseDecoder.decode(Netty4Utils.toByteBuf(message.v1())); + assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); + assertEquals(HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE, response.status()); } public void testEncodeHttpResponse() { - NioHttpNettyAdaptor nioHttpNettyAdaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, - Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); - ESEmbeddedChannel adaptor = nioHttpNettyAdaptor.getAdaptor(nioSocketChannel); + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - prepareAdaptorForResponse(adaptor); + prepareAdaptorForResponse(channelAdaptor); HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - adaptor.writeOutbound(defaultFullHttpResponse); - Tuple encodedMessage = adaptor.getMessage(); + channelAdaptor.writeOutbound(defaultFullHttpResponse); + Tuple encodedMessage = channelAdaptor.getMessage(); - responseDecoder.writeInbound(Netty4Utils.toByteBuf(encodedMessage.v1())); - HttpResponse response = responseDecoder.readInbound(); + HttpResponse response = responseDecoder.decode(Netty4Utils.toByteBuf(encodedMessage.v1())); assertEquals(HttpResponseStatus.OK, response.status()); assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); @@ -176,8 +168,8 @@ private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { String uri = "http://localhost:9090/" + randomAlphaOfLength(8); HttpRequest request = new DefaultFullHttpRequest(version, method, uri); - requestEncoder.writeOutbound(request); - ByteBuf buf = requestEncoder.readOutbound(); + ByteBuf buf = requestEncoder.encode(request); + adaptor.writeInbound(buf); HttpPipelinedRequest pipelinedRequest = adaptor.readInbound(); FullHttpRequest requestParsed = (FullHttpRequest) pipelinedRequest.last(); @@ -186,4 +178,24 @@ private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { assertEquals(requestParsed.protocolVersion(), version); assertEquals(requestParsed.uri(), uri); } + + private static class RequestEncoder { + + private final EmbeddedChannel requestEncoder = new EmbeddedChannel(new HttpRequestEncoder()); + + private ByteBuf encode(HttpRequest httpRequest) { + requestEncoder.writeOutbound(httpRequest); + return requestEncoder.readOutbound(); + } + } + + private static class ResponseDecoder { + + private final EmbeddedChannel responseDecoder = new EmbeddedChannel(new HttpResponseDecoder()); + + private HttpResponse decode(ByteBuf response) { + responseDecoder.writeInbound(response); + return responseDecoder.readInbound(); + } + } } From b04078d14d55efafbd1232206065f8be6ead6668 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 15 Aug 2017 15:14:08 -0500 Subject: [PATCH 13/35] WIP --- .../support/PlainListenableActionFuture.java | 2 +- .../http/nio/NioHttpNettyAdaptor.java | 5 +- .../http/nio/NioHttpNettyAdaptorTests.java | 55 +++++++++++++++++-- .../nio/channel/AbstractNioChannel.java | 2 +- .../nio/channel/ChannelConsumerAdaptor.java | 31 +++++++++++ .../transport/nio/channel/ChannelFactory.java | 5 +- .../transport/nio/channel/CloseFuture.java | 32 ++--------- .../channel/NioServerSocketChannelTests.java | 4 +- .../nio/channel/NioSocketChannelTests.java | 4 +- 9 files changed, 96 insertions(+), 44 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java diff --git a/core/src/main/java/org/elasticsearch/action/support/PlainListenableActionFuture.java b/core/src/main/java/org/elasticsearch/action/support/PlainListenableActionFuture.java index 749bf1fea019d..943c36797096c 100644 --- a/core/src/main/java/org/elasticsearch/action/support/PlainListenableActionFuture.java +++ b/core/src/main/java/org/elasticsearch/action/support/PlainListenableActionFuture.java @@ -33,7 +33,7 @@ public class PlainListenableActionFuture extends AdapterActionFuture im volatile Object listeners; boolean executedListeners = false; - private PlainListenableActionFuture() {} + protected PlainListenableActionFuture() {} /** * This method returns a listenable future. The listeners will be called on completion of the future. diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 0ed0cf105eb11..9b8788bc5268c 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -50,6 +50,7 @@ import org.elasticsearch.http.netty4.pipelining.HttpPipeliningHandler; import org.elasticsearch.transport.netty4.ByteBufBytesReferenceTests; import org.elasticsearch.transport.netty4.Netty4Utils; +import org.elasticsearch.transport.nio.channel.ChannelConsumerAdaptor; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; @@ -127,12 +128,10 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } }); ch.pipeline().addLast("close_adaptor", new ChannelOutboundHandlerAdapter() { - @Override public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { - channel.closeAsync(); + channel.closeAsync().addListener(new ESChannelPromise(promise)); } - }); return ch; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index b6760d0425cdd..e85739fc33aa8 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.http.nio; import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelPromise; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.DecoderResult; @@ -34,6 +35,7 @@ import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.http.HttpStatus; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; @@ -44,11 +46,14 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.netty4.Netty4Utils; import org.elasticsearch.transport.nio.ByteWriteOperation; +import org.elasticsearch.transport.nio.channel.CloseFuture; +import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.WriteContext; import org.junit.Before; import org.mockito.ArgumentCaptor; +import java.io.IOException; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; @@ -73,18 +78,38 @@ public void setMocks() { nioSocketChannel = mock(NioSocketChannel.class); } + @SuppressWarnings("unchecked") public void testCloseAdaptorSchedulesRealChannelForClose() { NioSocketChannel channel = mock(NioSocketChannel.class); ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(channel); + ArgumentCaptor captor = ArgumentCaptor.forClass(ActionListener.class); + CloseFuture closeFuture = mock(CloseFuture.class); + when(channel.closeAsync()).thenReturn(closeFuture); - channelAdaptor.close(); + ChannelFuture nettyFuture = channelAdaptor.close(); verify(channel).closeAsync(); + verify(closeFuture).addListener(captor.capture()); + + ActionListener listener = captor.getValue(); + assertFalse(nettyFuture.isDone()); + if (randomBoolean()) { + listener.onResponse(channel); + assertTrue(nettyFuture.isSuccess()); + } else { + IOException e = new IOException(); + listener.onFailure(e); + assertFalse(nettyFuture.isSuccess()); + assertSame(e, nettyFuture.cause()); + } + + assertTrue(nettyFuture.isDone()); } public void testSuccessfulDecodeHttpRequest() { ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + String uri = "localhost:9090/" + randomAlphaOfLength(8); + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); ByteBuf buf = requestEncoder.encode(httpRequest); int slicePoint = randomInt(buf.writerIndex() - 1); @@ -105,7 +130,8 @@ public void testSuccessfulDecodeHttpRequest() { public void testDecodeHttpRequestError() { ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "localhost:9090/got/got"); + String uri = "localhost:9090/" + randomAlphaOfLength(8); + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); ByteBuf buf = requestEncoder.encode(httpRequest); buf.setByte(0, ' '); @@ -124,7 +150,8 @@ public void testDecodeHttpRequestError() { public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); - HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "localhost:9090/got/got", false); + String uri = "localhost:9090/" + randomAlphaOfLength(8); + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, uri, false); HttpUtil.setContentLength(httpRequest, 1025); ByteBuf buf = requestEncoder.encode(httpRequest); @@ -145,8 +172,6 @@ public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { } public void testEncodeHttpResponse() { - NioSocketChannel nioSocketChannel = mock(NioSocketChannel.class); - when(nioSocketChannel.getWriteContext()).thenReturn(mock(WriteContext.class)); ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); prepareAdaptorForResponse(channelAdaptor); @@ -162,6 +187,24 @@ public void testEncodeHttpResponse() { assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); } +// public void testEncodeHttpResponseAfterClose() { +// ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); +// +// prepareAdaptorForResponse(channelAdaptor); +// +// HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); +// +// ChannelFuture close = channelAdaptor.close(); +// +// channelAdaptor.writeOutbound(defaultFullHttpResponse); +// Tuple encodedMessage = channelAdaptor.getMessage(); +// +// HttpResponse response = responseDecoder.decode(Netty4Utils.toByteBuf(encodedMessage.v1())); +// +// assertEquals(HttpResponseStatus.OK, response.status()); +// assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); +// } + private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { HttpMethod method = HttpMethod.GET; HttpVersion version = HttpVersion.HTTP_1_1; diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java index c02312aab51d6..36f776a625aea 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/AbstractNioChannel.java @@ -113,7 +113,7 @@ public void closeFromSelector() { closeRawChannel(); closedOnThisCall = closeFuture.channelClosed(this); } catch (IOException e) { - closedOnThisCall = closeFuture.channelCloseThrewException(this, e); + closedOnThisCall = closeFuture.channelCloseThrewException(e); } finally { if (closedOnThisCall) { selector.removeRegisteredChannel(this); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java new file mode 100644 index 0000000000000..fc605171c348a --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java @@ -0,0 +1,31 @@ +package org.elasticsearch.transport.nio.channel; + +import org.elasticsearch.action.ActionListener; + +import java.util.function.Consumer; + +public class ChannelConsumerAdaptor implements ActionListener { + + private final NioChannel channel; + private final Consumer consumer; + + private ChannelConsumerAdaptor(NioChannel channel, Consumer consumer) { + this.channel = channel; + this.consumer = consumer; + } + + static ChannelConsumerAdaptor adapt(NioChannel channel, Consumer consumer) { + return new ChannelConsumerAdaptor(channel, consumer); + } + + @Override + public void onResponse(NioChannel channel) { + consumer.accept(channel); + } + + @Override + public void onFailure(Exception e) { + consumer.accept(channel); + } +} + diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java index 83659c647ac01..e2c4ea3295c3e 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelFactory.java @@ -21,6 +21,7 @@ import org.apache.lucene.util.IOUtils; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.mocksocket.PrivilegedSocketAccess; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.nio.AcceptingSelector; @@ -59,7 +60,7 @@ public NioSocketChannel openNioChannel(InetSocketAddress remoteAddress, SocketSe SocketChannel rawChannel = rawChannelFactory.openNioChannel(remoteAddress); NioSocketChannel channel = new NioSocketChannel(NioChannel.CLIENT, rawChannel, selector); contextSetter.accept(channel); - channel.getCloseFuture().setListener(closeListener); + channel.getCloseFuture().addListener(ChannelConsumerAdaptor.adapt(channel, closeListener)); scheduleChannel(channel, selector); return channel; } @@ -69,7 +70,7 @@ public NioSocketChannel acceptNioChannel(NioServerSocketChannel serverChannel, S SocketChannel rawChannel = rawChannelFactory.acceptNioChannel(serverChannel); NioSocketChannel channel = new NioSocketChannel(serverChannel.getProfile(), rawChannel, selector); contextSetter.accept(channel); - channel.getCloseFuture().setListener(closeListener); + channel.getCloseFuture().addListener(ChannelConsumerAdaptor.adapt(channel, closeListener)); scheduleChannel(channel, selector); return channel; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/CloseFuture.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/CloseFuture.java index c27ba306e0e60..33441f1b0ac12 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/CloseFuture.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/CloseFuture.java @@ -19,18 +19,14 @@ package org.elasticsearch.transport.nio.channel; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.common.util.concurrent.BaseFuture; +import org.elasticsearch.action.support.PlainListenableActionFuture; import java.io.IOException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.function.Consumer; -public class CloseFuture extends BaseFuture { - - private final SetOnce> listener = new SetOnce<>(); +public class CloseFuture extends PlainListenableActionFuture { @Override public boolean cancel(boolean mayInterruptIfRunning) { @@ -76,31 +72,13 @@ public boolean isClosed() { return super.isDone(); } - public void setListener(Consumer listener) { - this.listener.set(listener); - } - boolean channelClosed(NioChannel channel) { - boolean set = set(channel); - if (set) { - Consumer listener = this.listener.get(); - if (listener != null) { - listener.accept(channel); - } - } - return set; + return set(channel); } - boolean channelCloseThrewException(NioChannel channel, IOException ex) { - boolean set = setException(ex); - if (set) { - Consumer listener = this.listener.get(); - if (listener != null) { - listener.accept(channel); - } - } - return set; + boolean channelCloseThrewException(IOException ex) { + return setException(ex); } } diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java index 6f05d3c1f34c6..b3693bfb7866e 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioServerSocketChannelTests.java @@ -64,10 +64,10 @@ public void testClose() throws IOException, TimeoutException, InterruptedExcepti CountDownLatch latch = new CountDownLatch(1); NioChannel channel = new DoNotCloseServerChannel("nio", mock(ServerSocketChannel.class), mock(ChannelFactory.class), selector); - channel.getCloseFuture().setListener((c) -> { + channel.getCloseFuture().addListener(ChannelConsumerAdaptor.adapt(channel, (c) -> { ref.set(c); latch.countDown(); - }); + })); CloseFuture closeFuture = channel.getCloseFuture(); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java index 3d039b41a8a68..90002c38af724 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/NioSocketChannelTests.java @@ -67,10 +67,10 @@ public void testClose() throws IOException, TimeoutException, InterruptedExcepti NioSocketChannel socketChannel = new DoNotCloseChannel(NioChannel.CLIENT, mock(SocketChannel.class), selector); socketChannel.setContexts(mock(ReadContext.class), mock(WriteContext.class)); - socketChannel.getCloseFuture().setListener((c) -> { + socketChannel.getCloseFuture().addListener(ChannelConsumerAdaptor.adapt(socketChannel, (c) -> { ref.set(c); latch.countDown(); - }); + })); CloseFuture closeFuture = socketChannel.getCloseFuture(); assertFalse(closeFuture.isClosed()); From 2280d7bbab0dc2f94c369648d4a4d76ab1448a9d Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 16 Aug 2017 14:23:59 -0500 Subject: [PATCH 14/35] Cleanup close interaction --- .../pipelining/HttpPipelinedResponse.java | 2 +- .../http/nio/NioHttpNettyAdaptor.java | 40 +++++--------- .../http/nio/NioHttpNettyAdaptorTests.java | 53 ++++++++++--------- 3 files changed, 43 insertions(+), 52 deletions(-) diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/pipelining/HttpPipelinedResponse.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/pipelining/HttpPipelinedResponse.java index 6b6db94d69a59..ac0e12f3d46c2 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/pipelining/HttpPipelinedResponse.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/pipelining/HttpPipelinedResponse.java @@ -23,7 +23,7 @@ import io.netty.handler.codec.http.FullHttpResponse; import io.netty.util.ReferenceCounted; -class HttpPipelinedResponse implements Comparable, ReferenceCounted { +public class HttpPipelinedResponse implements Comparable, ReferenceCounted { private final FullHttpResponse response; private final ChannelPromise promise; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java index 9b8788bc5268c..5b587b242cf97 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java @@ -19,42 +19,24 @@ package org.elasticsearch.http.nio; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; -import io.netty.channel.ChannelConfig; -import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; -import io.netty.channel.DefaultChannelConfig; -import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.codec.http.HttpContentCompressor; import io.netty.handler.codec.http.HttpContentDecompressor; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequestDecoder; -import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseEncoder; -import io.netty.util.ReferenceCountUtil; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.FutureListener; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.Tuple; +import org.apache.logging.log4j.Logger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; import org.elasticsearch.http.netty4.pipelining.HttpPipeliningHandler; -import org.elasticsearch.transport.netty4.ByteBufBytesReferenceTests; -import org.elasticsearch.transport.netty4.Netty4Utils; -import org.elasticsearch.transport.nio.channel.ChannelConsumerAdaptor; -import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import java.util.LinkedList; import java.util.function.BiConsumer; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION; @@ -68,6 +50,7 @@ public class NioHttpNettyAdaptor { + private final Logger logger; private final BiConsumer exceptionHandler; private final Netty4CorsConfig corsConfig; private final int maxContentLength; @@ -80,8 +63,9 @@ public class NioHttpNettyAdaptor { private final int maxInitialLineLength; private final int maxCompositeBufferComponents; - protected NioHttpNettyAdaptor(Settings settings, BiConsumer exceptionHandler, + protected NioHttpNettyAdaptor(Logger logger, Settings settings, BiConsumer exceptionHandler, Netty4CorsConfig config, int maxContentLength) { + this.logger = logger; this.exceptionHandler = exceptionHandler; this.maxContentLength = maxContentLength; this.corsConfig = config; @@ -104,6 +88,14 @@ protected ESEmbeddedChannel getAdaptor(NioSocketChannel channel) { final HttpRequestDecoder decoder = new HttpRequestDecoder(maxInitialLineLength, maxHeaderSize, maxChunkSize); decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR); + ch.pipeline().addLast("close_adaptor", new ChannelOutboundHandlerAdapter() { + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + channel.closeAsync().addListener(new ESChannelPromise(promise)); + // Should we forward the close() call with a different promise? This is the last item in + // the outbound pipeline. So it should not be necessary I think. + } + }); ch.pipeline().addLast(decoder); ch.pipeline().addLast(new HttpContentDecompressor()); ch.pipeline().addLast(new HttpResponseEncoder()); @@ -119,7 +111,7 @@ protected ESEmbeddedChannel getAdaptor(NioSocketChannel channel) { ch.pipeline().addLast("cors", new Netty4CorsHandler(corsConfig)); } if (pipelining) { - ch.pipeline().addLast("pipelining", new HttpPipeliningHandler(pipeliningMaxEvents)); + ch.pipeline().addLast("pipelining", new HttpPipeliningHandler(logger, pipeliningMaxEvents)); } ch.pipeline().addLast("read_exception_handler", new ChannelInboundHandlerAdapter() { @Override @@ -127,12 +119,6 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E exceptionHandler.accept(channel, cause); } }); - ch.pipeline().addLast("close_adaptor", new ChannelOutboundHandlerAdapter() { - @Override - public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { - channel.closeAsync().addListener(new ESChannelPromise(promise)); - } - }); return ch; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index e85739fc33aa8..e4c59fa566b03 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -27,6 +27,7 @@ import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpRequestEncoder; @@ -36,24 +37,22 @@ import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.client.http.HttpStatus; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.http.HttpStats; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; +import org.elasticsearch.http.netty4.pipelining.HttpPipelinedResponse; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.netty4.Netty4Utils; -import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.channel.CloseFuture; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.channel.WriteContext; import org.junit.Before; import org.mockito.ArgumentCaptor; import java.io.IOException; +import java.nio.channels.ClosedChannelException; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; @@ -74,7 +73,7 @@ public class NioHttpNettyAdaptorTests extends ESTestCase { public void setMocks() { exceptionHandler = mock(BiConsumer.class); - adaptor = new NioHttpNettyAdaptor(Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + adaptor = new NioHttpNettyAdaptor(logger, Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); nioSocketChannel = mock(NioSocketChannel.class); } @@ -187,25 +186,30 @@ public void testEncodeHttpResponse() { assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); } -// public void testEncodeHttpResponseAfterClose() { -// ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); -// -// prepareAdaptorForResponse(channelAdaptor); -// -// HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); -// -// ChannelFuture close = channelAdaptor.close(); -// -// channelAdaptor.writeOutbound(defaultFullHttpResponse); -// Tuple encodedMessage = channelAdaptor.getMessage(); -// -// HttpResponse response = responseDecoder.decode(Netty4Utils.toByteBuf(encodedMessage.v1())); -// -// assertEquals(HttpResponseStatus.OK, response.status()); -// assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); -// } - - private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { + public void testResponsesAreClearedOnClose() { + adaptor = new NioHttpNettyAdaptor(logger, Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + + prepareAdaptorForResponse(channelAdaptor); + HttpPipelinedRequest pipelinedRequest2 = prepareAdaptorForResponse(channelAdaptor); + + FullHttpResponse httpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + ChannelPromise writePromise = channelAdaptor.newPromise(); + HttpPipelinedResponse pipelinedResponse = pipelinedRequest2.createHttpResponse(httpResponse, writePromise); + + channelAdaptor.write(pipelinedResponse, writePromise); + assertNull(channelAdaptor.getMessage()); + assertFalse(writePromise.isDone()); + + when(nioSocketChannel.closeAsync()).thenReturn(mock(CloseFuture.class)); + ChannelFuture close = channelAdaptor.close(); + + assertFalse(close.isDone()); + assertTrue(writePromise.isDone()); + assertTrue(writePromise.cause() instanceof ClosedChannelException); + } + + private HttpPipelinedRequest prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { HttpMethod method = HttpMethod.GET; HttpVersion version = HttpVersion.HTTP_1_1; String uri = "http://localhost:9090/" + randomAlphaOfLength(8); @@ -220,6 +224,7 @@ private void prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { assertEquals(requestParsed.method(), method); assertEquals(requestParsed.protocolVersion(), version); assertEquals(requestParsed.uri(), uri); + return pipelinedRequest; } private static class RequestEncoder { From 19b3c98bcb2ffc6709e03cc99cb2af3082b0ca63 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 16 Aug 2017 17:21:33 -0500 Subject: [PATCH 15/35] WIP --- .../http/nio/ESEmbeddedChannel.java | 6 + .../http/nio/HttpReadContext.java | 11 +- .../http/nio/HttpReadContextTests.java | 142 ++++++++++++++++++ .../http/nio/NioHttpNettyAdaptorTests.java | 19 +-- .../nio/channel/TcpReadContextTests.java | 1 - 5 files changed, 162 insertions(+), 17 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java index 51692a58dc21b..85844b98f50a4 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java @@ -29,6 +29,7 @@ import org.elasticsearch.transport.netty4.Netty4Utils; import java.util.LinkedList; +import java.util.Queue; public class ESEmbeddedChannel extends EmbeddedChannel { @@ -54,6 +55,11 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) }); } + public Queue decode(ByteBuf inboundBytes) { + writeInbound(inboundBytes); + return inboundMessages(); + } + public Tuple getMessage() { return messages.pollFirst(); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index a6d105a7b5315..bcd55ef993e28 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -30,18 +30,19 @@ import java.io.IOException; import java.util.LinkedList; +import java.util.Queue; public class HttpReadContext implements ReadContext { private final NioSocketChannel channel; - private final EmbeddedChannel nettyPipelineAdaptor; + private final ESEmbeddedChannel nettyPipelineAdaptor; private final LinkedList references = new LinkedList<>(); private final NioHttpRequestHandler requestHandler; - public HttpReadContext(NioSocketChannel channel, NioHttpNettyAdaptor adaptor, NioHttpRequestHandler requestHandler) { + public HttpReadContext(NioSocketChannel channel, ESEmbeddedChannel adaptor, NioHttpRequestHandler requestHandler) { this.channel = channel; this.requestHandler = requestHandler; - this.nettyPipelineAdaptor = adaptor.getAdaptor(channel); + this.nettyPipelineAdaptor = adaptor; } @Override @@ -61,10 +62,10 @@ public int read() throws IOException { ByteBuf inboundBytes = toByteBuf(size); - nettyPipelineAdaptor.writeInbound(inboundBytes); + Queue requests = nettyPipelineAdaptor.decode(inboundBytes); Object msg; - while ((msg = nettyPipelineAdaptor.readInbound()) != null) { + while ((msg = requests.poll()) != null) { requestHandler.handleMessage(channel, nettyPipelineAdaptor, msg); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java new file mode 100644 index 0000000000000..e3d7d1772e3bb --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java @@ -0,0 +1,142 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.buffer.UnpooledHeapByteBuf; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.nio.NetworkBytesReference; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.junit.Before; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class HttpReadContextTests extends ESTestCase { + + private NioHttpRequestHandler handler; + private NioSocketChannel channel; + private ESEmbeddedChannel adaptor; + private int messageLength; + + private HttpReadContext readContext; + + @Before + public void init() throws IOException { + handler = mock(NioHttpRequestHandler.class); + channel = mock(NioSocketChannel.class); + adaptor = mock(ESEmbeddedChannel.class); + messageLength = randomInt(96) + 10; + + + readContext = new HttpReadContext(channel, adaptor, handler); + } + + public void testSuccessfulRequest() throws IOException { + byte[] bytes = createMessage(messageLength); + + final AtomicInteger bufferCapacity = new AtomicInteger(); + when(channel.read(any(NetworkBytesReference.class))).thenAnswer(invocationOnMock -> { + NetworkBytesReference reference = (NetworkBytesReference) invocationOnMock.getArguments()[0]; + ByteBuffer buffer = reference.getWriteByteBuffer(); + bufferCapacity.set(reference.getWriteRemaining()); + buffer.put(bytes); + reference.incrementWrite(bytes.length); + return bytes.length; + }); + + String uri = "localhost:9090/" + randomAlphaOfLength(8); + DefaultFullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); + when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>(Collections.singletonList(request))); + + readContext.read(); + + verify(handler).handleMessage(channel, adaptor, request); + } + + public void testMultipleRequests() throws IOException { + byte[] bytes = createMessage(messageLength); + + final AtomicInteger bufferCapacity = new AtomicInteger(); + when(channel.read(any(NetworkBytesReference.class))).thenAnswer(invocationOnMock -> { + NetworkBytesReference reference = (NetworkBytesReference) invocationOnMock.getArguments()[0]; + ByteBuffer buffer = reference.getWriteByteBuffer(); + bufferCapacity.set(reference.getWriteRemaining()); + buffer.put(bytes); + reference.incrementWrite(bytes.length); + return bytes.length; + }); + + String uri1 = "localhost:9090/" + randomAlphaOfLength(8); + String uri2 = "localhost:9090/" + randomAlphaOfLength(8); + DefaultFullHttpRequest request1 = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri1); + DefaultFullHttpRequest request2 = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri2); + when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>(Arrays.asList(request1, request2))); + + readContext.read(); + + verify(handler).handleMessage(channel, adaptor, request1); + verify(handler).handleMessage(channel, adaptor, request2); + } + + public void testZeroToOneOutboundMessageMeansWriteInterested() throws IOException { + byte[] bytes = createMessage(messageLength); + + final AtomicInteger bufferCapacity = new AtomicInteger(); + when(channel.read(any(NetworkBytesReference.class))).thenAnswer(invocationOnMock -> { + NetworkBytesReference reference = (NetworkBytesReference) invocationOnMock.getArguments()[0]; + ByteBuffer buffer = reference.getWriteByteBuffer(); + bufferCapacity.set(reference.getWriteRemaining()); + buffer.put(bytes); + reference.incrementWrite(bytes.length); + return bytes.length; + }); + + when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>(Arrays.asList())); + + readContext.read(); + + // TODO: Implement + } + + private static byte[] createMessage(int length) { + byte[] bytes = new byte[length]; + for (int i = 0; i < length; ++i) { + bytes[i] = randomByte(); + } + return bytes; + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index e4c59fa566b03..ac3d62ab6dd4a 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -53,6 +53,7 @@ import java.io.IOException; import java.nio.channels.ClosedChannelException; +import java.util.Queue; import java.util.function.BiConsumer; import static org.mockito.Mockito.mock; @@ -114,12 +115,12 @@ public void testSuccessfulDecodeHttpRequest() { int slicePoint = randomInt(buf.writerIndex() - 1); ByteBuf slicedBuf = buf.retainedSlice(0, slicePoint); - channelAdaptor.writeInbound(slicedBuf); + Queue messages = channelAdaptor.decode(slicedBuf); - assertNull(channelAdaptor.readInbound()); + assertTrue(messages.isEmpty()); - channelAdaptor.writeInbound(buf.retainedSlice(slicePoint, buf.writerIndex() - slicePoint)); - HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); + messages = channelAdaptor.decode(buf.retainedSlice(slicePoint, buf.writerIndex() - slicePoint)); + HttpPipelinedRequest decodedRequest = (HttpPipelinedRequest) messages.poll(); FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); assertEquals(httpRequest.protocolVersion(), fullHttpRequest.protocolVersion()); @@ -137,8 +138,7 @@ public void testDecodeHttpRequestError() { buf.setByte(1, ' '); buf.setByte(2, ' '); - channelAdaptor.writeInbound(buf.retainedDuplicate()); - HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); + HttpPipelinedRequest decodedRequest = (HttpPipelinedRequest) channelAdaptor.decode(buf.retainedDuplicate()).poll(); FullHttpRequest fullHttpRequest = (FullHttpRequest) decodedRequest.last(); DecoderResult decoderResult = fullHttpRequest.decoderResult(); @@ -157,9 +157,7 @@ public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { channelAdaptor.writeInbound(buf.retainedDuplicate()); - HttpPipelinedRequest decodedRequest = channelAdaptor.readInbound(); - - assertNull(decodedRequest); + assertTrue(channelAdaptor.decode(buf.retainedDuplicate()).isEmpty()); Tuple message = channelAdaptor.getMessage(); @@ -217,8 +215,7 @@ private HttpPipelinedRequest prepareAdaptorForResponse(ESEmbeddedChannel adaptor HttpRequest request = new DefaultFullHttpRequest(version, method, uri); ByteBuf buf = requestEncoder.encode(request); - adaptor.writeInbound(buf); - HttpPipelinedRequest pipelinedRequest = adaptor.readInbound(); + HttpPipelinedRequest pipelinedRequest = (HttpPipelinedRequest) adaptor.decode(buf).poll(); FullHttpRequest requestParsed = (FullHttpRequest) pipelinedRequest.last(); assertNotNull(requestParsed); assertEquals(requestParsed.method(), method); diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java index fc8d7e48ab0bf..b7fd47aa09b09 100644 --- a/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java +++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/channel/TcpReadContextTests.java @@ -146,5 +146,4 @@ private static byte[] createMessage(int length) { } return bytes; } - } From bca982a94b2fafbe9de9ef72703e46436b8bb15a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 16 Aug 2017 22:57:29 -0500 Subject: [PATCH 16/35] WIP --- .../http/nio/HttpReadContext.java | 6 +++++ .../http/nio/HttpReadContextTests.java | 23 +++++++++++++++---- .../transport/nio/utils/TestSelectionKey.java | 0 3 files changed, 25 insertions(+), 4 deletions(-) rename test/framework/src/{test => main}/java/org/elasticsearch/transport/nio/utils/TestSelectionKey.java (100%) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index bcd55ef993e28..e7281655e9ec7 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -27,6 +27,7 @@ import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; import org.elasticsearch.transport.nio.channel.ReadContext; +import org.elasticsearch.transport.nio.channel.SelectionKeyUtils; import java.io.IOException; import java.util.LinkedList; @@ -58,6 +59,7 @@ public int read() throws IOException { return bytesRead; } + boolean noPendingPriorToDecode = !nettyPipelineAdaptor.hasMessages(); int size = references.size(); ByteBuf inboundBytes = toByteBuf(size); @@ -69,6 +71,10 @@ public int read() throws IOException { requestHandler.handleMessage(channel, nettyPipelineAdaptor, msg); } + if (noPendingPriorToDecode && nettyPipelineAdaptor.hasMessages()) { + SelectionKeyUtils.setWriteInterested(channel); + } + return bytesRead; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java index e3d7d1772e3bb..f97a57fd1c49f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java @@ -21,20 +21,23 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import io.netty.buffer.UnpooledByteBufAllocator; -import io.netty.buffer.UnpooledHeapByteBuf; +import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.HttpMethod; -import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.channel.SelectionKeyUtils; +import org.elasticsearch.transport.nio.utils.TestSelectionKey; import org.junit.Before; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; import java.util.Arrays; import java.util.Collections; import java.util.LinkedList; @@ -63,6 +66,8 @@ public void init() throws IOException { readContext = new HttpReadContext(channel, adaptor, handler); + + when(channel.getSelectionKey()).thenReturn(new TestSelectionKey(0)); } public void testSuccessfulRequest() throws IOException { @@ -125,10 +130,20 @@ public void testZeroToOneOutboundMessageMeansWriteInterested() throws IOExceptio return bytes.length; }); - when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>(Arrays.asList())); + when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>(Collections.emptyList())); + when(adaptor.hasMessages()).thenReturn(false, true); + + assertFalse((channel.getSelectionKey().interestOps() & SelectionKey.OP_WRITE) != 0); readContext.read(); + assertTrue((channel.getSelectionKey().interestOps() & SelectionKey.OP_WRITE) != 0); + } + + public void testReadIndexesIncremented() { + NetworkBytesReference ref = new NetworkBytesReference(new BytesArray(new byte[10]), 10, 5); + ByteBuf byteBuf = Unpooled.wrappedBuffer(ref.getReadByteBuffer()); + // TODO: Implement } diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/utils/TestSelectionKey.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/utils/TestSelectionKey.java similarity index 100% rename from test/framework/src/test/java/org/elasticsearch/transport/nio/utils/TestSelectionKey.java rename to test/framework/src/main/java/org/elasticsearch/transport/nio/utils/TestSelectionKey.java From 683abbc71db584a648a624d381d4049876f5f971 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 10:17:34 -0500 Subject: [PATCH 17/35] Read context tests --- .../http/nio/HttpReadContext.java | 12 ++++--- .../http/nio/HttpReadContextTests.java | 36 ++++++++++++++++--- 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index e7281655e9ec7..87283f3365c67 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -30,6 +30,7 @@ import org.elasticsearch.transport.nio.channel.SelectionKeyUtils; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.LinkedList; import java.util.Queue; @@ -60,11 +61,11 @@ public int read() throws IOException { } boolean noPendingPriorToDecode = !nettyPipelineAdaptor.hasMessages(); - int size = references.size(); - ByteBuf inboundBytes = toByteBuf(size); + ByteBuf inboundBytes = toByteBuf(references); Queue requests = nettyPipelineAdaptor.decode(inboundBytes); + NetworkBytesReference.vectorizedIncrementReadIndexes(references, inboundBytes.readableBytes()); Object msg; while ((msg = requests.poll()) != null) { @@ -78,14 +79,17 @@ public int read() throws IOException { return bytesRead; } - private ByteBuf toByteBuf(int size) { + private static ByteBuf toByteBuf(LinkedList references) { + int size = references.size(); if (size == 1) { return Unpooled.wrappedBuffer(references.getFirst().getReadByteBuffer()); } else { CompositeByteBuf byteBuf = Unpooled.compositeBuffer(size); for (NetworkBytesReference reference : references) { // TODO: Do I need to increment reader indexes? - byteBuf.addComponent(true, Unpooled.wrappedBuffer(reference.getReadByteBuffer())); + ByteBuffer buffer = reference.getReadByteBuffer(); + ByteBuf component = Unpooled.wrappedBuffer(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + byteBuf.addComponent(true, component); } return byteBuf; } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java index f97a57fd1c49f..0eda8303b7e7f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java @@ -41,10 +41,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.LinkedList; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -140,11 +142,37 @@ public void testZeroToOneOutboundMessageMeansWriteInterested() throws IOExceptio assertTrue((channel.getSelectionKey().interestOps() & SelectionKey.OP_WRITE) != 0); } - public void testReadIndexesIncremented() { - NetworkBytesReference ref = new NetworkBytesReference(new BytesArray(new byte[10]), 10, 5); - ByteBuf byteBuf = Unpooled.wrappedBuffer(ref.getReadByteBuffer()); + public void testMultipleReadsForRequest() throws IOException { + byte[] bytes = createMessage(messageLength); + byte[] bytes2 = createMessage(messageLength + randomInt(100)); + + final AtomicInteger bufferCapacity = new AtomicInteger(); + final AtomicBoolean isFirst = new AtomicBoolean(true); + when(channel.read(any(NetworkBytesReference.class))).thenAnswer(invocationOnMock -> { + NetworkBytesReference reference = (NetworkBytesReference) invocationOnMock.getArguments()[0]; + ByteBuffer buffer = reference.getWriteByteBuffer(); + bufferCapacity.set(reference.getWriteRemaining()); + int length; + if (isFirst.compareAndSet(true, false)) { + length = bytes.length; + buffer.put(bytes); + } else { + length = bytes2.length; + buffer.put(bytes2); + } + reference.incrementWrite(length); + return length; + }); + + String uri = "localhost:9090/" + randomAlphaOfLength(8); + DefaultFullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); + when(adaptor.decode(Unpooled.wrappedBuffer(bytes))).thenReturn(new LinkedList<>()); + when(adaptor.decode(Unpooled.wrappedBuffer(bytes2))).thenReturn(new LinkedList<>(Collections.singletonList(request))); + + readContext.read(); + readContext.read(); - // TODO: Implement + verify(handler, times(1)).handleMessage(channel, adaptor, request); } private static byte[] createMessage(int length) { From bbe84d86ac5eb7a23f469b353c05cbd0550e61a8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 11:00:44 -0500 Subject: [PATCH 18/35] Implement read indexes --- .../java/org/elasticsearch/http/nio/HttpReadContext.java | 7 ++++--- .../org/elasticsearch/http/nio/HttpReadContextTests.java | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index 87283f3365c67..9c09b32fd1efe 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -60,19 +60,20 @@ public int read() throws IOException { return bytesRead; } - boolean noPendingPriorToDecode = !nettyPipelineAdaptor.hasMessages(); + boolean noPendingWritesPriorToDecode = !nettyPipelineAdaptor.hasMessages(); ByteBuf inboundBytes = toByteBuf(references); + int readDelta = inboundBytes.readableBytes(); Queue requests = nettyPipelineAdaptor.decode(inboundBytes); - NetworkBytesReference.vectorizedIncrementReadIndexes(references, inboundBytes.readableBytes()); + NetworkBytesReference.vectorizedIncrementReadIndexes(references, readDelta); Object msg; while ((msg = requests.poll()) != null) { requestHandler.handleMessage(channel, nettyPipelineAdaptor, msg); } - if (noPendingPriorToDecode && nettyPipelineAdaptor.hasMessages()) { + if (noPendingWritesPriorToDecode && nettyPipelineAdaptor.hasMessages()) { SelectionKeyUtils.setWriteInterested(channel); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java index 0eda8303b7e7f..96fdc7e436788 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java @@ -69,7 +69,7 @@ public void init() throws IOException { readContext = new HttpReadContext(channel, adaptor, handler); - when(channel.getSelectionKey()).thenReturn(new TestSelectionKey(0)); + when(channel.getSelectionKey()).thenReturn(new TestSelectionKey(SelectionKey.OP_READ)); } public void testSuccessfulRequest() throws IOException { From 575ef118fcfc558510c2e6d9d206e021a9c75e56 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 11:01:06 -0500 Subject: [PATCH 19/35] Remove comment --- .../test/java/org/elasticsearch/http/nio/HttpReadContext.java | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java index 9c09b32fd1efe..42380a8592070 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -87,7 +87,6 @@ private static ByteBuf toByteBuf(LinkedList references) { } else { CompositeByteBuf byteBuf = Unpooled.compositeBuffer(size); for (NetworkBytesReference reference : references) { - // TODO: Do I need to increment reader indexes? ByteBuffer buffer = reference.getReadByteBuffer(); ByteBuf component = Unpooled.wrappedBuffer(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); byteBuf.addComponent(true, component); From 74ff853106cd25589cdc1da52d92e0284c2e3445 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 14:33:41 -0500 Subject: [PATCH 20/35] Implement write context tests --- .../http/nio/ESEmbeddedChannel.java | 7 +- .../http/nio/HttpWriteContext.java | 25 +- .../http/nio/HttpWriteContextTests.java | 240 ++++++++++++++++++ .../http/nio/NioHttpNettyAdaptorTests.java | 6 +- .../transport/nio/channel/WriteContext.java | 10 +- 5 files changed, 264 insertions(+), 24 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java index 85844b98f50a4..a430051984c31 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java @@ -36,7 +36,6 @@ public class ESEmbeddedChannel extends EmbeddedChannel { private LinkedList> messages = new LinkedList<>(); public ESEmbeddedChannel() { - super(); pipeline().addFirst("promise_captor", new ChannelOutboundHandlerAdapter() { @Override @@ -60,7 +59,7 @@ public Queue decode(ByteBuf inboundBytes) { return inboundMessages(); } - public Tuple getMessage() { + public Tuple popMessage() { return messages.pollFirst(); } @@ -68,7 +67,7 @@ public boolean hasMessages() { return messages.size() > 0; } - public LinkedList> getMessages() { - return messages; + public void closeNettyChannel() { + close(); } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java index 7597006e4d928..4426b6d0010ec 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -36,7 +36,6 @@ import java.io.IOException; import java.nio.channels.ClosedChannelException; import java.util.LinkedList; -import java.util.Queue; public class HttpWriteContext implements WriteContext { @@ -89,11 +88,9 @@ public void flushChannel() throws IOException { } } - LinkedList> messages = adaptor.getMessages(); - Tuple message; - boolean lastMessageFullyFlushed = true; - while ((message = messages.pop()) != null && lastMessageFullyFlushed) { + boolean previousMessageFullyFlushed = true; + while (previousMessageFullyFlushed && (message = adaptor.popMessage()) != null) { ChannelPromise promise = message.v2(); ESChannelPromise listener; if (promise instanceof ESChannelPromise) { @@ -105,7 +102,7 @@ public void flushChannel() throws IOException { if (WriteContext.flushOperation(channel, writeOperation) == false) { partiallyFlushed = writeOperation; - lastMessageFullyFlushed = false; + previousMessageFullyFlushed = false; } } } @@ -113,20 +110,24 @@ public void flushChannel() throws IOException { @Override public boolean hasQueuedWriteOps() { assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; - return adaptor.hasMessages(); + return partiallyFlushed != null || adaptor.hasMessages(); } @Override public void clearQueuedWriteOps(Exception e) { assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to access queued writes"; - for (Tuple message : adaptor.getMessages()) { + + // Right now there is an assumption that all resources will be released by the promise completion + if (partiallyFlushed != null) { + partiallyFlushed.getListener().onFailure(e); + } + + Tuple message; + while ((message = adaptor.popMessage()) != null) { message.v2().setFailure(e); BytesReference bytes = message.v1(); - if (bytes instanceof Releasable) { - Releasables.close((Releasable) bytes); - } } - adaptor.close().syncUninterruptibly(); + adaptor.closeNettyChannel(); } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java new file mode 100644 index 0000000000000..8c049ad27237f --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java @@ -0,0 +1,240 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.CompositeBytesReference; +import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.nio.NetworkBytesReference; +import org.elasticsearch.transport.nio.SocketSelector; +import org.elasticsearch.transport.nio.WriteOperation; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.junit.Before; +import org.mockito.ArgumentCaptor; + +import java.io.IOException; +import java.nio.channels.ClosedChannelException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@SuppressWarnings("unchecked") +public class HttpWriteContextTests extends ESTestCase { + + private ESEmbeddedChannel adaptor; + private NioSocketChannel channel; + private SocketSelector selector; + private ESChannelPromise listener; + + private HttpWriteContext writeContext; + + @Before + public void initMocks() { + adaptor = mock(ESEmbeddedChannel.class); + channel = mock(NioSocketChannel.class); + selector = mock(SocketSelector.class); + listener = mock(ESChannelPromise.class); + + writeContext = new HttpWriteContext(channel, adaptor); + + when(channel.isWritable()).thenReturn(true); + when(channel.getSelector()).thenReturn(selector); + when(selector.isOnCurrentThread()).thenReturn(true); + } + + public void testCannotSendIfChannelNotWritable() { + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + when(channel.isWritable()).thenReturn(false); + + writeContext.sendMessage(response, listener); + + verify(listener).onFailure(any(ClosedChannelException.class)); + } + + public void testSendFromDifferentThread() { + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + ArgumentCaptor captor = ArgumentCaptor.forClass(WriteOperation.class); + + when(selector.isOnCurrentThread()).thenReturn(false); + writeContext.sendMessage(response, listener); + + verify(selector).queueWrite(captor.capture()); + + HttpWriteOperation httpWriteOperation = (HttpWriteOperation) captor.getValue(); + assertSame(response, httpWriteOperation.getHttpResponse()); + assertSame(listener, httpWriteOperation.getListener()); + } + + public void testSendFromSelectorThread() { + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + ArgumentCaptor captor = ArgumentCaptor.forClass(WriteOperation.class); + + writeContext.sendMessage(response, listener); + + verify(selector).queueWriteInChannelBuffer(captor.capture()); + + HttpWriteOperation httpWriteOperation = (HttpWriteOperation) captor.getValue(); + assertSame(response, httpWriteOperation.getHttpResponse()); + assertSame(listener, httpWriteOperation.getListener()); + } + + public void testQueueWriteOperationIsDelegatedToNettyAdaptor() { + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + + writeContext.queueWriteOperations(new HttpWriteOperation(channel, response, listener)); + + verify(adaptor).write(response, listener); + } + + public void testSingleFlush() throws IOException { + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + + List messages = new ArrayList<>(); + when(channel.write(any())).thenAnswer(invocationOnMock -> { + NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; + if (references.length > 1) { + throw new IOException("Only expected 1 reference"); + } + int readRemaining = references[0].getReadRemaining(); + messages.add(references[0].utf8ToString()); + references[0].incrementRead(readRemaining); + return readRemaining; + }); + + writeContext.flushChannel(); + + verify(listener).onResponse(channel); + assertEquals("message", messages.get(0)); + } + + public void testMultiFlush() throws IOException { + ESChannelPromise listener2 = mock(ESChannelPromise.class); + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message1"), listener), + new Tuple<>(new BytesArray("message2"), listener2), null); + + List messages = new ArrayList<>(); + when(channel.write(any())).thenAnswer(invocationOnMock -> { + NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; + if (references.length > 1) { + throw new IOException("Only expected 1 reference"); + } + int readRemaining = references[0].getReadRemaining(); + messages.add(references[0].utf8ToString()); + references[0].incrementRead(readRemaining); + return readRemaining; + }); + + writeContext.flushChannel(); + + verify(listener).onResponse(channel); + verify(listener2).onResponse(channel); + assertEquals("message1", messages.get(0)); + assertEquals("message2", messages.get(1)); + } + + public void testPartialFlush() throws IOException { + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + + List messages = new ArrayList<>(); + AtomicBoolean firstCall = new AtomicBoolean(true); + when(channel.write(any())).thenAnswer(invocationOnMock -> { + NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; + if (references.length > 1) { + throw new IOException("Only expected 1 reference"); + } + int read; + if (firstCall.compareAndSet(true, false)) { + messages.add(references[0].slice(0, 1)); + references[0].incrementRead(1); + read = 1; + } else { + int readRemaining = references[0].getReadRemaining(); + messages.add(references[0].slice(1, readRemaining)); + references[0].incrementRead(readRemaining); + read = readRemaining; + } + return read; + }); + + writeContext.flushChannel(); + verify(listener, times(0)).onResponse(channel); + + writeContext.flushChannel(); + verify(listener, times(1)).onResponse(channel); + assertEquals("message", new CompositeBytesReference(messages.toArray(new BytesReference[0])).utf8ToString()); + } + + public void testHasQueuedReferencesAdaptor() throws IOException { + when(adaptor.hasMessages()).thenReturn(true); + + assertTrue(writeContext.hasQueuedWriteOps()); + } + + public void testHasQueuedWillConsiderPartialFlush() throws IOException { + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + when(channel.write(any())).thenAnswer(invocationOnMock -> { + NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; + if (references.length > 1) { + throw new IOException("Only expected 1 reference"); + } + references[0].incrementRead(1); + return 1; + }); + + writeContext.flushChannel(); + + when(adaptor.hasMessages()).thenReturn(false); + + assertTrue(writeContext.hasQueuedWriteOps()); + } + + public void testAdaptorMessagesAndPartialMessagesAreClosed() throws IOException { + ESChannelPromise listener2 = mock(ESChannelPromise.class); + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message1"), listener), + new Tuple<>(new BytesArray("message2"), listener2), null); + when(channel.write(any())).thenAnswer(invocationOnMock -> { + NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; + references[0].incrementRead(1); + return 1; + }); + + writeContext.flushChannel(); + + ClosedChannelException closedChannelException = new ClosedChannelException(); + + writeContext.clearQueuedWriteOps(closedChannelException); + + verify(listener).onFailure(closedChannelException); + verify(listener2).setFailure(closedChannelException); + verify(adaptor).closeNettyChannel(); + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index ac3d62ab6dd4a..2975ce7df9ae9 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -159,7 +159,7 @@ public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { assertTrue(channelAdaptor.decode(buf.retainedDuplicate()).isEmpty()); - Tuple message = channelAdaptor.getMessage(); + Tuple message = channelAdaptor.popMessage(); assertFalse(message.v2().isDone()); @@ -176,7 +176,7 @@ public void testEncodeHttpResponse() { HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); channelAdaptor.writeOutbound(defaultFullHttpResponse); - Tuple encodedMessage = channelAdaptor.getMessage(); + Tuple encodedMessage = channelAdaptor.popMessage(); HttpResponse response = responseDecoder.decode(Netty4Utils.toByteBuf(encodedMessage.v1())); @@ -196,7 +196,7 @@ public void testResponsesAreClearedOnClose() { HttpPipelinedResponse pipelinedResponse = pipelinedRequest2.createHttpResponse(httpResponse, writePromise); channelAdaptor.write(pipelinedResponse, writePromise); - assertNull(channelAdaptor.getMessage()); + assertNull(channelAdaptor.popMessage()); assertFalse(writePromise.isDone()); when(nioSocketChannel.closeAsync()).thenReturn(mock(CloseFuture.class)); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java index 29d26133927bc..312aafe0f5e98 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/WriteContext.java @@ -38,16 +38,16 @@ public interface WriteContext { void clearQueuedWriteOps(Exception e); - static boolean flushOperation(NioSocketChannel channel, ByteWriteOperation headOp) throws IOException { + static boolean flushOperation(NioSocketChannel channel, ByteWriteOperation op) throws IOException { try { - headOp.flush(); + op.flush(); } catch (IOException e) { - headOp.getListener().onFailure(e); + op.getListener().onFailure(e); throw e; } - if (headOp.isFullyFlushed()) { - headOp.getListener().onResponse(channel); + if (op.isFullyFlushed()) { + op.getListener().onResponse(channel); return true; } else { return false; From 5e7d770f95bc83b653b8496422340aece5dda0fc Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 14:53:25 -0500 Subject: [PATCH 21/35] Release netty byte buffers --- .../netty4/ByteBufBytesReference.java | 4 +-- .../http/nio/ESEmbeddedChannel.java | 4 ++- .../http/nio/NioHttpNettyAdaptorTests.java | 28 +++++++++++++++++++ 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/ByteBufBytesReference.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/ByteBufBytesReference.java index 3b4aba1028119..85a6644bc224d 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/ByteBufBytesReference.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/ByteBufBytesReference.java @@ -27,7 +27,7 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; -final class ByteBufBytesReference extends BytesReference { +public final class ByteBufBytesReference extends BytesReference { private final ByteBuf buffer; private final int length; @@ -65,7 +65,7 @@ public void writeTo(OutputStream os) throws IOException { buffer.getBytes(offset, os, length); } - ByteBuf toByteBuf() { + public ByteBuf toByteBuf() { return buffer.duplicate(); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java index a430051984c31..a21aef4c013f7 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java @@ -45,7 +45,9 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) // intercept the promise and pass a different promise back to the rest of the pipeline. try { - BytesReference bytesReference = Netty4Utils.toBytesReference((ByteBuf) msg); + ByteBuf message = (ByteBuf) msg; + BytesReference bytesReference = Netty4Utils.toBytesReference(message); + promise.addListener((f) -> message.release()); messages.add(new Tuple<>(bytesReference, promise)); } catch (Exception e) { promise.setFailure(e); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java index 2975ce7df9ae9..69576ffdc4029 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java @@ -44,6 +44,7 @@ import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedResponse; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.netty4.ByteBufBytesReference; import org.elasticsearch.transport.netty4.Netty4Utils; import org.elasticsearch.transport.nio.channel.CloseFuture; import org.elasticsearch.transport.nio.channel.NioChannel; @@ -184,6 +185,33 @@ public void testEncodeHttpResponse() { assertEquals(HttpVersion.HTTP_1_1, response.protocolVersion()); } + public void testEncodedMessageIsReleasedWhenPromiseCompleted() { + ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + + prepareAdaptorForResponse(channelAdaptor); + + HttpResponse defaultFullHttpResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + + channelAdaptor.writeOutbound(defaultFullHttpResponse); + Tuple encodedMessage = channelAdaptor.popMessage(); + + ByteBufBytesReference reference = (ByteBufBytesReference) encodedMessage.v1(); + + ByteBuf byteBuf = reference.toByteBuf(); + assertEquals(1, byteBuf.refCnt()); + byteBuf.retain(); + assertEquals(2, byteBuf.refCnt()); + + if (randomBoolean()) { + encodedMessage.v2().setSuccess(); + } else { + encodedMessage.v2().setFailure(new ClosedChannelException()); + } + + assertEquals(1, byteBuf.refCnt()); + assertTrue(byteBuf.release()); + } + public void testResponsesAreClearedOnClose() { adaptor = new NioHttpNettyAdaptor(logger, Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); From bc54ef8afa123a21ca1a3b73cd93e02b9599fe3d Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 16:46:59 -0500 Subject: [PATCH 22/35] Implement some integration tests --- .../netty4/Netty4HttpServerTransport.java | 36 +---- .../http/netty4/cors/Netty4CorsConfig.java | 45 ++++++ .../org/elasticsearch/ESNioIntegTestCase.java | 72 ++++++++++ .../http/netty4/Netty4HttpClient.java | 6 +- .../Netty4HttpServerTransportTests.java | 4 +- .../http/nio/HttpWriteContext.java | 3 +- .../http/nio/HttpWriteContextTests.java | 10 +- .../http/nio/HttpWriteOperation.java | 7 +- .../http/nio/NioHttpChannel.java | 20 +-- .../elasticsearch/http/nio/NioHttpPlugin.java | 90 ++++++++++++ .../http/nio/NioHttpRequestHandler.java | 4 +- .../http/nio/NioHttpRequestSizeLimitIT.java | 131 ++++++++++++++++++ .../http/nio/NioHttpTransport.java | 41 ++++-- .../http/nio/NioPipeliningDisabledIT.java | 78 +++++++++++ .../http/nio/NioPipeliningEnabledIT.java | 76 ++++++++++ test/framework/build.gradle | 7 - 16 files changed, 547 insertions(+), 83 deletions(-) create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java create mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java 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 a9f42f564b396..6cafe8ab7fc66 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 @@ -249,7 +249,7 @@ public Netty4HttpServerTransport(Settings settings, NetworkService networkServic this.compressionLevel = SETTING_HTTP_COMPRESSION_LEVEL.get(settings); this.pipelining = SETTING_PIPELINING.get(settings); this.pipeliningMaxEvents = SETTING_PIPELINING_MAX_EVENTS.get(settings); - this.corsConfig = buildCorsConfig(settings); + this.corsConfig = Netty4CorsConfig.buildCorsConfig(settings); // validate max content length if (maxContentLength.getBytes() > Integer.MAX_VALUE) { @@ -373,40 +373,6 @@ static int resolvePublishPort(Settings settings, List boundAdd return publishPort; } - // package private for testing - static Netty4CorsConfig buildCorsConfig(Settings settings) { - if (SETTING_CORS_ENABLED.get(settings) == false) { - return Netty4CorsConfigBuilder.forOrigins().disable().build(); - } - String origin = SETTING_CORS_ALLOW_ORIGIN.get(settings); - final Netty4CorsConfigBuilder builder; - if (Strings.isNullOrEmpty(origin)) { - builder = Netty4CorsConfigBuilder.forOrigins(); - } else if (origin.equals(ANY_ORIGIN)) { - builder = Netty4CorsConfigBuilder.forAnyOrigin(); - } else { - Pattern p = RestUtils.checkCorsSettingForRegex(origin); - if (p == null) { - builder = Netty4CorsConfigBuilder.forOrigins(RestUtils.corsSettingAsArray(origin)); - } else { - builder = Netty4CorsConfigBuilder.forPattern(p); - } - } - if (SETTING_CORS_ALLOW_CREDENTIALS.get(settings)) { - builder.allowCredentials(); - } - String[] strMethods = Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_METHODS.get(settings), ","); - HttpMethod[] methods = Arrays.asList(strMethods) - .stream() - .map(HttpMethod::valueOf) - .toArray(size -> new HttpMethod[size]); - return builder.allowedRequestMethods(methods) - .maxAge(SETTING_CORS_MAX_AGE.get(settings)) - .allowedRequestHeaders(Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_HEADERS.get(settings), ",")) - .shortCircuit() - .build(); - } - private TransportAddress bindAddress(final InetAddress hostAddress) { final AtomicReference lastException = new AtomicReference<>(); final AtomicReference boundSocket = new AtomicReference<>(); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java index 9c81c07e66314..a5dbd8319a939 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/cors/Netty4CorsConfig.java @@ -23,7 +23,11 @@ import io.netty.handler.codec.http.EmptyHttpHeaders; import io.netty.handler.codec.http.HttpHeaders; import io.netty.handler.codec.http.HttpMethod; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.RestUtils; +import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashSet; import java.util.Map; @@ -32,6 +36,14 @@ import java.util.concurrent.Callable; import java.util.regex.Pattern; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_MAX_AGE; +import static org.elasticsearch.http.netty4.cors.Netty4CorsHandler.ANY_ORIGIN; + /** * Configuration for Cross-Origin Resource Sharing (CORS). * @@ -232,4 +244,37 @@ public String toString() { ", preflightHeaders=" + preflightHeaders + ']'; } + public static Netty4CorsConfig buildCorsConfig(Settings settings) { + if (SETTING_CORS_ENABLED.get(settings) == false) { + return Netty4CorsConfigBuilder.forOrigins().disable().build(); + } + String origin = SETTING_CORS_ALLOW_ORIGIN.get(settings); + final Netty4CorsConfigBuilder builder; + if (Strings.isNullOrEmpty(origin)) { + builder = Netty4CorsConfigBuilder.forOrigins(); + } else if (origin.equals(ANY_ORIGIN)) { + builder = Netty4CorsConfigBuilder.forAnyOrigin(); + } else { + Pattern p = RestUtils.checkCorsSettingForRegex(origin); + if (p == null) { + builder = Netty4CorsConfigBuilder.forOrigins(RestUtils.corsSettingAsArray(origin)); + } else { + builder = Netty4CorsConfigBuilder.forPattern(p); + } + } + if (SETTING_CORS_ALLOW_CREDENTIALS.get(settings)) { + builder.allowCredentials(); + } + String[] strMethods = Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_METHODS.get(settings), ","); + HttpMethod[] methods = Arrays.asList(strMethods) + .stream() + .map(HttpMethod::valueOf) + .toArray(size -> new HttpMethod[size]); + return builder.allowedRequestMethods(methods) + .maxAge(SETTING_CORS_MAX_AGE.get(settings)) + .allowedRequestHeaders(Strings.tokenizeToStringArray(SETTING_CORS_ALLOW_HEADERS.get(settings), ",")) + .shortCircuit() + .build(); + } + } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java b/modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java new file mode 100644 index 0000000000000..8821cb5746860 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch; + +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.nio.NioHttpPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.nio.NioTransport; + +import java.util.Arrays; +import java.util.Collection; + +public abstract class ESNioIntegTestCase extends ESIntegTestCase { + + @Override + protected boolean ignoreExternalCluster() { + return true; + } + + @Override + protected boolean addMockTransportService() { + return false; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal)); + // randomize netty settings + if (randomBoolean()) { + builder.put(NioTransport.NIO_WORKER_COUNT.getKey(), random().nextInt(3) + 1); + } + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioHttpPlugin.NIO_TRANSPORT_NAME); + builder.put(NetworkModule.HTTP_TYPE_KEY, NioHttpPlugin.NIO_HTTP_TRANSPORT_NAME); + return builder.build(); + } + + @Override + protected Settings transportClientSettings() { + Settings.Builder builder = Settings.builder().put(super.transportClientSettings()); + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioHttpPlugin.NIO_TRANSPORT_NAME); + return builder.build(); + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(NioHttpPlugin.class); + } + + @Override + protected Collection> transportClientPlugins() { + return Arrays.asList(NioHttpPlugin.class); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java index 7c4471e249102..55b6eefe66cb8 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java @@ -61,7 +61,7 @@ /** * Tiny helper to send http requests over netty. */ -class Netty4HttpClient implements Closeable { +public class Netty4HttpClient implements Closeable { static Collection returnHttpResponseBodies(Collection responses) { List list = new ArrayList<>(responses.size()); @@ -71,7 +71,7 @@ static Collection returnHttpResponseBodies(Collection return list; } - static Collection returnOpaqueIds(Collection responses) { + public static Collection returnOpaqueIds(Collection responses) { List list = new ArrayList<>(responses.size()); for (HttpResponse response : responses) { list.add(response.headers().get("X-Opaque-Id")); @@ -81,7 +81,7 @@ static Collection returnOpaqueIds(Collection responses private final Bootstrap clientBootstrap; - Netty4HttpClient() { + public Netty4HttpClient() { clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).group(new NioEventLoopGroup()); } 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 846c59565c245..ecf097dfbce14 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 @@ -117,7 +117,7 @@ public void testCorsConfig() { .put(SETTING_CORS_ALLOW_HEADERS.getKey(), collectionToDelimitedString(headers, ",", prefix, "")) .put(SETTING_CORS_ALLOW_CREDENTIALS.getKey(), true) .build(); - final Netty4CorsConfig corsConfig = Netty4HttpServerTransport.buildCorsConfig(settings); + final Netty4CorsConfig corsConfig = Netty4CorsConfig.buildCorsConfig(settings); assertTrue(corsConfig.isAnyOriginSupported()); assertEquals(headers, corsConfig.allowedRequestHeaders()); assertEquals(methods, corsConfig.allowedRequestMethods().stream().map(HttpMethod::name).collect(Collectors.toSet())); @@ -128,7 +128,7 @@ public void testCorsConfigWithDefaults() { final Set headers = Strings.commaDelimitedListToSet(SETTING_CORS_ALLOW_HEADERS.getDefault(Settings.EMPTY)); final long maxAge = SETTING_CORS_MAX_AGE.getDefault(Settings.EMPTY); final Settings settings = Settings.builder().put(SETTING_CORS_ENABLED.getKey(), true).build(); - final Netty4CorsConfig corsConfig = Netty4HttpServerTransport.buildCorsConfig(settings); + final Netty4CorsConfig corsConfig = Netty4CorsConfig.buildCorsConfig(settings); assertFalse(corsConfig.isAnyOriginSupported()); assertEquals(Collections.emptySet(), corsConfig.origins().get()); assertEquals(headers, corsConfig.allowedRequestHeaders()); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java index 4426b6d0010ec..8bd9e8794d2d1 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java @@ -54,9 +54,8 @@ public void sendMessage(Object message, ActionListener listener) { listener.onFailure(new ClosedChannelException()); return; } - HttpResponse response = (HttpResponse) message; - WriteOperation writeOperation = new HttpWriteOperation(channel, response, listener); + WriteOperation writeOperation = new HttpWriteOperation(channel, message, listener); SocketSelector selector = channel.getSelector(); if (selector.isOnCurrentThread() == false) { selector.queueWrite(writeOperation); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java index 8c049ad27237f..d676329d9698c 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.http.nio; +import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.DefaultHttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; @@ -116,7 +117,8 @@ public void testQueueWriteOperationIsDelegatedToNettyAdaptor() { } public void testSingleFlush() throws IOException { - when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), + (Tuple) null); List messages = new ArrayList<>(); when(channel.write(any())).thenAnswer(invocationOnMock -> { @@ -162,7 +164,8 @@ public void testMultiFlush() throws IOException { } public void testPartialFlush() throws IOException { - when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), + (Tuple) null); List messages = new ArrayList<>(); AtomicBoolean firstCall = new AtomicBoolean(true); @@ -200,7 +203,8 @@ public void testHasQueuedReferencesAdaptor() throws IOException { } public void testHasQueuedWillConsiderPartialFlush() throws IOException { - when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), null); + when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message"), listener), + (Tuple) null); when(channel.write(any())).thenAnswer(invocationOnMock -> { NetworkBytesReference[] references = (NetworkBytesReference[]) invocationOnMock.getArguments()[0]; if (references.length > 1) { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java index c93656066ab6f..084a1351f844e 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java @@ -19,7 +19,6 @@ package org.elasticsearch.http.nio; -import io.netty.handler.codec.http.HttpResponse; import org.elasticsearch.action.ActionListener; import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioChannel; @@ -27,14 +26,14 @@ public class HttpWriteOperation extends WriteOperation { - private final HttpResponse httpResponse; + private final Object httpResponse; - public HttpWriteOperation(NioSocketChannel channel, HttpResponse response, ActionListener listener) { + public HttpWriteOperation(NioSocketChannel channel, Object response, ActionListener listener) { super(channel, listener); this.httpResponse = response; } - public HttpResponse getHttpResponse() { + public Object getHttpResponse() { return httpResponse; } } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java index 8bfbec56ca73c..3fae7cd3ce93f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java @@ -72,14 +72,14 @@ public final class NioHttpChannel extends AbstractRestChannel { /** * @param request The request that is handled by this channel. - * @param channel + * @param channel The channel * @param pipelinedRequest If HTTP pipelining is enabled provide the corresponding pipelined request. May be null if * HTTP pipelining is disabled. * @param detailedErrorsEnabled true iff error messages should include stack traces. * @param threadContext the thread context for the channel */ - public NioHttpChannel(Netty4HttpRequest request, NioSocketChannel channel, HttpPipelinedRequest pipelinedRequest, boolean detailedErrorsEnabled, - ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { + public NioHttpChannel(Netty4HttpRequest request, NioSocketChannel channel, HttpPipelinedRequest pipelinedRequest, + boolean detailedErrorsEnabled, ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { super(request, detailedErrorsEnabled); this.channel = request.getChannel(); this.nettyRequest = request.request(); @@ -131,7 +131,7 @@ public void sendResponse(RestResponse response) { addCookies(resp); - final ChannelPromise promise = channel.newPromise(); + final ESChannelPromise promise = new ESChannelPromise(channel.newPromise()); if (releaseContent) { promise.addListener(f -> ((Releasable) content).close()); @@ -151,17 +151,7 @@ public void sendResponse(RestResponse response) { } else { msg = resp; } - nioChannel.getWriteContext().sendMessage(msg, new ActionListener() { - @Override - public void onResponse(NioChannel nioChannel) { - promise.setSuccess(); - } - - @Override - public void onFailure(Exception e) { - promise.setFailure(e); - } - }); + nioChannel.getWriteContext().sendMessage(msg, promise); releaseContent = false; releaseBytesStreamOutput = false; } finally { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java new file mode 100644 index 0000000000000..652709241e869 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java @@ -0,0 +1,90 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.network.NetworkService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.plugins.NetworkPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.nio.NioTransport; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +public class NioHttpPlugin extends Plugin implements NetworkPlugin { + + public static final String NIO_TRANSPORT_NAME = "nio"; + public static final String NIO_HTTP_TRANSPORT_NAME = "nio"; + + @Override + public List> getSettings() { + return Arrays.asList( + NioHttpTransport.NIO_HTTP_WORKER_COUNT, + NioHttpTransport.NIO_HTTP_ACCEPTOR_COUNT, + NioHttpTransport.SETTING_HTTP_TCP_NO_DELAY, + NioHttpTransport.SETTING_HTTP_TCP_KEEP_ALIVE, + NioHttpTransport.SETTING_HTTP_TCP_REUSE_ADDRESS, + NioHttpTransport.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, + NioHttpTransport.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, + NioTransport.NIO_WORKER_COUNT, + NioTransport.NIO_ACCEPTOR_COUNT + ); + } + + @Override + public Settings additionalSettings() { + return Settings.builder() + .put(NetworkModule.HTTP_DEFAULT_TYPE_SETTING.getKey(), NIO_HTTP_TRANSPORT_NAME) + .put(NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING.getKey(), NIO_TRANSPORT_NAME) + .build(); + } + + @Override + public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, + CircuitBreakerService circuitBreakerService, + NamedWriteableRegistry namedWriteableRegistry, + NetworkService networkService) { + return Collections.singletonMap(NIO_TRANSPORT_NAME, () -> new NioTransport(settings, threadPool, networkService, bigArrays, + namedWriteableRegistry, circuitBreakerService)); + } + + @Override + public Map> getHttpTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, + CircuitBreakerService circuitBreakerService, + NamedWriteableRegistry namedWriteableRegistry, + NamedXContentRegistry xContentRegistry, + NetworkService networkService, + HttpServerTransport.Dispatcher dispatcher) { + return Collections.singletonMap(NIO_HTTP_TRANSPORT_NAME, + () -> new NioHttpTransport(settings, networkService, bigArrays, threadPool, xContentRegistry, dispatcher)); + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java index bd6f0beea8db9..02708bba453a5 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java @@ -72,8 +72,8 @@ public void handleMessage(NioSocketChannel channel, Channel nettyChannel, Object request.headers(), request.trailingHeaders()); final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, nettyChannel); - final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, channel, pipelinedRequest, detailedErrorsEnabled, threadContext, corsConfig, - resetCookies); + final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, channel, pipelinedRequest, detailedErrorsEnabled, threadContext, + corsConfig, resetCookies); if (request.decoderResult().isSuccess()) { transport.dispatchRequest(httpRequest, httpChannel); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java new file mode 100644 index 0000000000000..8f9280655a9ae --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java @@ -0,0 +1,131 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import org.elasticsearch.ESNetty4IntegTestCase; +import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; +import org.elasticsearch.test.ESIntegTestCase.ClusterScope; +import org.elasticsearch.test.ESIntegTestCase.Scope; + +import java.util.Collection; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; + +/** + * This test checks that in-flight requests are limited on HTTP level and that requests that are excluded from limiting can pass. + * + * As the same setting is also used to limit in-flight requests on transport level, we avoid transport messages by forcing + * a single node "cluster". We also force test infrastructure to use the node client instead of the transport client for the same reason. + */ +@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numClientNodes = 0, numDataNodes = 1, transportClientRatio = 0) +public class NioHttpRequestSizeLimitIT extends ESNioIntegTestCase { + + private static final ByteSizeValue LIMIT = new ByteSizeValue(2, ByteSizeUnit.KB); + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(NetworkModule.HTTP_ENABLED.getKey(), true) + .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), LIMIT) + .build(); + } + + public void testLimitsInFlightRequests() throws Exception { + ensureGreen(); + + // we use the limit size as a (very) rough indication on how many requests we should sent to hit the limit + int numRequests = LIMIT.bytesAsInt() / 100; + + StringBuilder bulkRequest = new StringBuilder(); + for (int i = 0; i < numRequests; i++) { + bulkRequest.append("{\"index\": {}}"); + bulkRequest.append(System.lineSeparator()); + bulkRequest.append("{ \"field\" : \"value\" }"); + bulkRequest.append(System.lineSeparator()); + } + + @SuppressWarnings("unchecked") + Tuple[] requests = new Tuple[150]; + for (int i = 0; i < requests.length; i++) { + requests[i] = Tuple.tuple("/index/type/_bulk", bulkRequest); + } + + HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress + ().boundAddresses()); + + try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + Collection singleResponse = nettyHttpClient.post(transportAddress.address(), requests[0]); + assertThat(singleResponse, hasSize(1)); + assertAtLeastOnceExpectedStatus(singleResponse, HttpResponseStatus.OK); + + Collection multipleResponses = nettyHttpClient.post(transportAddress.address(), requests); + assertThat(multipleResponses, hasSize(requests.length)); + assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.SERVICE_UNAVAILABLE); + } + } + + public void testDoesNotLimitExcludedRequests() throws Exception { + ensureGreen(); + + @SuppressWarnings("unchecked") + Tuple[] requestUris = new Tuple[1500]; + for (int i = 0; i < requestUris.length; i++) { + requestUris[i] = Tuple.tuple("/_cluster/settings", + "{ \"transient\": {\"search.default_search_timeout\": \"40s\" } }"); + } + + HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress + ().boundAddresses()); + + try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + Collection responses = nettyHttpClient.put(transportAddress.address(), requestUris); + assertThat(responses, hasSize(requestUris.length)); + assertAllInExpectedStatus(responses, HttpResponseStatus.OK); + } + } + + private void assertAtLeastOnceExpectedStatus(Collection responses, HttpResponseStatus expectedStatus) { + long countExpectedStatus = responses.stream().filter(r -> r.status().equals(expectedStatus)).count(); + assertThat("Expected at least one request with status [" + expectedStatus + "]", countExpectedStatus, greaterThan(0L)); + } + + private void assertAllInExpectedStatus(Collection responses, HttpResponseStatus expectedStatus) { + long countUnexpectedStatus = responses.stream().filter(r -> r.status().equals(expectedStatus) == false).count(); + assertThat("Expected all requests with status [" + expectedStatus + "] but [" + countUnexpectedStatus + + "] requests had a different one", countUnexpectedStatus, equalTo(0L)); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java index a4162e32b7279..b822dd4ddaafd 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java @@ -44,6 +44,8 @@ import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpStats; import org.elasticsearch.http.netty4.Netty4HttpRequest; +import org.elasticsearch.http.netty4.Netty4HttpServerTransport; +import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BindTransportException; @@ -71,10 +73,13 @@ import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.common.settings.Setting.intSetting; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_BIND_HOST; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PORT; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX; import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_WORKER_THREAD_NAME_PREFIX; @@ -97,6 +102,7 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); + private final Netty4CorsConfig corsConfig; private final PortsRange port; private final NetworkService networkService; private final BigArrays bigArrays; @@ -105,18 +111,22 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http private final Dispatcher dispatcher; private final String[] publishHosts; private final String bindHosts[]; - private final Consumer contextSetter; - private final ByteSizeValue maxContentLength; - private OpenChannels openChannels; - private ArrayList socketSelectors; - private ArrayList acceptors; - private BoundTransportAddress boundAddress; + private final int maxContentLength; private final boolean tcpNoDelay; private final boolean tcpKeepAlive; private final boolean tcpReuseAddress; private final int tcpSendBufferSize; private final int tcpReceiveBufferSize; + private final boolean resetCookies; + private final boolean detailedErrorsEnabled; + private final boolean pipelining; + private int acceptorNumber; + private OpenChannels openChannels; + private ArrayList socketSelectors; + private ArrayList acceptors; + private BoundTransportAddress boundAddress; + private volatile Consumer contextSetter; public NioHttpTransport(Settings settings, NetworkService networkService, BigArrays bigArrays, ThreadPool threadPool, NamedXContentRegistry xContentRegistry, Dispatcher dispatcher) { @@ -134,9 +144,12 @@ public NioHttpTransport(Settings settings, NetworkService networkService, BigArr List httpPublishHost = SETTING_HTTP_PUBLISH_HOST.get(settings); this.publishHosts = (httpPublishHost.isEmpty() ? NetworkService.GLOBAL_NETWORK_PUBLISHHOST_SETTING.get(settings) : httpPublishHost) .toArray(Strings.EMPTY_ARRAY); - maxContentLength = SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings); + maxContentLength = Math.toIntExact(SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings).getBytes()); + detailedErrorsEnabled = SETTING_HTTP_DETAILED_ERRORS_ENABLED.get(settings); + pipelining = SETTING_PIPELINING.get(settings); + resetCookies = SETTING_HTTP_RESET_COOKIES.get(settings); + corsConfig = Netty4CorsConfig.buildCorsConfig(settings); - contextSetter = (c) -> c.setContexts(new HttpReadContext(c, null, null), new TcpWriteContext(c)); tcpNoDelay = SETTING_HTTP_TCP_NO_DELAY.get(settings); tcpKeepAlive = SETTING_HTTP_TCP_KEEP_ALIVE.get(settings); tcpReuseAddress = SETTING_HTTP_TCP_REUSE_ADDRESS.get(settings); @@ -148,6 +161,15 @@ public NioHttpTransport(Settings settings, NetworkService networkService, BigArr protected void doStart() { boolean success = false; try { + NioHttpNettyAdaptor nettyAdaptor = new NioHttpNettyAdaptor(logger, settings, this::exceptionCaught, corsConfig, + maxContentLength); + NioHttpRequestHandler handler = new NioHttpRequestHandler(this, xContentRegistry, detailedErrorsEnabled, + threadPool.getThreadContext(), pipelining, corsConfig, resetCookies); + contextSetter = (c) -> { + ESEmbeddedChannel adaptor = nettyAdaptor.getAdaptor(c); + c.setContexts(new HttpReadContext(c, adaptor, handler), new HttpWriteContext(c, adaptor)); + }; + this.openChannels = new OpenChannels(logger); socketSelectors = NioSelectors.socketSelectors(settings, () -> new SocketEventHandler(logger, this::exceptionCaught), @@ -189,7 +211,7 @@ public HttpInfo info() { if (boundTransportAddress == null) { return null; } - return new HttpInfo(boundTransportAddress, maxContentLength.getBytes()); + return new HttpInfo(boundTransportAddress, maxContentLength); } @Override @@ -313,7 +335,6 @@ private TransportAddress bindAddress(final InetAddress hostAddress) { tcpReuseAddress, tcpSendBufferSize, tcpReceiveBufferSize); ChannelFactory channelFactory = new ChannelFactory(rawChannelFactory, contextSetter); NioServerSocketChannel serverChannel = channelFactory.openNioServerSocketChannel("http-server", address, selector); - selector.scheduleForRegistration(serverChannel); boundSocket.set(serverChannel.getLocalAddress()); } } catch (Exception e) { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java new file mode 100644 index 0000000000000..cfe660d4583cf --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java @@ -0,0 +1,78 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.http.nio; + +import io.netty.handler.codec.http.FullHttpResponse; +import org.elasticsearch.ESNetty4IntegTestCase; +import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.test.ESIntegTestCase.ClusterScope; +import org.elasticsearch.test.ESIntegTestCase.Scope; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Locale; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasSize; + +@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numDataNodes = 1) +public class NioPipeliningDisabledIT extends ESNioIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(NetworkModule.HTTP_ENABLED.getKey(), true) + .put("http.pipelining", false) + .build(); + } + + public void testThatNettyHttpServerDoesNotSupportPipelining() throws Exception { + ensureGreen(); + String[] requests = new String[] {"/", "/_nodes/stats", "/", "/_cluster/state", "/", "/_nodes", "/"}; + + HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); + TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); + + try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + Collection responses = nettyHttpClient.get(transportAddress.address(), requests); + assertThat(responses, hasSize(requests.length)); + + List opaqueIds = new ArrayList<>(Netty4HttpClient.returnOpaqueIds(responses)); + + assertResponsesOutOfOrder(opaqueIds); + } + } + + /** + * checks if all responses are there, but also tests that they are out of order because pipelining is disabled + */ + private void assertResponsesOutOfOrder(List opaqueIds) { + String message = String.format(Locale.ROOT, "Expected returned http message ids to be in any order of: %s", opaqueIds); + assertThat(message, opaqueIds, containsInAnyOrder("0", "1", "2", "3", "4", "5", "6")); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java new file mode 100644 index 0000000000000..c364dc04e8c20 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java @@ -0,0 +1,76 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http.nio; + +import io.netty.handler.codec.http.FullHttpResponse; +import org.elasticsearch.ESNetty4IntegTestCase; +import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.test.ESIntegTestCase.ClusterScope; +import org.elasticsearch.test.ESIntegTestCase.Scope; + +import java.util.Collection; +import java.util.Locale; + +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; + +@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numDataNodes = 1) +public class NioPipeliningEnabledIT extends ESNioIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(NetworkModule.HTTP_ENABLED.getKey(), true) + .put("http.pipelining", true) + .build(); + } + + public void testThatNettyHttpServerSupportsPipelining() throws Exception { + String[] requests = new String[]{"/", "/_nodes/stats", "/", "/_cluster/state", "/"}; + + HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); + TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); + + try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + Collection responses = nettyHttpClient.get(transportAddress.address(), requests); + assertThat(responses, hasSize(5)); + + Collection opaqueIds = Netty4HttpClient.returnOpaqueIds(responses); + assertOpaqueIdsInOrder(opaqueIds); + } + } + + private void assertOpaqueIdsInOrder(Collection opaqueIds) { + // check if opaque ids are monotonically increasing + int i = 0; + String msg = String.format(Locale.ROOT, "Expected list of opaque ids to be monotonically increasing, got [%s]", opaqueIds); + for (String opaqueId : opaqueIds) { + assertThat(msg, opaqueId, is(String.valueOf(i++))); + } + } + +} diff --git a/test/framework/build.gradle b/test/framework/build.gradle index 6704464b2ec74..09382763057c9 100644 --- a/test/framework/build.gradle +++ b/test/framework/build.gradle @@ -31,13 +31,6 @@ dependencies { compile "commons-codec:commons-codec:${versions.commonscodec}" compile "org.elasticsearch:securemock:${versions.securemock}" compile "org.elasticsearch:mocksocket:${versions.mocksocket}" - compile "io.netty:netty-buffer:4.1.13.Final" - compile "io.netty:netty-codec:4.1.13.Final" - compile "io.netty:netty-codec-http:4.1.13.Final" - compile "io.netty:netty-common:4.1.13.Final" - compile "io.netty:netty-handler:4.1.13.Final" - compile "io.netty:netty-resolver:4.1.13.Final" - compile "io.netty:netty-transport:4.1.13.Final" } compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes,-try,-unchecked' From e9baacfd57506c5fd3bf01d42f36fd9816963e29 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 17 Aug 2017 18:58:07 -0500 Subject: [PATCH 23/35] WIP --- plugins/transport-nio/build.gradle | 29 +++ .../config/discovery-ec2/log4j2.properties | 8 + .../http/nio/ESChannelPromise.java | 0 .../http/nio/ESEmbeddedChannel.java | 0 .../http/nio/HttpReadContext.java | 2 +- .../http/nio/HttpWriteContext.java | 0 .../http/nio/HttpWriteOperation.java | 0 .../http/nio/NioHttpChannel.java | 0 .../http/nio/NioHttpNettyAdaptor.java | 0 .../elasticsearch/http/nio/NioHttpPlugin.java | 0 .../http/nio/NioHttpRequestHandler.java | 0 .../http/nio/NioHttpTransport.java | 4 +- .../plugin-metadata/plugin-security.policy | 21 ++ .../org/elasticsearch/ESNioIntegTestCase.java | 0 .../elasticsearch/http/Netty4HttpClient.java | 200 ++++++++++++++++++ .../http/nio/HttpReadContextTests.java | 6 - .../http/nio/HttpWriteContextTests.java | 0 .../http/nio/NioHttpNettyAdaptorTests.java | 0 .../http/nio/NioHttpRequestSizeLimitIT.java | 3 +- .../http/nio/NioPipeliningDisabledIT.java | 3 +- .../http/nio/NioPipeliningEnabledIT.java | 3 +- settings.gradle | 1 + 22 files changed, 265 insertions(+), 15 deletions(-) create mode 100644 plugins/transport-nio/build.gradle create mode 100644 plugins/transport-nio/config/discovery-ec2/log4j2.properties rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/ESChannelPromise.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/HttpReadContext.java (98%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/HttpWriteContext.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/HttpWriteOperation.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/NioHttpChannel.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/NioHttpPlugin.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java (100%) rename {modules/transport-netty4/src/test => plugins/transport-nio/src/main}/java/org/elasticsearch/http/nio/NioHttpTransport.java (98%) create mode 100644 plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/ESNioIntegTestCase.java (100%) create mode 100644 plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java (96%) rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java (100%) rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java (100%) rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java (98%) rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java (96%) rename {modules/transport-netty4 => plugins/transport-nio}/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java (96%) diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle new file mode 100644 index 0000000000000..5a48cd5fb9c0f --- /dev/null +++ b/plugins/transport-nio/build.gradle @@ -0,0 +1,29 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +esplugin { + description 'The nio transport.' + classname 'org.elasticsearch.http.nio.NioHttpPlugin' +} + + +dependencies { + compile project(path: ':modules:transport-netty4', configuration: 'runtime') + compile project(path: ':test:framework', configuration: 'runtime') +} diff --git a/plugins/transport-nio/config/discovery-ec2/log4j2.properties b/plugins/transport-nio/config/discovery-ec2/log4j2.properties new file mode 100644 index 0000000000000..aa52f0232e009 --- /dev/null +++ b/plugins/transport-nio/config/discovery-ec2/log4j2.properties @@ -0,0 +1,8 @@ +logger.com_amazonaws.name = com.amazonaws +logger.com_amazonaws.level = warn + +logger.com_amazonaws_jmx_SdkMBeanRegistrySupport.name = com.amazonaws.jmx.SdkMBeanRegistrySupport +logger.com_amazonaws_jmx_SdkMBeanRegistrySupport.level = error + +logger.com_amazonaws_metrics_AwsSdkMetrics.name = com.amazonaws.metrics.AwsSdkMetrics +logger.com_amazonaws_metrics_AwsSdkMetrics.level = error diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESChannelPromise.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESChannelPromise.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESChannelPromise.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java similarity index 98% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java index 42380a8592070..b20316361e7d2 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java @@ -51,7 +51,7 @@ public HttpReadContext(NioSocketChannel channel, ESEmbeddedChannel adaptor, NioH public int read() throws IOException { NetworkBytesReference last = references.peekLast(); if (last == null || last.hasWriteRemaining() == false) { - this.references.add(NetworkBytesReference.wrap(new BytesArray(new byte[DEFAULT_READ_LENGTH]))); + this.references.add(NetworkBytesReference.wrap(new BytesArray(new byte[ReadContext.DEFAULT_READ_LENGTH]))); } int bytesRead = channel.read(references.getLast()); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteContext.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContext.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteContext.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteOperation.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpChannel.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpPlugin.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpPlugin.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpPlugin.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java similarity index 98% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java index b822dd4ddaafd..40b982f386bc9 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java @@ -173,10 +173,10 @@ protected void doStart() { this.openChannels = new OpenChannels(logger); socketSelectors = NioSelectors.socketSelectors(settings, () -> new SocketEventHandler(logger, this::exceptionCaught), - NIO_HTTP_WORKER_COUNT.get(settings), TRANSPORT_WORKER_THREAD_NAME_PREFIX); + NIO_HTTP_WORKER_COUNT.get(settings), NioTransport.TRANSPORT_WORKER_THREAD_NAME_PREFIX); acceptors = NioSelectors.acceptingSelectors(logger, settings, openChannels, socketSelectors, - NIO_HTTP_ACCEPTOR_COUNT.get(settings), TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); + NIO_HTTP_ACCEPTOR_COUNT.get(settings), NioTransport.TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX); this.boundAddress = createBoundHttpAddress(); if (logger.isInfoEnabled()) { diff --git a/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 0000000000000..9b21daca38111 --- /dev/null +++ b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy @@ -0,0 +1,21 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +grant { +}; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java b/plugins/transport-nio/src/test/java/org/elasticsearch/ESNioIntegTestCase.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/ESNioIntegTestCase.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/ESNioIntegTestCase.java diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java new file mode 100644 index 0000000000000..922f9f5a27b77 --- /dev/null +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java @@ -0,0 +1,200 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.http; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelFuture; +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.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpMethod; +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.HttpRequestEncoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseDecoder; +import io.netty.handler.codec.http.HttpVersion; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; + +import java.io.Closeable; +import java.net.SocketAddress; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static io.netty.handler.codec.http.HttpHeaderNames.HOST; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; + +/** + * Tiny helper to send http requests over netty. + */ +public class Netty4HttpClient implements Closeable { + + static Collection returnHttpResponseBodies(Collection responses) { + List list = new ArrayList<>(responses.size()); + for (FullHttpResponse response : responses) { + list.add(response.content().toString(StandardCharsets.UTF_8)); + } + return list; + } + + public static Collection returnOpaqueIds(Collection responses) { + List list = new ArrayList<>(responses.size()); + for (HttpResponse response : responses) { + list.add(response.headers().get("X-Opaque-Id")); + } + return list; + } + + private final Bootstrap clientBootstrap; + + public Netty4HttpClient() { + clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).group(new NioEventLoopGroup()); + } + + public Collection get(SocketAddress remoteAddress, String... uris) throws InterruptedException { + Collection requests = new ArrayList<>(uris.length); + for (int i = 0; i < uris.length; i++) { + final HttpRequest httpRequest = new DefaultFullHttpRequest(HTTP_1_1, HttpMethod.GET, uris[i]); + httpRequest.headers().add(HOST, "localhost"); + httpRequest.headers().add("X-Opaque-ID", String.valueOf(i)); + requests.add(httpRequest); + } + return sendRequests(remoteAddress, requests); + } + + @SafeVarargs // Safe not because it doesn't do anything with the type parameters but because it won't leak them into other methods. + public final Collection post(SocketAddress remoteAddress, Tuple... urisAndBodies) + throws InterruptedException { + return processRequestsWithBody(HttpMethod.POST, remoteAddress, urisAndBodies); + } + + public final FullHttpResponse post(SocketAddress remoteAddress, FullHttpRequest httpRequest) throws InterruptedException { + Collection responses = sendRequests(remoteAddress, Collections.singleton(httpRequest)); + assert responses.size() == 1 : "expected 1 and only 1 http response"; + return responses.iterator().next(); + } + + @SafeVarargs // Safe not because it doesn't do anything with the type parameters but because it won't leak them into other methods. + public final Collection put(SocketAddress remoteAddress, Tuple... urisAndBodies) + throws InterruptedException { + return processRequestsWithBody(HttpMethod.PUT, remoteAddress, urisAndBodies); + } + + private Collection processRequestsWithBody(HttpMethod method, SocketAddress remoteAddress, Tuple... urisAndBodies) throws InterruptedException { + Collection requests = new ArrayList<>(urisAndBodies.length); + for (Tuple uriAndBody : urisAndBodies) { + ByteBuf content = Unpooled.copiedBuffer(uriAndBody.v2(), StandardCharsets.UTF_8); + HttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, method, uriAndBody.v1(), content); + request.headers().add(HttpHeaderNames.HOST, "localhost"); + request.headers().add(HttpHeaderNames.CONTENT_LENGTH, content.readableBytes()); + request.headers().add(HttpHeaderNames.CONTENT_TYPE, "application/json"); + requests.add(request); + } + return sendRequests(remoteAddress, requests); + } + + private synchronized Collection sendRequests( + final SocketAddress remoteAddress, + final Collection requests) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(requests.size()); + final Collection content = Collections.synchronizedList(new ArrayList<>(requests.size())); + + clientBootstrap.handler(new CountDownLatchHandler(latch, content)); + + ChannelFuture channelFuture = null; + try { + channelFuture = clientBootstrap.connect(remoteAddress); + channelFuture.sync(); + + for (HttpRequest request : requests) { + channelFuture.channel().writeAndFlush(request); + } + latch.await(10, TimeUnit.SECONDS); + + } finally { + if (channelFuture != null) { + channelFuture.channel().close().sync(); + } + } + + return content; + } + + @Override + public void close() { + clientBootstrap.config().group().shutdownGracefully().awaitUninterruptibly(); + } + + /** + * helper factory which adds returned data to a list and uses a count down latch to decide when done + */ + private static class CountDownLatchHandler extends ChannelInitializer { + + private final CountDownLatch latch; + private final Collection content; + + CountDownLatchHandler(final CountDownLatch latch, final Collection content) { + this.latch = latch; + this.content = content; + } + + @Override + protected void initChannel(SocketChannel ch) throws Exception { + final int maxContentLength = new ByteSizeValue(100, ByteSizeUnit.MB).bytesAsInt(); + ch.pipeline().addLast(new HttpResponseDecoder()); + ch.pipeline().addLast(new HttpRequestEncoder()); + ch.pipeline().addLast(new HttpObjectAggregator(maxContentLength)); + ch.pipeline().addLast(new SimpleChannelInboundHandler() { + @Override + protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception { + final FullHttpResponse response = (FullHttpResponse) msg; + content.add(response.copy()); + latch.countDown(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + super.exceptionCaught(ctx, cause); + latch.countDown(); + } + }); + } + + } + +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java similarity index 96% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java index 96fdc7e436788..ba119f08d9de0 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java @@ -19,19 +19,13 @@ package org.elasticsearch.http.nio; -import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpVersion; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.channel.SelectionKeyUtils; import org.elasticsearch.transport.nio.utils.TestSelectionKey; import org.junit.Before; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java similarity index 100% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java similarity index 98% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java index 8f9280655a9ae..ef5949462e2f5 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java @@ -21,7 +21,6 @@ import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; -import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.ESNioIntegTestCase; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.network.NetworkModule; @@ -30,7 +29,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java similarity index 96% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java index cfe660d4583cf..6c26fab1f54f5 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java @@ -19,13 +19,12 @@ package org.elasticsearch.http.nio; import io.netty.handler.codec.http.FullHttpResponse; -import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.ESNioIntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java similarity index 96% rename from modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java index c364dc04e8c20..e79d5eddde825 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java @@ -20,13 +20,12 @@ package org.elasticsearch.http.nio; import io.netty.handler.codec.http.FullHttpResponse; -import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.ESNioIntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.netty4.Netty4HttpClient; +import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/settings.gradle b/settings.gradle index e34e55eb3fd61..9c2e59a8e388a 100644 --- a/settings.gradle +++ b/settings.gradle @@ -60,6 +60,7 @@ List projects = [ 'plugins:repository-gcs', 'plugins:repository-hdfs', 'plugins:repository-s3', + 'plugins:transport-nio', 'plugins:jvm-example', 'plugins:store-smb', 'qa:auto-create-index', From ce2a7b9238b97118f15e8fe993953d3dd55ab875 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 10:18:12 -0500 Subject: [PATCH 24/35] WIP --- plugins/transport-nio/build.gradle | 2 +- .../nio/NioPlugin.java} | 10 +++++----- .../nio/http}/ESChannelPromise.java | 2 +- .../nio/http}/ESEmbeddedChannel.java | 2 +- .../nio/http}/HttpReadContext.java | 3 +-- .../nio/http}/HttpWriteContext.java | 6 +----- .../nio/http}/HttpWriteOperation.java | 2 +- .../nio => transport/nio/http}/NioHttpChannel.java | 6 +----- .../nio/http}/NioHttpNettyAdaptor.java | 2 +- .../nio/http}/NioHttpRequestHandler.java | 2 +- .../nio/http}/NioHttpTransport.java | 6 +----- .../main/plugin-metadata/plugin-security.policy | 13 ++++++++++++- .../{ => transport/nio}/ESNioIntegTestCase.java | 14 +++++++------- .../nio/http}/HttpReadContextTests.java | 5 ++++- .../nio/http}/HttpWriteContextTests.java | 7 +++++-- .../{ => transport/nio}/http/Netty4HttpClient.java | 2 +- .../nio/http}/NioHttpNettyAdaptorTests.java | 4 +++- .../nio/http}/NioHttpRequestSizeLimitIT.java | 5 ++--- .../nio/http}/NioPipeliningDisabledIT.java | 5 ++--- .../nio/http}/NioPipeliningEnabledIT.java | 5 ++--- .../transport/nio/NioTransportPlugin.java | 2 +- 21 files changed, 54 insertions(+), 51 deletions(-) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio/NioHttpPlugin.java => transport/nio/NioPlugin.java} (92%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/ESChannelPromise.java (99%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/ESEmbeddedChannel.java (98%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/HttpReadContext.java (97%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/HttpWriteContext.java (95%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/HttpWriteOperation.java (96%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpChannel.java (98%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpNettyAdaptor.java (99%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpRequestHandler.java (98%) rename plugins/transport-nio/src/main/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpTransport.java (97%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{ => transport/nio}/ESNioIntegTestCase.java (82%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/HttpReadContextTests.java (96%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/HttpWriteContextTests.java (97%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{ => transport/nio}/http/Netty4HttpClient.java (99%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpNettyAdaptorTests.java (98%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/NioHttpRequestSizeLimitIT.java (97%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/NioPipeliningDisabledIT.java (96%) rename plugins/transport-nio/src/test/java/org/elasticsearch/{http/nio => transport/nio/http}/NioPipeliningEnabledIT.java (95%) diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 5a48cd5fb9c0f..2db9b1e3f5aa6 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -19,7 +19,7 @@ esplugin { description 'The nio transport.' - classname 'org.elasticsearch.http.nio.NioHttpPlugin' + classname 'org.elasticsearch.http.nio.NioPlugin' } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java similarity index 92% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpPlugin.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java index 652709241e869..f6c45b61f4865 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpPlugin.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkModule; @@ -27,12 +27,12 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.transport.nio.http.NioHttpTransport; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.nio.NioTransport; import java.util.Arrays; import java.util.Collections; @@ -40,10 +40,10 @@ import java.util.Map; import java.util.function.Supplier; -public class NioHttpPlugin extends Plugin implements NetworkPlugin { +public class NioPlugin extends Plugin implements NetworkPlugin { - public static final String NIO_TRANSPORT_NAME = "nio"; - public static final String NIO_HTTP_TRANSPORT_NAME = "nio"; + public static final String NIO_TRANSPORT_NAME = "nio-transport"; + public static final String NIO_HTTP_TRANSPORT_NAME = "nio-http-transport"; @Override public List> getSettings() { diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESChannelPromise.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java similarity index 99% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESChannelPromise.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java index 527c30583203f..9f9d6bfa4bf2f 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESChannelPromise.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.channel.Channel; import io.netty.channel.ChannelPromise; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java similarity index 98% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java index a21aef4c013f7..462bb51e67af2 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/ESEmbeddedChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java similarity index 97% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java index b20316361e7d2..6a5bc2e7d4445 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadContext.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java @@ -17,12 +17,11 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import io.netty.channel.embedded.EmbeddedChannel; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java similarity index 95% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteContext.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java index 8bd9e8794d2d1..cb8b748dad40a 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteContext.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java @@ -17,15 +17,12 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.http.HttpResponse; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.transport.nio.ByteWriteOperation; import org.elasticsearch.transport.nio.SocketSelector; import org.elasticsearch.transport.nio.WriteOperation; @@ -35,7 +32,6 @@ import java.io.IOException; import java.nio.channels.ClosedChannelException; -import java.util.LinkedList; public class HttpWriteContext implements WriteContext { diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteOperation.java similarity index 96% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteOperation.java index 084a1351f844e..24ff58e71b4af 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteOperation.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import org.elasticsearch.action.ActionListener; import org.elasticsearch.transport.nio.WriteOperation; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java similarity index 98% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java index 3fae7cd3ce93f..04cea79b8fa38 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java @@ -17,13 +17,12 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.FullHttpResponse; @@ -35,7 +34,6 @@ import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http.cookie.ServerCookieDecoder; import io.netty.handler.codec.http.cookie.ServerCookieEncoder; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; @@ -50,10 +48,8 @@ import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.transport.netty4.Netty4Utils; -import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; import java.util.List; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java similarity index 99% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java index 5b587b242cf97..1474780ea93f4 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpNettyAdaptor.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.ChannelHandlerContext; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java similarity index 98% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java index 02708bba453a5..371e9fb38fd2d 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpRequestHandler.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java similarity index 97% rename from plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java index 40b982f386bc9..f2a81e2e72ae3 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.IntSet; @@ -44,7 +44,6 @@ import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpStats; import org.elasticsearch.http.netty4.Netty4HttpRequest; -import org.elasticsearch.http.netty4.Netty4HttpServerTransport; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.threadpool.ThreadPool; @@ -59,7 +58,6 @@ import org.elasticsearch.transport.nio.channel.ChannelFactory; import org.elasticsearch.transport.nio.channel.NioServerSocketChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.channel.TcpWriteContext; import java.io.IOException; import java.net.InetAddress; @@ -80,8 +78,6 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; -import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_ACCEPTOR_THREAD_NAME_PREFIX; -import static org.elasticsearch.transport.nio.NioTransport.TRANSPORT_WORKER_THREAD_NAME_PREFIX; public class NioHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport { diff --git a/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy index 9b21daca38111..e3253c76ef433 100644 --- a/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy +++ b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy @@ -17,5 +17,16 @@ * under the License. */ -grant { +grant codeBase "${codebase.netty-common-4.1.13.Final.jar}" { + // for reading the system-wide configuration for the backlog of established sockets + permission java.io.FilePermission "/proc/sys/net/core/somaxconn", "read"; + + // netty makes and accepts socket connections + permission java.net.SocketPermission "*", "accept,connect"; }; + +grant codeBase "${codebase.netty-transport-4.1.13.Final.jar}" { + // Netty NioEventLoop wants to change this, because of https://bugs.openjdk.java.net/browse/JDK-6427854 + // the bug says it only happened rarely, and that its fixed, but apparently it still happens rarely! + permission java.util.PropertyPermission "sun.nio.ch.bugLevel", "write"; +}; \ No newline at end of file diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/ESNioIntegTestCase.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java similarity index 82% rename from plugins/transport-nio/src/test/java/org/elasticsearch/ESNioIntegTestCase.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java index 8821cb5746860..e3ba68f228bce 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/ESNioIntegTestCase.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java @@ -16,11 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch; +package org.elasticsearch.transport.nio; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.http.nio.NioHttpPlugin; +import org.elasticsearch.transport.nio.NioPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.transport.nio.NioTransport; @@ -47,26 +47,26 @@ protected Settings nodeSettings(int nodeOrdinal) { if (randomBoolean()) { builder.put(NioTransport.NIO_WORKER_COUNT.getKey(), random().nextInt(3) + 1); } - builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioHttpPlugin.NIO_TRANSPORT_NAME); - builder.put(NetworkModule.HTTP_TYPE_KEY, NioHttpPlugin.NIO_HTTP_TRANSPORT_NAME); + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioPlugin.NIO_TRANSPORT_NAME); + builder.put(NetworkModule.HTTP_TYPE_KEY, NioPlugin.NIO_HTTP_TRANSPORT_NAME); return builder.build(); } @Override protected Settings transportClientSettings() { Settings.Builder builder = Settings.builder().put(super.transportClientSettings()); - builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioHttpPlugin.NIO_TRANSPORT_NAME); + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, NioPlugin.NIO_TRANSPORT_NAME); return builder.build(); } @Override protected Collection> nodePlugins() { - return Arrays.asList(NioHttpPlugin.class); + return Arrays.asList(NioPlugin.class); } @Override protected Collection> transportClientPlugins() { - return Arrays.asList(NioHttpPlugin.class); + return Arrays.asList(NioPlugin.class); } } diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java similarity index 96% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java index ba119f08d9de0..59413ba0b638c 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadContextTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.Unpooled; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -26,6 +26,9 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; +import org.elasticsearch.transport.nio.http.HttpReadContext; +import org.elasticsearch.transport.nio.http.NioHttpRequestHandler; import org.elasticsearch.transport.nio.utils.TestSelectionKey; import org.junit.Before; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java similarity index 97% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java index d676329d9698c..7329005acf7fa 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpWriteContextTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.DefaultHttpResponse; @@ -26,13 +26,16 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.CompositeBytesReference; -import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.SocketSelector; import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.http.ESChannelPromise; +import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; +import org.elasticsearch.transport.nio.http.HttpWriteContext; +import org.elasticsearch.transport.nio.http.HttpWriteOperation; import org.junit.Before; import org.mockito.ArgumentCaptor; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java similarity index 99% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java index 922f9f5a27b77..f1c4fc00d588c 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/Netty4HttpClient.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http; +package org.elasticsearch.transport.nio.http; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java similarity index 98% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java index 69576ffdc4029..7babd24bc7828 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpNettyAdaptorTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelFuture; @@ -49,6 +49,8 @@ import org.elasticsearch.transport.nio.channel.CloseFuture; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; +import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; +import org.elasticsearch.transport.nio.http.NioHttpNettyAdaptor; import org.junit.Before; import org.mockito.ArgumentCaptor; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java similarity index 97% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java index ef5949462e2f5..bacb07e2472b4 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpRequestSizeLimitIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java @@ -17,11 +17,11 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; -import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.transport.nio.ESNioIntegTestCase; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; @@ -29,7 +29,6 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java similarity index 96% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java index 6c26fab1f54f5..b46d18a27d228 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningDisabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java @@ -16,15 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.handler.codec.http.FullHttpResponse; -import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.transport.nio.ESNioIntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java similarity index 95% rename from plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java index e79d5eddde825..078a06563386a 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioPipeliningEnabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java @@ -17,15 +17,14 @@ * under the License. */ -package org.elasticsearch.http.nio; +package org.elasticsearch.transport.nio.http; import io.netty.handler.codec.http.FullHttpResponse; -import org.elasticsearch.ESNioIntegTestCase; +import org.elasticsearch.transport.nio.ESNioIntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.http.Netty4HttpClient; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java index 733351fe429d3..8abdc5c17d656 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java @@ -34,7 +34,7 @@ public class NioTransportPlugin extends Plugin implements NetworkPlugin { - public static final String NIO_TRANSPORT_NAME = "nio-transport"; + public static final String NIO_TRANSPORT_NAME = "nio-tcp-transport"; @Override public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, From 39eb8dee777900f7ff6adb72e3ee0e161c467deb Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 15:09:19 -0500 Subject: [PATCH 25/35] Make gradle check run --- .../common/settings/ClusterSettings.java | 5 +++++ .../http/HttpTransportSettings.java | 13 +++++++++++++ .../http/netty4/Netty4HttpServerTransport.java | 16 +++++----------- .../elasticsearch/transport/Netty4Plugin.java | 5 ----- plugins/transport-nio/build.gradle | 6 +++++- .../elasticsearch/transport/nio/NioPlugin.java | 13 ------------- .../transport/nio/http/ESChannelPromise.java | 11 ++++++++--- .../transport/nio/http/NioHttpTransport.java | 15 +++++---------- .../transport/nio/ESNioIntegTestCase.java | 1 - .../nio/http/NioHttpRequestSizeLimitIT.java | 4 ++-- .../nio/http/NioPipeliningDisabledIT.java | 2 +- .../nio/http/NioPipeliningEnabledIT.java | 2 +- 12 files changed, 45 insertions(+), 48 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index d8cb231523dca..b56b56c788e9d 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -245,6 +245,11 @@ public void apply(Settings value, Settings current, Settings previous) { HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE, HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH, HttpTransportSettings.SETTING_HTTP_RESET_COOKIES, + HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY, + HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE, + HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS, + HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, + HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, diff --git a/core/src/main/java/org/elasticsearch/http/HttpTransportSettings.java b/core/src/main/java/org/elasticsearch/http/HttpTransportSettings.java index 9bf8be2da45dd..54be8b4ecd78f 100644 --- a/core/src/main/java/org/elasticsearch/http/HttpTransportSettings.java +++ b/core/src/main/java/org/elasticsearch/http/HttpTransportSettings.java @@ -20,6 +20,7 @@ package org.elasticsearch.http; import org.elasticsearch.common.Booleans; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.transport.PortsRange; @@ -30,6 +31,7 @@ import java.util.function.Function; import static java.util.Collections.emptyList; +import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.common.settings.Setting.listSetting; public final class HttpTransportSettings { @@ -91,6 +93,17 @@ public final class HttpTransportSettings { public static final Setting SETTING_HTTP_RESET_COOKIES = Setting.boolSetting("http.reset_cookies", false, Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_NO_DELAY = + boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = + boolSetting("http.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_REUSE_ADDRESS = + boolSetting("http.tcp.reuse_address", NetworkService.TCP_REUSE_ADDRESS, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_SEND_BUFFER_SIZE = + Setting.byteSizeSetting("http.tcp.send_buffer_size", NetworkService.TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope); + public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = + Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); + private HttpTransportSettings() { } } 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 6cafe8ab7fc66..bcd62b187848f 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 @@ -108,6 +108,11 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; import static org.elasticsearch.http.netty4.cors.Netty4CorsHandler.ANY_ORIGIN; @@ -125,17 +130,6 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem (s) -> Integer.toString(EsExecutors.numberOfProcessors(s) * 2), (s) -> Setting.parseInt(s, 1, "http.netty.worker_count"), Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_NO_DELAY = - boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = - boolSetting("http.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_REUSE_ADDRESS = - boolSetting("http.tcp.reuse_address", NetworkService.TCP_REUSE_ADDRESS, Property.NodeScope); - - public static final Setting SETTING_HTTP_TCP_SEND_BUFFER_SIZE = - Setting.byteSizeSetting("http.tcp.send_buffer_size", NetworkService.TCP_SEND_BUFFER_SIZE, Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = - Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Property.NodeScope); public static final Setting SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE = Setting.byteSizeSetting("http.netty.receive_predictor_size", new ByteSizeValue(64, ByteSizeUnit.KB), Property.NodeScope); public static final Setting SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN = diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java index 61d818c0d323a..49fc35cb7cfd6 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java @@ -55,11 +55,6 @@ public List> getSettings() { return Arrays.asList( Netty4HttpServerTransport.SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS, Netty4HttpServerTransport.SETTING_HTTP_WORKER_COUNT, - Netty4HttpServerTransport.SETTING_HTTP_TCP_NO_DELAY, - Netty4HttpServerTransport.SETTING_HTTP_TCP_KEEP_ALIVE, - Netty4HttpServerTransport.SETTING_HTTP_TCP_REUSE_ADDRESS, - Netty4HttpServerTransport.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, - Netty4HttpServerTransport.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE, Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN, Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MAX, diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 2db9b1e3f5aa6..ec26bc6a5af4f 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -19,9 +19,13 @@ esplugin { description 'The nio transport.' - classname 'org.elasticsearch.http.nio.NioPlugin' + classname 'org.elasticsearch.transport.nio.NioPlugin' } +dependencyLicenses.enabled = false + +compileJava.options.compilerArgs << "-Xlint:-try" +compileTestJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked" dependencies { compile project(path: ':modules:transport-netty4', configuration: 'runtime') diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java index f6c45b61f4865..84e7cf88c07c5 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java @@ -50,24 +50,11 @@ public List> getSettings() { return Arrays.asList( NioHttpTransport.NIO_HTTP_WORKER_COUNT, NioHttpTransport.NIO_HTTP_ACCEPTOR_COUNT, - NioHttpTransport.SETTING_HTTP_TCP_NO_DELAY, - NioHttpTransport.SETTING_HTTP_TCP_KEEP_ALIVE, - NioHttpTransport.SETTING_HTTP_TCP_REUSE_ADDRESS, - NioHttpTransport.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, - NioHttpTransport.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, NioTransport.NIO_WORKER_COUNT, NioTransport.NIO_ACCEPTOR_COUNT ); } - @Override - public Settings additionalSettings() { - return Settings.builder() - .put(NetworkModule.HTTP_DEFAULT_TYPE_SETTING.getKey(), NIO_HTTP_TRANSPORT_NAME) - .put(NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING.getKey(), NIO_TRANSPORT_NAME) - .build(); - } - @Override public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, CircuitBreakerService circuitBreakerService, diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java index 9f9d6bfa4bf2f..9c918cb559353 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java @@ -24,6 +24,7 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.transport.nio.channel.NioChannel; import java.util.concurrent.ExecutionException; @@ -109,7 +110,9 @@ public ChannelPromise addListener(GenericFutureListener>... listeners) { + @SafeVarargs + @SuppressWarnings("varargs") + public final ChannelPromise addListeners(GenericFutureListener>... listeners) { return promise.addListeners(listeners); } @@ -119,7 +122,9 @@ public ChannelPromise removeListener(GenericFutureListener>... listeners) { + @SafeVarargs + @SuppressWarnings("varargs") + public final ChannelPromise removeListeners(GenericFutureListener>... listeners) { return promise.removeListeners(listeners); } @@ -170,7 +175,7 @@ public Void getNow() { @Override public boolean cancel(boolean mayInterruptIfRunning) { - return promise.cancel(mayInterruptIfRunning); + return FutureUtils.cancel(promise); } @Override diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java index f2a81e2e72ae3..c81502e5a8430 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java @@ -77,6 +77,11 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING; public class NioHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport { @@ -87,16 +92,6 @@ public class NioHttpTransport extends AbstractLifecycleComponent implements Http (s) -> Setting.parseInt(s, 1, "transport.nio.worker_count"), Setting.Property.NodeScope); public static final Setting NIO_HTTP_ACCEPTOR_COUNT = intSetting("transport.nio.http.acceptor_count", 1, 1, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_NO_DELAY = - boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = - boolSetting("http.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_REUSE_ADDRESS = - boolSetting("http.tcp.reuse_address", NetworkService.TCP_REUSE_ADDRESS, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_SEND_BUFFER_SIZE = - Setting.byteSizeSetting("http.tcp.send_buffer_size", NetworkService.TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope); - public static final Setting SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE = - Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); private final Netty4CorsConfig corsConfig; private final PortsRange port; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java index e3ba68f228bce..c4505caeae911 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/ESNioIntegTestCase.java @@ -68,5 +68,4 @@ protected Collection> nodePlugins() { protected Collection> transportClientPlugins() { return Arrays.asList(NioPlugin.class); } - } diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java index bacb07e2472b4..f816d15746ea8 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java @@ -80,7 +80,7 @@ public void testLimitsInFlightRequests() throws Exception { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { @@ -105,7 +105,7 @@ public void testDoesNotLimitExcludedRequests() throws Exception { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java index b46d18a27d228..0742c9849e257 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java @@ -53,7 +53,7 @@ public void testThatNettyHttpServerDoesNotSupportPipelining() throws Exception { HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); - TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = randomFrom(boundAddresses); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { Collection responses = nettyHttpClient.get(transportAddress.address(), requests); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java index 078a06563386a..671d57dbb0bc0 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java @@ -51,7 +51,7 @@ public void testThatNettyHttpServerSupportsPipelining() throws Exception { HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); - TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = randomFrom(boundAddresses); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { Collection responses = nettyHttpClient.get(transportAddress.address(), requests); From 257875b73ca4529890fbc3d8648ed514fec383a1 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 15:57:37 -0500 Subject: [PATCH 26/35] Ensure that integration tests are working --- .../transport/nio/NioPlugin.java | 8 + .../plugin-metadata/plugin-security.policy | 5 + .../nio/http/NioClientYamlTestSuiteIT.java | 42 ++++ .../transport/nio/rest/NioBadRequestIT.java | 77 +++++++ .../nio/rest/NioHeadBodyIsEmptyIT.java | 196 ++++++++++++++++++ .../resources/rest-api-spec/test/10_basic.yml | 19 ++ 6 files changed, 347 insertions(+) create mode 100644 plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioClientYamlTestSuiteIT.java create mode 100644 plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioBadRequestIT.java create mode 100644 plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java create mode 100644 plugins/transport-nio/src/test/resources/rest-api-spec/test/10_basic.yml diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java index 84e7cf88c07c5..86e9cb65c2df4 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java @@ -55,6 +55,14 @@ public List> getSettings() { ); } + @Override + public Settings additionalSettings() { + final Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put(NetworkModule.TRANSPORT_TYPE_KEY, NIO_TRANSPORT_NAME); + settingsBuilder.put(NetworkModule.HTTP_TYPE_KEY, NIO_HTTP_TRANSPORT_NAME); + return settingsBuilder.build(); + } + @Override public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, CircuitBreakerService circuitBreakerService, diff --git a/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy index e3253c76ef433..7096f599e2126 100644 --- a/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy +++ b/plugins/transport-nio/src/main/plugin-metadata/plugin-security.policy @@ -29,4 +29,9 @@ grant codeBase "${codebase.netty-transport-4.1.13.Final.jar}" { // Netty NioEventLoop wants to change this, because of https://bugs.openjdk.java.net/browse/JDK-6427854 // the bug says it only happened rarely, and that its fixed, but apparently it still happens rarely! permission java.util.PropertyPermission "sun.nio.ch.bugLevel", "write"; +}; + +grant codeBase "${codebase.mocksocket-1.2.jar}" { + // mocksocket makes and accepts socket connections + permission java.net.SocketPermission "*", "accept,connect"; }; \ No newline at end of file diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioClientYamlTestSuiteIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioClientYamlTestSuiteIT.java new file mode 100644 index 0000000000000..b7c139000553c --- /dev/null +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioClientYamlTestSuiteIT.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.http; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; +import org.apache.lucene.util.TimeUnits; +import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; + +//TODO: This is a *temporary* workaround to ensure a timeout does not mask other problems +@TimeoutSuite(millis = 30 * TimeUnits.MINUTE) +public class NioClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { + + public NioClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws Exception { + return ESClientYamlSuiteTestCase.createParameters(); + } + +} diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioBadRequestIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioBadRequestIT.java new file mode 100644 index 0000000000000..35dbf0e3713a0 --- /dev/null +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioBadRequestIT.java @@ -0,0 +1,77 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.rest; + +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.http.HttpTransportSettings; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.yaml.ObjectPath; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.Collections; +import java.util.Map; + +import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; +import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasToString; + +public class NioBadRequestIT extends ESRestTestCase { + + public void testBadRequest() throws IOException { + final Response response = client().performRequest("GET", "/_nodes/settings", Collections.emptyMap()); + final ObjectPath objectPath = ObjectPath.createFromResponse(response); + final Map map = objectPath.evaluate("nodes"); + int maxMaxInitialLineLength = Integer.MIN_VALUE; + final Setting httpMaxInitialLineLength = HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; + final String key = httpMaxInitialLineLength.getKey().substring("http.".length()); + for (Map.Entry entry : map.entrySet()) { + @SuppressWarnings("unchecked") final Map settings = + (Map)((Map)entry.getValue()).get("settings"); + final int maxIntialLineLength; + if (settings.containsKey("http")) { + @SuppressWarnings("unchecked") final Map httpSettings = (Map)settings.get("http"); + if (httpSettings.containsKey(key)) { + maxIntialLineLength = ByteSizeValue.parseBytesSizeValue((String)httpSettings.get(key), key).bytesAsInt(); + } else { + maxIntialLineLength = httpMaxInitialLineLength.getDefault(Settings.EMPTY).bytesAsInt(); + } + } else { + maxIntialLineLength = httpMaxInitialLineLength.getDefault(Settings.EMPTY).bytesAsInt(); + } + maxMaxInitialLineLength = Math.max(maxMaxInitialLineLength, maxIntialLineLength); + } + + final String path = "/" + new String(new byte[maxMaxInitialLineLength], Charset.forName("UTF-8")).replace('\0', 'a'); + final ResponseException e = + expectThrows( + ResponseException.class, + () -> client().performRequest(randomFrom("GET", "POST", "PUT"), path, Collections.emptyMap())); + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(BAD_REQUEST.getStatus())); + assertThat(e, hasToString(containsString("too_long_frame_exception"))); + assertThat(e, hasToString(matches("An HTTP line is larger than \\d+ bytes"))); + } +} diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java new file mode 100644 index 0000000000000..3b10392dc9d8b --- /dev/null +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java @@ -0,0 +1,196 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.rest; + +import org.elasticsearch.client.Response; +import org.elasticsearch.client.http.entity.ContentType; +import org.elasticsearch.client.http.entity.StringEntity; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.hamcrest.Matcher; + +import java.io.IOException; +import java.util.Map; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.rest.RestStatus.NOT_FOUND; +import static org.elasticsearch.rest.RestStatus.OK; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +public class NioHeadBodyIsEmptyIT extends ESRestTestCase { + + public void testHeadRoot() throws IOException { + headTestCase("/", emptyMap(), greaterThan(0)); + headTestCase("/", singletonMap("pretty", ""), greaterThan(0)); + headTestCase("/", singletonMap("pretty", "true"), greaterThan(0)); + } + + private void createTestDoc() throws IOException { + createTestDoc("test", "test"); + } + + private void createTestDoc(final String indexName, final String typeName) throws IOException { + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.field("test", "test"); + } + builder.endObject(); + client().performRequest("PUT", "/" + indexName + "/" + typeName + "/" + "1", emptyMap(), + new StringEntity(builder.string(), ContentType.APPLICATION_JSON)); + } + } + + public void testDocumentExists() throws IOException { + createTestDoc(); + headTestCase("/test/test/1", emptyMap(), greaterThan(0)); + headTestCase("/test/test/1", singletonMap("pretty", "true"), greaterThan(0)); + headTestCase("/test/test/2", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + } + + public void testIndexExists() throws IOException { + createTestDoc(); + headTestCase("/test", emptyMap(), greaterThan(0)); + headTestCase("/test", singletonMap("pretty", "true"), greaterThan(0)); + } + + public void testTypeExists() throws IOException { + createTestDoc(); + headTestCase("/test/_mapping/test", emptyMap(), greaterThan(0)); + headTestCase("/test/_mapping/test", singletonMap("pretty", "true"), greaterThan(0)); + } + + public void testTypeDoesNotExist() throws IOException { + createTestDoc(); + headTestCase("/test/_mapping/does-not-exist", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + headTestCase("/text/_mapping/test,does-not-exist", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + } + + public void testAliasExists() throws IOException { + createTestDoc(); + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startArray("actions"); + { + builder.startObject(); + { + builder.startObject("add"); + { + builder.field("index", "test"); + builder.field("alias", "test_alias"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endArray(); + } + builder.endObject(); + + client().performRequest("POST", "_aliases", emptyMap(), new StringEntity(builder.string(), ContentType.APPLICATION_JSON)); + headTestCase("/_alias/test_alias", emptyMap(), greaterThan(0)); + headTestCase("/test/_alias/test_alias", emptyMap(), greaterThan(0)); + } + } + + public void testAliasDoesNotExist() throws IOException { + createTestDoc(); + headTestCase("/_alias/test_alias", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + headTestCase("/test/_alias/test_alias", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + } + + public void testTemplateExists() throws IOException { + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.array("index_patterns", "*"); + builder.startObject("settings"); + { + builder.field("number_of_replicas", 0); + } + builder.endObject(); + } + builder.endObject(); + + client().performRequest("PUT", "/_template/template", emptyMap(), + new StringEntity(builder.string(), ContentType.APPLICATION_JSON)); + headTestCase("/_template/template", emptyMap(), greaterThan(0)); + } + } + + public void testGetSourceAction() throws IOException { + createTestDoc(); + headTestCase("/test/test/1/_source", emptyMap(), greaterThan(0)); + headTestCase("/test/test/2/_source", emptyMap(), NOT_FOUND.getStatus(), equalTo(0)); + + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startObject("mappings"); + { + builder.startObject("test-no-source"); + { + builder.startObject("_source"); + { + builder.field("enabled", false); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + client().performRequest("PUT", "/test-no-source", emptyMap(), new StringEntity(builder.string(), ContentType.APPLICATION_JSON)); + createTestDoc("test-no-source", "test-no-source"); + headTestCase("/test-no-source/test-no-source/1/_source", emptyMap(), NOT_FOUND.getStatus(), equalTo(0)); + } + } + + public void testException() throws IOException { + /* + * This will throw an index not found exception which will be sent on the channel; previously when handling HEAD requests that would + * throw an exception, the content was swallowed and a content length header of zero was returned. Instead of swallowing the content + * we now let it rise up to the upstream channel so that it can compute the content length that would be returned. This test case is + * a test for this situation. + */ + headTestCase("/index-not-found-exception", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + } + + private void headTestCase(final String url, final Map params, final Matcher matcher) throws IOException { + headTestCase(url, params, OK.getStatus(), matcher); + } + + private void headTestCase( + final String url, + final Map params, + final int expectedStatusCode, + final Matcher matcher) throws IOException { + Response response = client().performRequest("HEAD", url, params); + assertEquals(expectedStatusCode, response.getStatusLine().getStatusCode()); + assertThat(Integer.valueOf(response.getHeader("Content-Length")), matcher); + assertNull("HEAD requests shouldn't have a response body but " + url + " did", response.getEntity()); + } + +} diff --git a/plugins/transport-nio/src/test/resources/rest-api-spec/test/10_basic.yml b/plugins/transport-nio/src/test/resources/rest-api-spec/test/10_basic.yml new file mode 100644 index 0000000000000..9361ac43f82dd --- /dev/null +++ b/plugins/transport-nio/src/test/resources/rest-api-spec/test/10_basic.yml @@ -0,0 +1,19 @@ +# Integration tests for Netty transport +# +"Netty loaded": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.plugins.0.name: transport-nio } + + - do: + cluster.stats: {} + + - match: { nodes.network_types.transport_types.nio-transport: 1 } + - match: { nodes.network_types.http_types.nio-http-transport: 1 } From f5e35af55858228eb1b557f58b5d3c7b160a58b8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 16:14:36 -0500 Subject: [PATCH 27/35] Fix some visibility issues --- .../http/netty4/Netty4HttpChannel.java | 4 +- .../http/netty4/Netty4HttpRequest.java | 4 +- .../transport/nio/http/ESEmbeddedChannel.java | 12 +- .../transport/nio/http/NioHttpChannel.java | 14 +- .../transport/nio/http/NioHttpRequest.java | 202 ++++++++++++++++++ .../nio/http/NioHttpRequestHandler.java | 7 +- .../transport/nio/http/NioHttpTransport.java | 7 +- 7 files changed, 224 insertions(+), 26 deletions(-) create mode 100644 plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequest.java diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java index be2411d33fab9..12db47908d1f3 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpChannel.java @@ -53,7 +53,7 @@ import java.util.Map; import java.util.Set; -public final class Netty4HttpChannel extends AbstractRestChannel { +final class Netty4HttpChannel extends AbstractRestChannel { private final Netty4HttpServerTransport transport; private final Channel channel; @@ -69,7 +69,7 @@ public final class Netty4HttpChannel extends AbstractRestChannel { * @param detailedErrorsEnabled true iff error messages should include stack traces. * @param threadContext the thread context for the channel */ - public Netty4HttpChannel( + Netty4HttpChannel( final Netty4HttpServerTransport transport, final Netty4HttpRequest request, final HttpPipelinedRequest pipelinedRequest, 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 c1b5632aadf21..0065d531ab4d0 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 @@ -39,13 +39,13 @@ import java.util.Set; import java.util.stream.Collectors; -public class Netty4HttpRequest extends RestRequest { +class Netty4HttpRequest extends RestRequest { private final FullHttpRequest request; private final Channel channel; private final BytesReference content; - public Netty4HttpRequest(NamedXContentRegistry xContentRegistry, FullHttpRequest request, Channel channel) { + Netty4HttpRequest(NamedXContentRegistry xContentRegistry, FullHttpRequest request, Channel channel) { super(xContentRegistry, request.uri(), new HttpHeadersMap(request.headers())); this.request = request; this.channel = channel; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java index 462bb51e67af2..5743bc04ef3f7 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java @@ -31,11 +31,11 @@ import java.util.LinkedList; import java.util.Queue; -public class ESEmbeddedChannel extends EmbeddedChannel { +class ESEmbeddedChannel extends EmbeddedChannel { private LinkedList> messages = new LinkedList<>(); - public ESEmbeddedChannel() { + ESEmbeddedChannel() { pipeline().addFirst("promise_captor", new ChannelOutboundHandlerAdapter() { @Override @@ -56,20 +56,20 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) }); } - public Queue decode(ByteBuf inboundBytes) { + Queue decode(ByteBuf inboundBytes) { writeInbound(inboundBytes); return inboundMessages(); } - public Tuple popMessage() { + Tuple popMessage() { return messages.pollFirst(); } - public boolean hasMessages() { + boolean hasMessages() { return messages.size() > 0; } - public void closeNettyChannel() { + void closeNettyChannel() { close(); } } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java index 04cea79b8fa38..ecd98b434b7c6 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.http.netty4.Netty4HttpRequest; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsHandler; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; @@ -56,11 +55,11 @@ import java.util.Map; import java.util.Set; -public final class NioHttpChannel extends AbstractRestChannel { +final class NioHttpChannel extends AbstractRestChannel { - private final Channel channel; private final FullHttpRequest nettyRequest; private final NioSocketChannel nioChannel; + private final Channel nettyChannel; private final HttpPipelinedRequest pipelinedRequest; private final ThreadContext threadContext; private final Netty4CorsConfig corsConfig; @@ -69,17 +68,18 @@ public final class NioHttpChannel extends AbstractRestChannel { /** * @param request The request that is handled by this channel. * @param channel The channel + * @param nettyChannel The netty channel * @param pipelinedRequest If HTTP pipelining is enabled provide the corresponding pipelined request. May be null if * HTTP pipelining is disabled. * @param detailedErrorsEnabled true iff error messages should include stack traces. * @param threadContext the thread context for the channel */ - public NioHttpChannel(Netty4HttpRequest request, NioSocketChannel channel, HttpPipelinedRequest pipelinedRequest, - boolean detailedErrorsEnabled, ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { + NioHttpChannel(NioHttpRequest request, NioSocketChannel channel, Channel nettyChannel, HttpPipelinedRequest pipelinedRequest, + boolean detailedErrorsEnabled, ThreadContext threadContext, Netty4CorsConfig corsConfig, boolean resetCookies) { super(request, detailedErrorsEnabled); - this.channel = request.getChannel(); this.nettyRequest = request.request(); this.nioChannel = channel; + this.nettyChannel = nettyChannel; this.pipelinedRequest = pipelinedRequest; this.threadContext = threadContext; this.corsConfig = corsConfig; @@ -127,7 +127,7 @@ public void sendResponse(RestResponse response) { addCookies(resp); - final ESChannelPromise promise = new ESChannelPromise(channel.newPromise()); + final ESChannelPromise promise = new ESChannelPromise(nettyChannel.newPromise()); if (releaseContent) { promise.addListener(f -> ((Releasable) content).close()); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequest.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequest.java new file mode 100644 index 0000000000000..10bbdb3dac056 --- /dev/null +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequest.java @@ -0,0 +1,202 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio.http; + +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpMethod; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.transport.netty4.Netty4Utils; +import org.elasticsearch.transport.nio.channel.NioSocketChannel; + +import java.net.SocketAddress; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +class NioHttpRequest extends RestRequest { + + private final FullHttpRequest request; + private final NioSocketChannel channel; + private final BytesReference content; + + NioHttpRequest(NamedXContentRegistry xContentRegistry, FullHttpRequest request, NioSocketChannel channel) { + super(xContentRegistry, request.uri(), new HttpHeadersMap(request.headers())); + this.request = request; + this.channel = channel; + if (request.content().isReadable()) { + this.content = Netty4Utils.toBytesReference(request.content()); + } else { + this.content = BytesArray.EMPTY; + } + } + + FullHttpRequest request() { + return this.request; + } + + @Override + public Method method() { + HttpMethod httpMethod = request.method(); + if (httpMethod == HttpMethod.GET) + return Method.GET; + + if (httpMethod == HttpMethod.POST) + return Method.POST; + + if (httpMethod == HttpMethod.PUT) + return Method.PUT; + + if (httpMethod == HttpMethod.DELETE) + return Method.DELETE; + + if (httpMethod == HttpMethod.HEAD) { + return Method.HEAD; + } + + if (httpMethod == HttpMethod.OPTIONS) { + return Method.OPTIONS; + } + + return Method.GET; + } + + @Override + public String uri() { + return request.uri(); + } + + @Override + public boolean hasContent() { + return content.length() > 0; + } + + @Override + public BytesReference content() { + return content; + } + + /** + * Returns the remote address where this rest request channel is "connected to". The + * returned {@link SocketAddress} is supposed to be down-cast into more + * concrete type such as {@link java.net.InetSocketAddress} to retrieve + * the detailed information. + */ + @Override + public SocketAddress getRemoteAddress() { + return channel.getRemoteAddress(); + } + + /** + * Returns the local address where this request channel is bound to. The returned + * {@link SocketAddress} is supposed to be down-cast into more concrete + * type such as {@link java.net.InetSocketAddress} to retrieve the detailed + * information. + */ + @Override + public SocketAddress getLocalAddress() { + return channel.getLocalAddress(); + } + + /** + * A wrapper of {@link HttpHeaders} that implements a map to prevent copying unnecessarily. This class does not support modifications + * and due to the underlying implementation, it performs case insensitive lookups of key to values. + * + * It is important to note that this implementation does have some downsides in that each invocation of the + * {@link #values()} and {@link #entrySet()} methods will perform a copy of the values in the HttpHeaders rather than returning a + * view of the underlying values. + */ + private static class HttpHeadersMap implements Map> { + + private final HttpHeaders httpHeaders; + + private HttpHeadersMap(HttpHeaders httpHeaders) { + this.httpHeaders = httpHeaders; + } + + @Override + public int size() { + return httpHeaders.size(); + } + + @Override + public boolean isEmpty() { + return httpHeaders.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return key instanceof String && httpHeaders.contains((String) key); + } + + @Override + public boolean containsValue(Object value) { + return value instanceof List && httpHeaders.names().stream().map(httpHeaders::getAll).anyMatch(value::equals); + } + + @Override + public List get(Object key) { + return key instanceof String ? httpHeaders.getAll((String) key) : null; + } + + @Override + public List put(String key, List value) { + throw new UnsupportedOperationException("modifications are not supported"); + } + + @Override + public List remove(Object key) { + throw new UnsupportedOperationException("modifications are not supported"); + } + + @Override + public void putAll(Map> m) { + throw new UnsupportedOperationException("modifications are not supported"); + } + + @Override + public void clear() { + throw new UnsupportedOperationException("modifications are not supported"); + } + + @Override + public Set keySet() { + return httpHeaders.names(); + } + + @Override + public Collection> values() { + return httpHeaders.names().stream().map(k -> Collections.unmodifiableList(httpHeaders.getAll(k))).collect(Collectors.toList()); + } + + @Override + public Set>> entrySet() { + return httpHeaders.names().stream().map(k -> new AbstractMap.SimpleImmutableEntry<>(k, httpHeaders.getAll(k))) + .collect(Collectors.toSet()); + } + } +} diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java index 371e9fb38fd2d..193fdefe8e4df 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpRequestHandler.java @@ -25,7 +25,6 @@ import io.netty.handler.codec.http.FullHttpRequest; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.http.netty4.Netty4HttpRequest; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.pipelining.HttpPipelinedRequest; import org.elasticsearch.transport.nio.channel.NioSocketChannel; @@ -71,9 +70,9 @@ public void handleMessage(NioSocketChannel channel, Channel nettyChannel, Object Unpooled.copiedBuffer(request.content()), request.headers(), request.trailingHeaders()); - final Netty4HttpRequest httpRequest = new Netty4HttpRequest(xContentRegistry, copy, nettyChannel); - final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, channel, pipelinedRequest, detailedErrorsEnabled, threadContext, - corsConfig, resetCookies); + final NioHttpRequest httpRequest = new NioHttpRequest(xContentRegistry, copy, channel); + final NioHttpChannel httpChannel = new NioHttpChannel(httpRequest, channel, nettyChannel, pipelinedRequest, detailedErrorsEnabled, + threadContext, corsConfig, resetCookies); if (request.decoderResult().isSuccess()) { transport.dispatchRequest(httpRequest, httpChannel); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java index c81502e5a8430..4c8d5a4cd17bd 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpTransport.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.common.transport.PortsRange; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -43,7 +42,6 @@ import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpStats; -import org.elasticsearch.http.netty4.Netty4HttpRequest; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.threadpool.ThreadPool; @@ -68,7 +66,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import static org.elasticsearch.common.settings.Setting.boolSetting; import static org.elasticsearch.common.settings.Setting.intSetting; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_BIND_HOST; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED; @@ -212,14 +209,14 @@ public HttpStats stats() { ? 0 : openChannels.getAcceptedChannels().size() + serverChannelsCount); } - public void dispatchRequest(Netty4HttpRequest httpRequest, RestChannel channel) { + public void dispatchRequest(NioHttpRequest httpRequest, RestChannel channel) { final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { dispatcher.dispatchRequest(httpRequest, channel, threadContext); } } - public void dispatchBadRequest(Netty4HttpRequest httpRequest, RestChannel channel, Throwable cause) { + public void dispatchBadRequest(NioHttpRequest httpRequest, RestChannel channel, Throwable cause) { final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { dispatcher.dispatchBadRequest(httpRequest, channel, threadContext, cause); From 95171b8a3db8a35814fc62f3cb85efe883e06377 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 16:21:30 -0500 Subject: [PATCH 28/35] A few cleanups --- .../http/netty4/Netty4HttpClient.java | 6 ++-- .../nio/http/NioHttpRequestSizeLimitIT.java | 4 +-- ...tpClient.java => NioNetty4HttpClient.java} | 4 +-- .../nio/http/NioPipeliningDisabledIT.java | 4 +-- .../nio/http/NioPipeliningEnabledIT.java | 4 +-- .../nio/channel/ChannelConsumerAdaptor.java | 31 ------------------- 6 files changed, 11 insertions(+), 42 deletions(-) rename plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/{Netty4HttpClient.java => NioNetty4HttpClient.java} (98%) delete mode 100644 test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java index 55b6eefe66cb8..7c4471e249102 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpClient.java @@ -61,7 +61,7 @@ /** * Tiny helper to send http requests over netty. */ -public class Netty4HttpClient implements Closeable { +class Netty4HttpClient implements Closeable { static Collection returnHttpResponseBodies(Collection responses) { List list = new ArrayList<>(responses.size()); @@ -71,7 +71,7 @@ static Collection returnHttpResponseBodies(Collection return list; } - public static Collection returnOpaqueIds(Collection responses) { + static Collection returnOpaqueIds(Collection responses) { List list = new ArrayList<>(responses.size()); for (HttpResponse response : responses) { list.add(response.headers().get("X-Opaque-Id")); @@ -81,7 +81,7 @@ public static Collection returnOpaqueIds(Collection re private final Bootstrap clientBootstrap; - public Netty4HttpClient() { + Netty4HttpClient() { clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).group(new NioEventLoopGroup()); } diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java index f816d15746ea8..f181ab62e3ba1 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpRequestSizeLimitIT.java @@ -83,7 +83,7 @@ public void testLimitsInFlightRequests() throws Exception { TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress ().boundAddresses()); - try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + try (NioNetty4HttpClient nettyHttpClient = new NioNetty4HttpClient()) { Collection singleResponse = nettyHttpClient.post(transportAddress.address(), requests[0]); assertThat(singleResponse, hasSize(1)); assertAtLeastOnceExpectedStatus(singleResponse, HttpResponseStatus.OK); @@ -108,7 +108,7 @@ public void testDoesNotLimitExcludedRequests() throws Exception { TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress ().boundAddresses()); - try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + try (NioNetty4HttpClient nettyHttpClient = new NioNetty4HttpClient()) { Collection responses = nettyHttpClient.put(transportAddress.address(), requestUris); assertThat(responses, hasSize(requestUris.length)); assertAllInExpectedStatus(responses, HttpResponseStatus.OK); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioNetty4HttpClient.java similarity index 98% rename from plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java rename to plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioNetty4HttpClient.java index f1c4fc00d588c..d57ee8cf8fb96 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/Netty4HttpClient.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioNetty4HttpClient.java @@ -61,7 +61,7 @@ /** * Tiny helper to send http requests over netty. */ -public class Netty4HttpClient implements Closeable { +public class NioNetty4HttpClient implements Closeable { static Collection returnHttpResponseBodies(Collection responses) { List list = new ArrayList<>(responses.size()); @@ -81,7 +81,7 @@ public static Collection returnOpaqueIds(Collection re private final Bootstrap clientBootstrap; - public Netty4HttpClient() { + public NioNetty4HttpClient() { clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).group(new NioEventLoopGroup()); } diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java index 0742c9849e257..85a350d09c57f 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningDisabledIT.java @@ -55,11 +55,11 @@ public void testThatNettyHttpServerDoesNotSupportPipelining() throws Exception { TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); TransportAddress transportAddress = randomFrom(boundAddresses); - try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + try (NioNetty4HttpClient nettyHttpClient = new NioNetty4HttpClient()) { Collection responses = nettyHttpClient.get(transportAddress.address(), requests); assertThat(responses, hasSize(requests.length)); - List opaqueIds = new ArrayList<>(Netty4HttpClient.returnOpaqueIds(responses)); + List opaqueIds = new ArrayList<>(NioNetty4HttpClient.returnOpaqueIds(responses)); assertResponsesOutOfOrder(opaqueIds); } diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java index 671d57dbb0bc0..db602149284a7 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioPipeliningEnabledIT.java @@ -53,11 +53,11 @@ public void testThatNettyHttpServerSupportsPipelining() throws Exception { TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); TransportAddress transportAddress = randomFrom(boundAddresses); - try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { + try (NioNetty4HttpClient nettyHttpClient = new NioNetty4HttpClient()) { Collection responses = nettyHttpClient.get(transportAddress.address(), requests); assertThat(responses, hasSize(5)); - Collection opaqueIds = Netty4HttpClient.returnOpaqueIds(responses); + Collection opaqueIds = NioNetty4HttpClient.returnOpaqueIds(responses); assertOpaqueIdsInOrder(opaqueIds); } } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java deleted file mode 100644 index fc605171c348a..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/channel/ChannelConsumerAdaptor.java +++ /dev/null @@ -1,31 +0,0 @@ -package org.elasticsearch.transport.nio.channel; - -import org.elasticsearch.action.ActionListener; - -import java.util.function.Consumer; - -public class ChannelConsumerAdaptor implements ActionListener { - - private final NioChannel channel; - private final Consumer consumer; - - private ChannelConsumerAdaptor(NioChannel channel, Consumer consumer) { - this.channel = channel; - this.consumer = consumer; - } - - static ChannelConsumerAdaptor adapt(NioChannel channel, Consumer consumer) { - return new ChannelConsumerAdaptor(channel, consumer); - } - - @Override - public void onResponse(NioChannel channel) { - consumer.accept(channel); - } - - @Override - public void onFailure(Exception e) { - consumer.accept(channel); - } -} - From 5f75f38c85e5bf368f48affa2534cff8b24a9d79 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 18 Aug 2017 16:32:18 -0500 Subject: [PATCH 29/35] Fix license --- .../transport/nio/WriteOperation.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java index 4bd4418826bf6..341d1e5a1c44f 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/WriteOperation.java @@ -1,3 +1,22 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.elasticsearch.transport.nio; import org.elasticsearch.action.ActionListener; From 40c4135bbafbcf0c20c268319b2de024de6fbc0a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 21 Aug 2017 11:44:30 -0500 Subject: [PATCH 30/35] Make changes based on review --- .../http/netty4/Netty4HttpRequest.java | 2 +- .../transport/nio/http/HttpReadContext.java | 4 +-- .../transport/nio/http/HttpWriteContext.java | 25 +++++++++++-------- ...lPromise.java => NettyActionListener.java} | 9 +++++-- ...dChannel.java => NettyChannelAdaptor.java} | 10 ++++++-- .../transport/nio/http/NioHttpChannel.java | 2 +- .../nio/http/NioHttpNettyAdaptor.java | 6 ++--- .../nio/http/NioHttpRequestHandler.java | 10 ++------ .../transport/nio/http/NioHttpTransport.java | 2 +- .../nio/http/HttpReadContextTests.java | 7 ++---- .../nio/http/HttpWriteContextTests.java | 16 +++++------- .../nio/http/NioHttpNettyAdaptorTests.java | 18 ++++++------- 12 files changed, 55 insertions(+), 56 deletions(-) rename plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/{ESChannelPromise.java => NettyActionListener.java} (93%) rename plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/{ESEmbeddedChannel.java => NettyChannelAdaptor.java} (88%) 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 0065d531ab4d0..0a37b31ec9874 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 @@ -39,7 +39,7 @@ import java.util.Set; import java.util.stream.Collectors; -class Netty4HttpRequest extends RestRequest { +final class Netty4HttpRequest extends RestRequest { private final FullHttpRequest request; private final Channel channel; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java index 6a5bc2e7d4445..0f41d79ebe88f 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpReadContext.java @@ -36,11 +36,11 @@ public class HttpReadContext implements ReadContext { private final NioSocketChannel channel; - private final ESEmbeddedChannel nettyPipelineAdaptor; + private final NettyChannelAdaptor nettyPipelineAdaptor; private final LinkedList references = new LinkedList<>(); private final NioHttpRequestHandler requestHandler; - public HttpReadContext(NioSocketChannel channel, ESEmbeddedChannel adaptor, NioHttpRequestHandler requestHandler) { + public HttpReadContext(NioSocketChannel channel, NettyChannelAdaptor adaptor, NioHttpRequestHandler requestHandler) { this.channel = channel; this.requestHandler = requestHandler; this.nettyPipelineAdaptor = adaptor; diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java index cb8b748dad40a..a2825bb6c6960 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/HttpWriteContext.java @@ -36,10 +36,10 @@ public class HttpWriteContext implements WriteContext { private final NioSocketChannel channel; - private final ESEmbeddedChannel adaptor; + private final NettyChannelAdaptor adaptor; private ByteWriteOperation partiallyFlushed; - public HttpWriteContext(NioSocketChannel channel, ESEmbeddedChannel adaptor) { + public HttpWriteContext(NioSocketChannel channel, NettyChannelAdaptor adaptor) { this.channel = channel; this.adaptor = adaptor; } @@ -53,13 +53,16 @@ public void sendMessage(Object message, ActionListener listener) { WriteOperation writeOperation = new HttpWriteOperation(channel, message, listener); SocketSelector selector = channel.getSelector(); - if (selector.isOnCurrentThread() == false) { + + // If we are on the selector thread, we can queue the message directly in the channel buffer. + // Otherwise we must call queueWrite which will dispatch to the selector thread. + if (selector.isOnCurrentThread()) { + // TODO: Eval if we will allow writes from sendMessage + selector.queueWriteInChannelBuffer(writeOperation); + } else { selector.queueWrite(writeOperation); - return; } - // TODO: Eval if we will allow writes from sendMessage - selector.queueWriteInChannelBuffer(writeOperation); } @Override @@ -67,7 +70,7 @@ public void queueWriteOperations(WriteOperation writeOperation) { assert channel.getSelector().isOnCurrentThread() : "Must be on selector thread to queue writes"; HttpWriteOperation httpWriteOperation = (HttpWriteOperation) writeOperation; - ESChannelPromise listener = (ESChannelPromise) httpWriteOperation.getListener(); + NettyActionListener listener = (NettyActionListener) httpWriteOperation.getListener(); adaptor.write(httpWriteOperation.getHttpResponse(), listener); } @@ -87,11 +90,11 @@ public void flushChannel() throws IOException { boolean previousMessageFullyFlushed = true; while (previousMessageFullyFlushed && (message = adaptor.popMessage()) != null) { ChannelPromise promise = message.v2(); - ESChannelPromise listener; - if (promise instanceof ESChannelPromise) { - listener = (ESChannelPromise) promise; + NettyActionListener listener; + if (promise instanceof NettyActionListener) { + listener = (NettyActionListener) promise; } else { - listener = new ESChannelPromise(promise); + listener = new NettyActionListener(promise); } ByteWriteOperation writeOperation = new ByteWriteOperation(channel, message.v1(), listener); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyActionListener.java similarity index 93% rename from plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyActionListener.java index 9c918cb559353..2a808291155d4 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESChannelPromise.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyActionListener.java @@ -31,11 +31,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class ESChannelPromise implements ActionListener, ChannelPromise { +/** + * This is an {@link ActionListener} that interfaces with netty code. It wraps a netty promise and will + * complete that promise when onResponse or onFailure is called. It delegates the normal promise methods + * to the underlying promise. + */ +public class NettyActionListener implements ActionListener, ChannelPromise { private final ChannelPromise promise; - public ESChannelPromise(ChannelPromise promise) { + public NettyActionListener(ChannelPromise promise) { this.promise = promise; } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyChannelAdaptor.java similarity index 88% rename from plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java rename to plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyChannelAdaptor.java index 5743bc04ef3f7..82487fb5b3e2c 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/ESEmbeddedChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NettyChannelAdaptor.java @@ -31,11 +31,17 @@ import java.util.LinkedList; import java.util.Queue; -class ESEmbeddedChannel extends EmbeddedChannel { +/** + * This class adapts a netty channel for our usage. In particular, it captures writes at the end of the + * pipeline and places them in a queue that can be accessed by our code. + */ +class NettyChannelAdaptor extends EmbeddedChannel { + + // TODO: Explore if this can be made more efficient by generating less garbage private LinkedList> messages = new LinkedList<>(); - ESEmbeddedChannel() { + NettyChannelAdaptor() { pipeline().addFirst("promise_captor", new ChannelOutboundHandlerAdapter() { @Override diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java index ecd98b434b7c6..0b9f67cf6938f 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpChannel.java @@ -127,7 +127,7 @@ public void sendResponse(RestResponse response) { addCookies(resp); - final ESChannelPromise promise = new ESChannelPromise(nettyChannel.newPromise()); + final NettyActionListener promise = new NettyActionListener(nettyChannel.newPromise()); if (releaseContent) { promise.addListener(f -> ((Releasable) content).close()); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java index 1474780ea93f4..b49fb9e51f501 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptor.java @@ -80,8 +80,8 @@ protected NioHttpNettyAdaptor(Logger logger, Settings settings, BiConsumer { - ESEmbeddedChannel adaptor = nettyAdaptor.getAdaptor(c); + NettyChannelAdaptor adaptor = nettyAdaptor.getAdaptor(c); c.setContexts(new HttpReadContext(c, adaptor, handler), new HttpWriteContext(c, adaptor)); }; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java index 59413ba0b638c..fab19030eef50 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpReadContextTests.java @@ -26,9 +26,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.nio.NetworkBytesReference; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; -import org.elasticsearch.transport.nio.http.HttpReadContext; -import org.elasticsearch.transport.nio.http.NioHttpRequestHandler; import org.elasticsearch.transport.nio.utils.TestSelectionKey; import org.junit.Before; @@ -51,7 +48,7 @@ public class HttpReadContextTests extends ESTestCase { private NioHttpRequestHandler handler; private NioSocketChannel channel; - private ESEmbeddedChannel adaptor; + private NettyChannelAdaptor adaptor; private int messageLength; private HttpReadContext readContext; @@ -60,7 +57,7 @@ public class HttpReadContextTests extends ESTestCase { public void init() throws IOException { handler = mock(NioHttpRequestHandler.class); channel = mock(NioSocketChannel.class); - adaptor = mock(ESEmbeddedChannel.class); + adaptor = mock(NettyChannelAdaptor.class); messageLength = randomInt(96) + 10; diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java index 7329005acf7fa..154985acf337e 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/HttpWriteContextTests.java @@ -32,10 +32,6 @@ import org.elasticsearch.transport.nio.SocketSelector; import org.elasticsearch.transport.nio.WriteOperation; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.http.ESChannelPromise; -import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; -import org.elasticsearch.transport.nio.http.HttpWriteContext; -import org.elasticsearch.transport.nio.http.HttpWriteOperation; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -54,19 +50,19 @@ @SuppressWarnings("unchecked") public class HttpWriteContextTests extends ESTestCase { - private ESEmbeddedChannel adaptor; + private NettyChannelAdaptor adaptor; private NioSocketChannel channel; private SocketSelector selector; - private ESChannelPromise listener; + private NettyActionListener listener; private HttpWriteContext writeContext; @Before public void initMocks() { - adaptor = mock(ESEmbeddedChannel.class); + adaptor = mock(NettyChannelAdaptor.class); channel = mock(NioSocketChannel.class); selector = mock(SocketSelector.class); - listener = mock(ESChannelPromise.class); + listener = mock(NettyActionListener.class); writeContext = new HttpWriteContext(channel, adaptor); @@ -142,7 +138,7 @@ public void testSingleFlush() throws IOException { } public void testMultiFlush() throws IOException { - ESChannelPromise listener2 = mock(ESChannelPromise.class); + NettyActionListener listener2 = mock(NettyActionListener.class); when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message1"), listener), new Tuple<>(new BytesArray("message2"), listener2), null); @@ -225,7 +221,7 @@ public void testHasQueuedWillConsiderPartialFlush() throws IOException { } public void testAdaptorMessagesAndPartialMessagesAreClosed() throws IOException { - ESChannelPromise listener2 = mock(ESChannelPromise.class); + NettyActionListener listener2 = mock(NettyActionListener.class); when(adaptor.popMessage()).thenReturn(new Tuple<>(new BytesArray("message1"), listener), new Tuple<>(new BytesArray("message2"), listener2), null); when(channel.write(any())).thenAnswer(invocationOnMock -> { diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java index 7babd24bc7828..dd6354102cccb 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/http/NioHttpNettyAdaptorTests.java @@ -49,8 +49,6 @@ import org.elasticsearch.transport.nio.channel.CloseFuture; import org.elasticsearch.transport.nio.channel.NioChannel; import org.elasticsearch.transport.nio.channel.NioSocketChannel; -import org.elasticsearch.transport.nio.http.ESEmbeddedChannel; -import org.elasticsearch.transport.nio.http.NioHttpNettyAdaptor; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -84,7 +82,7 @@ public void setMocks() { @SuppressWarnings("unchecked") public void testCloseAdaptorSchedulesRealChannelForClose() { NioSocketChannel channel = mock(NioSocketChannel.class); - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(channel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(channel); ArgumentCaptor captor = ArgumentCaptor.forClass(ActionListener.class); CloseFuture closeFuture = mock(CloseFuture.class); when(channel.closeAsync()).thenReturn(closeFuture); @@ -109,7 +107,7 @@ public void testCloseAdaptorSchedulesRealChannelForClose() { } public void testSuccessfulDecodeHttpRequest() { - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); String uri = "localhost:9090/" + randomAlphaOfLength(8); HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); @@ -131,7 +129,7 @@ public void testSuccessfulDecodeHttpRequest() { } public void testDecodeHttpRequestError() { - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); String uri = "localhost:9090/" + randomAlphaOfLength(8); HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); @@ -150,7 +148,7 @@ public void testDecodeHttpRequestError() { } public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); String uri = "localhost:9090/" + randomAlphaOfLength(8); HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, uri, false); @@ -172,7 +170,7 @@ public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() { } public void testEncodeHttpResponse() { - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); prepareAdaptorForResponse(channelAdaptor); @@ -188,7 +186,7 @@ public void testEncodeHttpResponse() { } public void testEncodedMessageIsReleasedWhenPromiseCompleted() { - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); prepareAdaptorForResponse(channelAdaptor); @@ -216,7 +214,7 @@ public void testEncodedMessageIsReleasedWhenPromiseCompleted() { public void testResponsesAreClearedOnClose() { adaptor = new NioHttpNettyAdaptor(logger, Settings.EMPTY, exceptionHandler, Netty4CorsConfigBuilder.forAnyOrigin().build(), 1024); - ESEmbeddedChannel channelAdaptor = adaptor.getAdaptor(nioSocketChannel); + NettyChannelAdaptor channelAdaptor = adaptor.getAdaptor(nioSocketChannel); prepareAdaptorForResponse(channelAdaptor); HttpPipelinedRequest pipelinedRequest2 = prepareAdaptorForResponse(channelAdaptor); @@ -237,7 +235,7 @@ public void testResponsesAreClearedOnClose() { assertTrue(writePromise.cause() instanceof ClosedChannelException); } - private HttpPipelinedRequest prepareAdaptorForResponse(ESEmbeddedChannel adaptor) { + private HttpPipelinedRequest prepareAdaptorForResponse(NettyChannelAdaptor adaptor) { HttpMethod method = HttpMethod.GET; HttpVersion version = HttpVersion.HTTP_1_1; String uri = "http://localhost:9090/" + randomAlphaOfLength(8); From b92827d6e7ab4fa876a1de325f45dd5f864b186b Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 21 Aug 2017 13:29:43 -0500 Subject: [PATCH 31/35] Add bootstrap check --- .../nio/NioNotEnabledBootstrapCheck.java | 36 +++++++++++++++++++ .../transport/nio/NioPlugin.java | 8 ++++- 2 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java new file mode 100644 index 0000000000000..2e6c99a99b0f7 --- /dev/null +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java @@ -0,0 +1,36 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.transport.nio; + +import org.elasticsearch.bootstrap.BootstrapCheck; + +public class NioNotEnabledBootstrapCheck implements BootstrapCheck { + + @Override + public boolean check() { + return true; + } + + @Override + public String errorMessage() { + return "The transport-nio plugin is experimental and not ready for production usage. It should not " + + "be enabled in production."; + } +} diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java index 86e9cb65c2df4..06559863c310e 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioPlugin.java @@ -19,6 +19,7 @@ package org.elasticsearch.transport.nio; +import org.elasticsearch.bootstrap.BootstrapCheck; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.network.NetworkService; @@ -27,12 +28,12 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.transport.nio.http.NioHttpTransport; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.nio.http.NioHttpTransport; import java.util.Arrays; import java.util.Collections; @@ -72,6 +73,11 @@ public Map> getTransports(Settings settings, ThreadP namedWriteableRegistry, circuitBreakerService)); } + @Override + public List getBootstrapChecks() { + return Collections.singletonList(new NioNotEnabledBootstrapCheck()); + } + @Override public Map> getHttpTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, CircuitBreakerService circuitBreakerService, From ce2451d1bdd30bb4baac25768ec61b826d7f3b09 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 13 Sep 2017 09:11:09 -0600 Subject: [PATCH 32/35] Fix issue from merge --- .../transport/nio/rest/NioHeadBodyIsEmptyIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java index 3b10392dc9d8b..b86ae82b3f986 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/rest/NioHeadBodyIsEmptyIT.java @@ -19,9 +19,9 @@ package org.elasticsearch.transport.nio.rest; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; import org.elasticsearch.client.Response; -import org.elasticsearch.client.http.entity.ContentType; -import org.elasticsearch.client.http.entity.StringEntity; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.test.rest.ESRestTestCase; import org.hamcrest.Matcher; From 5ef5948ff4c7d24fd3fdf86ccf837f9ea321f641 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 13 Sep 2017 09:55:12 -0600 Subject: [PATCH 33/35] Fix visibility issue with class used in x-pack --- .../java/org/elasticsearch/http/netty4/Netty4HttpRequest.java | 2 +- 1 file changed, 1 insertion(+), 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 0a37b31ec9874..1eb961512aaf8 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 @@ -39,7 +39,7 @@ import java.util.Set; import java.util.stream.Collectors; -final class Netty4HttpRequest extends RestRequest { +public final class Netty4HttpRequest extends RestRequest { private final FullHttpRequest request; private final Channel channel; From 859f2b654da3de14cfc98167efdb1856aaa5a9f0 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 13 Sep 2017 13:28:39 -0600 Subject: [PATCH 34/35] Fix test --- docs/build.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/build.gradle b/docs/build.gradle index 4cb82b97152f8..7c6a6af0a3cb7 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -46,6 +46,9 @@ project.rootProject.subprojects.findAll { it.parent.path == ':plugins' }.each { if (subproj.path.startsWith(':plugins:repository-')) { return } + if (subproj.path.startsWith(':plugins:transport-nio')) { + return + } subproj.afterEvaluate { // need to wait until the project has been configured integTestCluster { plugin subproj.path From bd0a75cb97bc7dc19795d829e0e8a28cced1a9da Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 15 Sep 2017 13:04:27 -0600 Subject: [PATCH 35/35] Update for bootstrap changes --- .../transport/nio/NioNotEnabledBootstrapCheck.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java index 2e6c99a99b0f7..e998f96da5ca1 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioNotEnabledBootstrapCheck.java @@ -20,17 +20,13 @@ package org.elasticsearch.transport.nio; import org.elasticsearch.bootstrap.BootstrapCheck; +import org.elasticsearch.bootstrap.BootstrapContext; public class NioNotEnabledBootstrapCheck implements BootstrapCheck { @Override - public boolean check() { - return true; - } - - @Override - public String errorMessage() { - return "The transport-nio plugin is experimental and not ready for production usage. It should not " + - "be enabled in production."; + public BootstrapCheckResult check(BootstrapContext context) { + return BootstrapCheckResult.failure("The transport-nio plugin is experimental and not ready for production usage. It should " + + "not be enabled in production."); } }