|
| 1 | +/* |
| 2 | + * Licensed to ElasticSearch and Shay Banon under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. ElasticSearch licenses this |
| 6 | + * file to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.elasticsearch.bulk.udp; |
| 21 | + |
| 22 | +import org.elasticsearch.ElasticSearchException; |
| 23 | +import org.elasticsearch.action.bulk.BulkProcessor; |
| 24 | +import org.elasticsearch.action.bulk.BulkRequest; |
| 25 | +import org.elasticsearch.action.bulk.BulkResponse; |
| 26 | +import org.elasticsearch.client.Client; |
| 27 | +import org.elasticsearch.common.bytes.ChannelBufferBytesReference; |
| 28 | +import org.elasticsearch.common.component.AbstractLifecycleComponent; |
| 29 | +import org.elasticsearch.common.inject.Inject; |
| 30 | +import org.elasticsearch.common.network.NetworkService; |
| 31 | +import org.elasticsearch.common.settings.Settings; |
| 32 | +import org.elasticsearch.common.transport.PortsRange; |
| 33 | +import org.elasticsearch.common.unit.ByteSizeUnit; |
| 34 | +import org.elasticsearch.common.unit.ByteSizeValue; |
| 35 | +import org.elasticsearch.common.unit.TimeValue; |
| 36 | +import org.jboss.netty.bootstrap.ConnectionlessBootstrap; |
| 37 | +import org.jboss.netty.buffer.ChannelBuffer; |
| 38 | +import org.jboss.netty.channel.*; |
| 39 | +import org.jboss.netty.channel.socket.nio.NioDatagramChannelFactory; |
| 40 | + |
| 41 | +import java.io.IOException; |
| 42 | +import java.net.BindException; |
| 43 | +import java.net.InetAddress; |
| 44 | +import java.net.InetSocketAddress; |
| 45 | +import java.util.concurrent.Executors; |
| 46 | +import java.util.concurrent.atomic.AtomicReference; |
| 47 | + |
| 48 | +import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; |
| 49 | + |
| 50 | +/** |
| 51 | + */ |
| 52 | +public class BulkUdpService extends AbstractLifecycleComponent<BulkUdpService> { |
| 53 | + |
| 54 | + private final Client client; |
| 55 | + private final NetworkService networkService; |
| 56 | + |
| 57 | + private final boolean enabled; |
| 58 | + |
| 59 | + final String host; |
| 60 | + final String port; |
| 61 | + |
| 62 | + final ByteSizeValue receiveBufferSize; |
| 63 | + final ReceiveBufferSizePredictorFactory receiveBufferSizePredictorFactory; |
| 64 | + final int bulkActions; |
| 65 | + final ByteSizeValue bulkSize; |
| 66 | + final TimeValue flushInterval; |
| 67 | + final int concurrentRequests; |
| 68 | + |
| 69 | + private BulkProcessor bulkProcessor; |
| 70 | + private ConnectionlessBootstrap bootstrap; |
| 71 | + private Channel channel; |
| 72 | + |
| 73 | + @Inject |
| 74 | + public BulkUdpService(Settings settings, Client client, NetworkService networkService) { |
| 75 | + super(settings); |
| 76 | + this.client = client; |
| 77 | + this.networkService = networkService; |
| 78 | + |
| 79 | + this.host = componentSettings.get("host"); |
| 80 | + this.port = componentSettings.get("port", "9700-9800"); |
| 81 | + |
| 82 | + this.bulkActions = componentSettings.getAsInt("bulk_actions", 1000); |
| 83 | + this.bulkSize = componentSettings.getAsBytesSize("bulk_size", new ByteSizeValue(5, ByteSizeUnit.MB)); |
| 84 | + this.flushInterval = componentSettings.getAsTime("flush_interval", TimeValue.timeValueSeconds(5)); |
| 85 | + this.concurrentRequests = componentSettings.getAsInt("concurrent_requests", 4); |
| 86 | + |
| 87 | + this.receiveBufferSize = componentSettings.getAsBytesSize("receive_buffer_size", new ByteSizeValue(10, ByteSizeUnit.MB)); |
| 88 | + this.receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory(componentSettings.getAsBytesSize("receive_predictor_size", receiveBufferSize).bytesAsInt()); |
| 89 | + |
| 90 | + this.enabled = componentSettings.getAsBoolean("enabled", false); |
| 91 | + |
| 92 | + logger.debug("using enabled [{}], host [{}], port [{}], bulk_actions [{}], bulk_size [{}], flush_interval [{}], concurrent_requests [{}]", |
| 93 | + enabled, host, port, bulkActions, bulkSize, flushInterval, concurrentRequests); |
| 94 | + } |
| 95 | + |
| 96 | + @Override |
| 97 | + protected void doStart() throws ElasticSearchException { |
| 98 | + if (!enabled) { |
| 99 | + return; |
| 100 | + } |
| 101 | + bulkProcessor = BulkProcessor.builder(client, new BulkListener()) |
| 102 | + .setBulkActions(bulkActions) |
| 103 | + .setBulkSize(bulkSize) |
| 104 | + .setFlushInterval(flushInterval) |
| 105 | + .setConcurrentRequests(concurrentRequests) |
| 106 | + .build(); |
| 107 | + |
| 108 | + |
| 109 | + bootstrap = new ConnectionlessBootstrap(new NioDatagramChannelFactory(Executors.newCachedThreadPool(daemonThreadFactory(settings, "bulk_udp_worker")))); |
| 110 | + |
| 111 | + bootstrap.setOption("receiveBufferSize", receiveBufferSize.bytesAsInt()); |
| 112 | + bootstrap.setOption("receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory); |
| 113 | + |
| 114 | + // Enable broadcast |
| 115 | + bootstrap.setOption("broadcast", "false"); |
| 116 | + |
| 117 | + bootstrap.setPipelineFactory(new ChannelPipelineFactory() { |
| 118 | + @Override |
| 119 | + public ChannelPipeline getPipeline() throws Exception { |
| 120 | + return Channels.pipeline(new Handler()); |
| 121 | + } |
| 122 | + }); |
| 123 | + |
| 124 | + |
| 125 | + InetAddress hostAddressX; |
| 126 | + try { |
| 127 | + hostAddressX = networkService.resolveBindHostAddress(host); |
| 128 | + } catch (IOException e) { |
| 129 | + logger.warn("failed to resolve host {}", e, host); |
| 130 | + return; |
| 131 | + } |
| 132 | + final InetAddress hostAddress = hostAddressX; |
| 133 | + |
| 134 | + PortsRange portsRange = new PortsRange(port); |
| 135 | + final AtomicReference<Exception> lastException = new AtomicReference<Exception>(); |
| 136 | + boolean success = portsRange.iterate(new PortsRange.PortCallback() { |
| 137 | + @Override |
| 138 | + public boolean onPortNumber(int portNumber) { |
| 139 | + try { |
| 140 | + channel = bootstrap.bind(new InetSocketAddress(hostAddress, portNumber)); |
| 141 | + } catch (Exception e) { |
| 142 | + lastException.set(e); |
| 143 | + return false; |
| 144 | + } |
| 145 | + return true; |
| 146 | + } |
| 147 | + }); |
| 148 | + if (!success) { |
| 149 | + logger.warn("failed to bind to {}/{}", lastException.get(), hostAddress, port); |
| 150 | + return; |
| 151 | + } |
| 152 | + |
| 153 | + logger.info("address {}", channel.getLocalAddress()); |
| 154 | + } |
| 155 | + |
| 156 | + @Override |
| 157 | + protected void doStop() throws ElasticSearchException { |
| 158 | + if (!enabled) { |
| 159 | + return; |
| 160 | + } |
| 161 | + if (channel != null) { |
| 162 | + channel.close().awaitUninterruptibly(); |
| 163 | + } |
| 164 | + if (bootstrap != null) { |
| 165 | + bootstrap.releaseExternalResources(); |
| 166 | + } |
| 167 | + bulkProcessor.close(); |
| 168 | + } |
| 169 | + |
| 170 | + @Override |
| 171 | + protected void doClose() throws ElasticSearchException { |
| 172 | + } |
| 173 | + |
| 174 | + class Handler extends SimpleChannelUpstreamHandler { |
| 175 | + |
| 176 | + @Override |
| 177 | + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { |
| 178 | + ChannelBuffer buffer = (ChannelBuffer) e.getMessage(); |
| 179 | + try { |
| 180 | + bulkProcessor.add(new ChannelBufferBytesReference(buffer), false, null, null); |
| 181 | + } catch (Exception e1) { |
| 182 | + logger.warn("failed to execute bulk request", e1); |
| 183 | + } |
| 184 | + } |
| 185 | + |
| 186 | + @Override |
| 187 | + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception { |
| 188 | + if (e.getCause() instanceof BindException) { |
| 189 | + // ignore, this happens when we retry binding to several ports, its fine if we fail... |
| 190 | + return; |
| 191 | + } |
| 192 | + logger.warn("failure caught", e.getCause()); |
| 193 | + } |
| 194 | + } |
| 195 | + |
| 196 | + class BulkListener implements BulkProcessor.Listener { |
| 197 | + |
| 198 | + @Override |
| 199 | + public void beforeBulk(long executionId, BulkRequest request) { |
| 200 | + if (logger.isTraceEnabled()) { |
| 201 | + logger.trace("[{}] executing [{}]/[{}]", executionId, request.numberOfActions(), new ByteSizeValue(request.estimatedSizeInBytes())); |
| 202 | + } |
| 203 | + } |
| 204 | + |
| 205 | + @Override |
| 206 | + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { |
| 207 | + if (logger.isTraceEnabled()) { |
| 208 | + logger.trace("[{}] executed [{}]/[{}], took [{}]", executionId, request.numberOfActions(), new ByteSizeValue(request.estimatedSizeInBytes()), response.took()); |
| 209 | + } |
| 210 | + if (response.hasFailures()) { |
| 211 | + logger.warn("[{}] failed to execute bulk request: {}", executionId, response.buildFailureMessage()); |
| 212 | + } |
| 213 | + } |
| 214 | + |
| 215 | + @Override |
| 216 | + public void afterBulk(long executionId, BulkRequest request, Throwable e) { |
| 217 | + logger.warn("[{}] failed to execute bulk request", e, executionId); |
| 218 | + } |
| 219 | + } |
| 220 | +} |
0 commit comments