/* * Licensed to ElasticSearch and Shay Banon 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.memcached.netty; import org.elasticsearch.ElasticSearchException; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.netty.OpenChannelsHandler; import org.elasticsearch.common.netty.bootstrap.ServerBootstrap; import org.elasticsearch.common.netty.channel.Channel; import org.elasticsearch.common.netty.channel.ChannelPipeline; import org.elasticsearch.common.netty.channel.ChannelPipelineFactory; import org.elasticsearch.common.netty.channel.Channels; import org.elasticsearch.common.netty.channel.socket.nio.NioServerSocketChannelFactory; import org.elasticsearch.common.netty.channel.socket.oio.OioServerSocketChannelFactory; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.PortsRange; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.http.BindHttpException; import org.elasticsearch.memcached.MemcachedServerTransport; import org.elasticsearch.rest.RestController; import org.elasticsearch.transport.BindTransportException; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.common.network.NetworkService.TcpSettings.*; import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; /** */ public class NettyMemcachedServerTransport extends AbstractLifecycleComponent<MemcachedServerTransport> implements MemcachedServerTransport { private final RestController restController; private final NetworkService networkService; private final int workerCount; private final boolean blockingServer; private final String port; private final String bindHost; private final String publishHost; private final Boolean tcpNoDelay; private final Boolean tcpKeepAlive; private final Boolean reuseAddress; private final ByteSizeValue tcpSendBufferSize; private final ByteSizeValue tcpReceiveBufferSize; private volatile ServerBootstrap serverBootstrap; private volatile BoundTransportAddress boundAddress; private volatile Channel serverChannel; private volatile OpenChannelsHandler serverOpenChannels; @Inject public NettyMemcachedServerTransport(Settings settings, RestController restController, NetworkService networkService) { super(settings); this.restController = restController; this.networkService = networkService; this.workerCount = componentSettings.getAsInt("worker_count", Runtime.getRuntime().availableProcessors() * 2); this.blockingServer = componentSettings.getAsBoolean("memcached.blocking_server", settings.getAsBoolean(TCP_BLOCKING_SERVER, settings.getAsBoolean(TCP_BLOCKING, false))); this.port = componentSettings.get("port", settings.get("memcached.port", "11211-11311")); this.bindHost = componentSettings.get("bind_host"); this.publishHost = componentSettings.get("publish_host"); this.tcpNoDelay = componentSettings.getAsBoolean("tcp_no_delay", settings.getAsBoolean(TCP_NO_DELAY, true)); this.tcpKeepAlive = componentSettings.getAsBoolean("tcp_keep_alive", settings.getAsBoolean(TCP_KEEP_ALIVE, true)); this.reuseAddress = componentSettings.getAsBoolean("reuse_address", settings.getAsBoolean(TCP_REUSE_ADDRESS, NetworkUtils.defaultReuseAddress())); this.tcpSendBufferSize = componentSettings.getAsBytesSize("tcp_send_buffer_size", settings.getAsBytesSize(TCP_SEND_BUFFER_SIZE, TCP_DEFAULT_SEND_BUFFER_SIZE)); this.tcpReceiveBufferSize = componentSettings.getAsBytesSize("tcp_receive_buffer_size", settings.getAsBytesSize(TCP_RECEIVE_BUFFER_SIZE, TCP_DEFAULT_RECEIVE_BUFFER_SIZE)); } @Override public BoundTransportAddress boundAddress() { return boundAddress; } @Override protected void doStart() throws ElasticSearchException { this.serverOpenChannels = new OpenChannelsHandler(logger); if (blockingServer) { serverBootstrap = new ServerBootstrap(new OioServerSocketChannelFactory( Executors.newCachedThreadPool(daemonThreadFactory(settings, "memcached_server_boss")), Executors.newCachedThreadPool(daemonThreadFactory(settings, "memcached_server_worker")) )); } else { serverBootstrap = new ServerBootstrap(new NioServerSocketChannelFactory( Executors.newCachedThreadPool(daemonThreadFactory(settings, "memcached_server_boss")), Executors.newCachedThreadPool(daemonThreadFactory(settings, "memcached_server_worker")), workerCount)); } ChannelPipelineFactory pipelineFactory = new ChannelPipelineFactory() { @Override public ChannelPipeline getPipeline() throws Exception { ChannelPipeline pipeline = Channels.pipeline(); pipeline.addLast("openChannels", serverOpenChannels); pipeline.addLast("decoder", new MemcachedDecoder(logger)); pipeline.addLast("dispatcher", new MemcachedDispatcher(restController)); return pipeline; } }; serverBootstrap.setPipelineFactory(pipelineFactory); if (tcpNoDelay != null) { serverBootstrap.setOption("child.tcpNoDelay", tcpNoDelay); } if (tcpKeepAlive != null) { serverBootstrap.setOption("child.keepAlive", tcpKeepAlive); } if (tcpSendBufferSize != null) { serverBootstrap.setOption("child.sendBufferSize", tcpSendBufferSize.bytes()); } if (tcpReceiveBufferSize != null) { serverBootstrap.setOption("child.receiveBufferSize", tcpReceiveBufferSize.bytes()); } if (reuseAddress != null) { serverBootstrap.setOption("reuseAddress", reuseAddress); serverBootstrap.setOption("child.reuseAddress", reuseAddress); } // Bind and start to accept incoming connections. InetAddress hostAddressX; try { hostAddressX = networkService.resolveBindHostAddress(bindHost); } catch (IOException e) { throw new BindHttpException("Failed to resolve host [" + bindHost + "]", e); } final InetAddress hostAddress = hostAddressX; PortsRange portsRange = new PortsRange(port); final AtomicReference<Exception> lastException = new AtomicReference<Exception>(); boolean success = portsRange.iterate(new PortsRange.PortCallback() { @Override public boolean onPortNumber(int portNumber) { try { serverChannel = serverBootstrap.bind(new InetSocketAddress(hostAddress, portNumber)); } catch (Exception e) { lastException.set(e); return false; } return true; } }); if (!success) { throw new BindHttpException("Failed to bind to [" + port + "]", lastException.get()); } InetSocketAddress boundAddress = (InetSocketAddress) serverChannel.getLocalAddress(); InetSocketAddress publishAddress; try { publishAddress = new InetSocketAddress(networkService.resolvePublishHostAddress(publishHost), boundAddress.getPort()); } catch (Exception e) { throw new BindTransportException("Failed to resolve publish address", e); } this.boundAddress = new BoundTransportAddress(new InetSocketTransportAddress(boundAddress), new InetSocketTransportAddress(publishAddress)); } @Override protected void doStop() throws ElasticSearchException { if (serverChannel != null) { serverChannel.close().awaitUninterruptibly(); serverChannel = null; } if (serverOpenChannels != null) { serverOpenChannels.close(); serverOpenChannels = null; } if (serverBootstrap != null) { serverBootstrap.releaseExternalResources(); serverBootstrap = null; } } @Override protected void doClose() throws ElasticSearchException { } }