提交 c6391dbf 编写于 作者: A Aljoscha Krettek

[FLINK-2558] Add Streaming Connector for Elasticsearch

上级 5aa70f70
......@@ -1517,11 +1517,24 @@ Stream connectors
----------------
<!-- TODO: reintroduce flume -->
Connectors provide an interface for accessing data from various third party sources (message queues). Currently three connectors are natively supported, namely [Apache Kafka](https://kafka.apache.org/), [RabbitMQ](http://www.rabbitmq.com/) and the [Twitter Streaming API](https://dev.twitter.com/docs/streaming-apis).
Connectors provide code for interfacing with various third-party systems.
Typically the connector packages consist of a source and sink class
(with the exception of Twitter where only a source is provided and Elasticsearch
where only a sink is provided).
Typically the connector packages consist of a source and sink class (with the exception of Twitter where only a source is provided). To use these sources the user needs to pass Serialization/Deserialization schemas for the connectors for the desired types. (Or use some predefined ones)
Currently these systems are supported:
To run an application using one of these connectors, additional third party components are usually required to be installed and launched, e.g. the servers for the message queues. Further instructions for these can be found in the corresponding subsections. [Docker containers](#docker-containers-for-connectors) are also provided encapsulating these services to aid users getting started with connectors.
* [Apache Kafka](https://kafka.apache.org/) (sink/source)
* [Elasticsearch](https://elastic.co/) (sink)
* [RabbitMQ](http://www.rabbitmq.com/) (sink/source)
* [Twitter Streaming API](https://dev.twitter.com/docs/streaming-apis) (source)
To run an application using one of these connectors, additional third party
components are usually required to be installed and launched, e.g. the servers
for the message queues. Further instructions for these can be found in the
corresponding subsections. [Docker containers](#docker-containers-for-connectors)
are also provided encapsulating these services to aid users getting started
with connectors.
### Apache Kafka
......@@ -1661,6 +1674,165 @@ More about Kafka can be found [here](https://kafka.apache.org/documentation.html
[Back to top](#top)
### Elasticsearch
This connector provides a Sink that can write to an
[Elasticsearch](https://elastic.co/) Index. To use this connector, add the
following dependency to your project:
{% highlight xml %}
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-elasticsearch</artifactId>
<version>{{site.version }}</version>
</dependency>
{% endhighlight %}
Note that the streaming connectors are currently not part of the binary
distribution. See
[here](cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution)
for information about how to package the program with the libraries for
cluster execution.
#### Installing Elasticsearch
Instructions for setting up an Elasticsearch cluster can be found
[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup.html).
Make sure to set and remember a cluster name. This must be set when
creating a Sink for writing to your cluster
#### Elasticsearch Sink
The connector provides a Sink that can send data to an Elasticsearch Index.
The sink can use two different methods for communicating with Elasticsearch:
1. An embedded Node
2. The TransportClient
See [here](https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/client.html)
for information about the differences between the two modes.
This code shows how to create a sink that uses an embedded Node for
communication:
<div class="codetabs" markdown="1">
<div data-lang="java" markdown="1">
{% highlight java %}
DataStream<String> input = ...;
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put("bulk.flush.max.actions", "1");
config.put("cluster.name", "my-cluster-name");
input.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder<String>() {
@Override
public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
Map<String, Object> json = new HashMap<>();
json.put("data", element);
return Requests.indexRequest()
.index("my-index")
.type("my-type")
.source(json);
}
}));
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
{% highlight scala %}
val input: DataStream[String] = ...
val config = new util.HashMap[String, String]
config.put("bulk.flush.max.actions", "1")
config.put("cluster.name", "my-cluster-name")
text.addSink(new ElasticsearchSink(config, new IndexRequestBuilder[String] {
override def createIndexRequest(element: String, ctx: RuntimeContext): IndexRequest = {
val json = new util.HashMap[String, AnyRef]
json.put("data", element)
println("SENDING: " + element)
Requests.indexRequest.index("my-index").`type`("my-type").source(json)
}
}))
{% endhighlight %}
</div>
</div>
Not how a Map of Strings is used to configure the Sink. The configuration keys
are documented in the Elasticsearch documentation
[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/index.html).
Especially important is the `cluster.name` parameter that must correspond to
the name of your cluster.
Internally, the sink uses a `BulkProcessor` to send index requests to the cluster.
This will buffer elements before sending a request to the cluster. The behaviour of the
`BulkProcessor` can be configured using these config keys:
* **bulk.flush.max.actions**: Maximum amount of elements to buffer
* **bulk.flush.max.size.mb**: Maximum amount of data (in megabytes) to buffer
* **bulk.flush.interval.ms**: Interval at which to flush data regardless of the other two
settings in milliseconds
This example code does the same, but with a `TransportClient`:
<div class="codetabs" markdown="1">
<div data-lang="java" markdown="1">
{% highlight java %}
DataStream<String> input = ...;
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put("bulk.flush.max.actions", "1");
config.put("cluster.name", "my-cluster-name");
List<TransportAddress> transports = new ArrayList<String>();
transports.add(new InetSocketTransportAddress("node-1", 9300));
transports.add(new InetSocketTransportAddress("node-2", 9300));
input.addSink(new ElasticsearchSink<>(config, transports, new IndexRequestBuilder<String>() {
@Override
public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
Map<String, Object> json = new HashMap<>();
json.put("data", element);
return Requests.indexRequest()
.index("my-index")
.type("my-type")
.source(json);
}
}));
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
{% highlight scala %}
val input: DataStream[String] = ...
val config = new util.HashMap[String, String]
config.put("bulk.flush.max.actions", "1")
config.put("cluster.name", "my-cluster-name")
val transports = new ArrayList[String]
transports.add(new InetSocketTransportAddress("node-1", 9300))
transports.add(new InetSocketTransportAddress("node-2", 9300))
text.addSink(new ElasticsearchSink(config, transports, new IndexRequestBuilder[String] {
override def createIndexRequest(element: String, ctx: RuntimeContext): IndexRequest = {
val json = new util.HashMap[String, AnyRef]
json.put("data", element)
println("SENDING: " + element)
Requests.indexRequest.index("my-index").`type`("my-type").source(json)
}
}))
{% endhighlight %}
</div>
</div>
The difference is that we now need to provide a list of Elasticsearch Nodes
to which the sink should connect using a `TransportClient`.
More about information about Elasticsearch can be found [here](https://elastic.co).
[Back to top](#top)
### RabbitMQ
This connector provides access to data streams from [RabbitMQ](http://www.rabbitmq.com/). To use this connector, add the following dependency to your project:
......
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-connectors-parent</artifactId>
<version>0.10-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
<artifactId>flink-connector-elasticsearch</artifactId>
<name>flink-connector-elasticsearch</name>
<packaging>jar</packaging>
<!-- Allow users to pass custom connector versions -->
<properties>
<elasticsearch.version>1.7.1</elasticsearch.version>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch</artifactId>
<version>${elasticsearch.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-core</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-tests</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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.apache.flink.streaming.connectors.elasticsearch;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkProcessor;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.collect.ImmutableList;
import org.elasticsearch.common.settings.ImmutableSettings;
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.common.unit.TimeValue;
import org.elasticsearch.node.Node;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
/**
* Sink that emits its input elements to an Elasticsearch cluster.
*
* <p>
* When using the first constructor {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)}
* the sink will create a local {@link Node} for communicating with the
* Elasticsearch cluster. When using the second constructor
* {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)} a {@link TransportClient} will
* be used instead.
*
* <p>
* <b>Attention: </b> When using the {@code TransportClient} the sink will fail if no cluster
* can be connected to. With the {@code Node Client} the sink will block and wait for a cluster
* to come online.
*
* <p>
* The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
* the {@link Node} or {@link TransportClient}. The config keys can be found in the Elasticsearch
* documentation. An important setting is {@code cluster.name}, this should be set to the name
* of the cluster that the sink should emit to.
*
* <p>
* Internally, the sink will use a {@link BulkProcessor} to send {@link IndexRequest IndexRequests}.
* This will buffer elements before sending a request to the cluster. The behaviour of the
* {@code BulkProcessor} can be configured using these config keys:
* <ul>
* <li> {@code bulk.flush.max.actions}: Maximum amount of elements to buffer
* <li> {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer
* <li> {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two
* settings in milliseconds
* </ul>
*
* <p>
* You also have to provide an {@link IndexRequestBuilder}. This is used to create an
* {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
* {@link org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder} for an example.
*
* @param <T> Type of the elements emitted by this sink
*/
public class ElasticsearchSink<T> extends RichSinkFunction<T> {
public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions";
public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb";
public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms";
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
/**
* The user specified config map that we forward to Elasticsearch when we create the Client.
*/
private final Map<String, String> userConfig;
/**
* The list of nodes that the TransportClient should connect to. This is null if we are using
* an embedded Node to get a Client.
*/
private final List<TransportAddress> transportNodes;
/**
* The builder that is used to construct an {@link IndexRequest} from the incoming element.
*/
private final IndexRequestBuilder<T> indexRequestBuilder;
/**
* The embedded Node that is used to communicate with the Elasticsearch cluster. This is null
* if we are using a TransportClient.
*/
private transient Node node;
/**
* The Client that was either retrieved from a Node or is a TransportClient.
*/
private transient Client client;
/**
* Bulk processor that was created using the client
*/
private transient BulkProcessor bulkProcessor;
/**
* This is set from inside the BulkProcessor listener if there where failures in processing.
*/
private final AtomicBoolean hasFailure = new AtomicBoolean(false);
/**
* This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
*/
private final AtomicReference<Throwable> failureThrowable = new AtomicReference<>();
/**
* Creates a new ElasticsearchSink that connects to the cluster using an embedded Node.
*
* @param userConfig The map of user settings that are passed when constructing the Node and BulkProcessor
* @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
*/
public ElasticsearchSink(Map<String, String> userConfig, IndexRequestBuilder<T> indexRequestBuilder) {
this.userConfig = userConfig;
this.indexRequestBuilder = indexRequestBuilder;
transportNodes = null;
}
/**
* Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
*
* @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor
* @param transportNodes The Elasticsearch Nodes to which to connect using a {@code TransportClient}
* @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
*
*/
public ElasticsearchSink(Map<String, String> userConfig, List<TransportAddress> transportNodes, IndexRequestBuilder<T> indexRequestBuilder) {
this.userConfig = userConfig;
this.indexRequestBuilder = indexRequestBuilder;
this.transportNodes = transportNodes;
}
/**
* Initializes the connection to Elasticsearch by either creating an embedded
* {@link org.elasticsearch.node.Node} and retrieving the
* {@link org.elasticsearch.client.Client} from it or by creating a
* {@link org.elasticsearch.client.transport.TransportClient}.
*/
@Override
public void open(Configuration configuration) {
if (transportNodes == null) {
// Make sure that we disable http access to our embedded node
Settings settings =
ImmutableSettings.settingsBuilder()
.put(userConfig)
.put("http.enabled", false)
.build();
node =
nodeBuilder()
.settings(settings)
.client(true)
.data(false)
.node();
client = node.client();
if (LOG.isInfoEnabled()) {
LOG.info("Created Elasticsearch Client {} from embedded Node", client);
}
} else {
Settings settings = ImmutableSettings.settingsBuilder()
.put(userConfig)
.build();
TransportClient transportClient = new TransportClient(settings);
for (TransportAddress transport: transportNodes) {
transportClient.addTransportAddress(transport);
}
// verify that we actually are connected to a cluster
ImmutableList<DiscoveryNode> nodes = transportClient.connectedNodes();
if (nodes.isEmpty()) {
throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Connected to nodes: " + nodes.toString());
}
}
client = transportClient;
if (LOG.isInfoEnabled()) {
LOG.info("Created Elasticsearch TransportClient {}", client);
}
}
BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(
client,
new BulkProcessor.Listener() {
@Override
public void beforeBulk(long executionId,
BulkRequest request) {
}
@Override
public void afterBulk(long executionId,
BulkRequest request,
BulkResponse response) {
if (response.hasFailures()) {
for (BulkItemResponse itemResp : response.getItems()) {
if (itemResp.isFailed()) {
LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
}
}
hasFailure.set(true);
}
}
@Override
public void afterBulk(long executionId,
BulkRequest request,
Throwable failure) {
LOG.error(failure.getMessage());
failureThrowable.compareAndSet(null, failure);
hasFailure.set(true);
}
});
// This makes flush() blocking
bulkProcessorBuilder.setConcurrentRequests(0);
ParameterTool params = ParameterTool.fromMap(userConfig);
if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
}
if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
}
if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
}
bulkProcessor = bulkProcessorBuilder.build();
}
@Override
public void invoke(T element) {
IndexRequest indexRequest = indexRequestBuilder.createIndexRequest(element, getRuntimeContext());
if (LOG.isDebugEnabled()) {
LOG.debug("Emitting IndexRequest: {}", indexRequest);
}
bulkProcessor.add(indexRequest);
}
@Override
public void close() {
if (bulkProcessor != null) {
bulkProcessor.close();
bulkProcessor = null;
}
if (client != null) {
client.close();
}
if (node != null) {
node.close();
}
if (hasFailure.get()) {
Throwable cause = failureThrowable.get();
if (cause != null) {
throw new RuntimeException("An error occured in ElasticsearchSink.", cause);
} else {
throw new RuntimeException("An error occured in ElasticsearchSink.");
}
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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.apache.flink.streaming.connectors.elasticsearch;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.elasticsearch.action.index.IndexRequest;
import java.io.Serializable;
/**
* Function that creates an {@link IndexRequest} from an element in a Stream.
*
* <p>
* This is used by {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink}
* to prepare elements for sending them to Elasticsearch. See
* <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/index_.html">Index API</a>
* for information about how to format data for adding it to an Elasticsearch index.
*
* <p>
* Example:
*
* <pre>{@code
* private static class MyIndexRequestBuilder implements IndexRequestBuilder<String> {
*
* public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
* Map<String, Object> json = new HashMap<>();
* json.put("data", element);
*
* return Requests.indexRequest()
* .index("my-index")
* .type("my-type")
* .source(json);
* }
* }
* }</pre>
*
* @param <T> The type of the element handled by this {@code IndexRequestBuilder}
*/
public interface IndexRequestBuilder<T> extends Function, Serializable {
/**
* Creates an {@link org.elasticsearch.action.index.IndexRequest} from an element.
*
* @param element The element that needs to be turned in to an {@code IndexRequest}
* @param ctx The Flink {@link RuntimeContext} of the {@link ElasticsearchSink}
*
* @return The constructed {@code IndexRequest}
*/
IndexRequest createIndexRequest(T element, RuntimeContext ctx);
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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.apache.flink.streaming.connectors.elasticsearch.examples;
import com.google.common.collect.Maps;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink;
import org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Requests;
import java.util.HashMap;
import java.util.Map;
/**
* This example shows how to use the Elasticsearch Sink. Before running it you must ensure that
* you have a cluster names "elasticsearch" running or change the cluster name in the config map.
*/
public class ElasticsearchExample {
public static void main(String[] args) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<String> source = env.addSource(new SourceFunction<String>() {
private static final long serialVersionUID = 1L;
private volatile boolean running = true;
@Override
public void run(SourceContext<String> ctx) throws Exception {
for (int i = 0; i < 20 && running; i++) {
ctx.collect("message #" + i);
}
}
@Override
public void cancel() {
running = false;
}
});
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
source.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder<String>() {
@Override
public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
Map<String, Object> json = new HashMap<>();
json.put("data", element);
return Requests.indexRequest()
.index("my-index")
.type("my-type")
.source(json);
}
}));
env.execute("Elasticsearch Example");
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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.apache.flink.streaming.connectors.elasticsearch;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.node.Node;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
private static final int NUM_ELEMENTS = 20;
@ClassRule
public static TemporaryFolder tempFolder = new TemporaryFolder();
@Test
public void testNodeClient() throws Exception{
File dataDir = tempFolder.newFolder();
Node node = nodeBuilder()
.settings(ImmutableSettings.settingsBuilder()
.put("http.enabled", false)
.put("path.data", dataDir.getAbsolutePath()))
// set a custom cluster name to verify that user config works correctly
.clusterName("my-node-client-cluster")
.local(true)
.node();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
config.put("cluster.name", "my-node-client-cluster");
// connect to our local node
config.put("node.local", "true");
source.addSink(new ElasticsearchSink<>(config, new TestIndexRequestBuilder()));
env.execute("Elasticsearch Node Client Test");
// verify the results
Client client = node.client();
for (int i = 0; i < NUM_ELEMENTS; i++) {
GetResponse response = client.get(new GetRequest("my-index",
"my-type",
Integer.toString(i))).actionGet();
Assert.assertEquals("message #" + i, response.getSource().get("data"));
}
node.close();
}
@Test
public void testTransportClient() throws Exception {
File dataDir = tempFolder.newFolder();
Node node = nodeBuilder()
.settings(ImmutableSettings.settingsBuilder()
.put("http.enabled", false)
.put("path.data", dataDir.getAbsolutePath()))
// set a custom cluster name to verify that user config works correctly
.clusterName("my-node-client-cluster")
.local(true)
.node();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
config.put("cluster.name", "my-node-client-cluster");
// connect to our local node
config.put("node.local", "true");
List<TransportAddress> transports = Lists.newArrayList();
transports.add(new LocalTransportAddress("1"));
source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
env.execute("Elasticsearch TransportClient Test");
// verify the results
Client client = node.client();
for (int i = 0; i < NUM_ELEMENTS; i++) {
GetResponse response = client.get(new GetRequest("my-index",
"my-type",
Integer.toString(i))).actionGet();
Assert.assertEquals("message #" + i, response.getSource().get("data"));
}
node.close();
}
@Test(expected = JobExecutionException.class)
public void testTransportClientFails() throws Exception{
// this checks whether the TransportClient fails early when there is no cluster to
// connect to. We don't hava such as test for the Node Client version since that
// one will block and wait for a cluster to come online
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
Map<String, String> config = Maps.newHashMap();
// This instructs the sink to emit after every element, otherwise they would be buffered
config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
config.put("cluster.name", "my-node-client-cluster");
// connect to our local node
config.put("node.local", "true");
List<TransportAddress> transports = Lists.newArrayList();
transports.add(new LocalTransportAddress("1"));
source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
env.execute("Elasticsearch Node Client Test");
}
private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
private static final long serialVersionUID = 1L;
private volatile boolean running = true;
@Override
public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
for (int i = 0; i < NUM_ELEMENTS && running; i++) {
ctx.collect(Tuple2.of(i, "message #" + i));
}
}
@Override
public void cancel() {
running = false;
}
}
private static class TestIndexRequestBuilder implements IndexRequestBuilder<Tuple2<Integer, String>> {
private static final long serialVersionUID = 1L;
@Override
public IndexRequest createIndexRequest(Tuple2<Integer, String> element, RuntimeContext ctx) {
Map<String, Object> json = new HashMap<>();
json.put("data", element.f1);
return Requests.indexRequest()
.index("my-index")
.type("my-type")
.id(element.f0.toString())
.source(json);
}
}
}
################################################################################
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF 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.
################################################################################
log4j.rootLogger=OFF, testlogger
log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
log4j.appender.testlogger.target = System.err
log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
# suppress the irrelevant (wrong) warnings from the netty channel handler
log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file
<!--
~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. The ASF 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.
-->
<configuration>
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>
<root level="WARN">
<appender-ref ref="STDOUT"/>
</root>
<logger name="org.apache.flink.streaming" level="WARN"/>
</configuration>
\ No newline at end of file
......@@ -38,6 +38,7 @@ under the License.
<modules>
<module>flink-connector-flume</module>
<module>flink-connector-kafka</module>
<module>flink-connector-elasticsearch</module>
<module>flink-connector-rabbitmq</module>
<module>flink-connector-twitter</module>
</modules>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册