You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by smarthi <gi...@git.apache.org> on 2016/03/15 09:34:53 UTC

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

GitHub user smarthi opened a pull request:

    https://github.com/apache/flink/pull/1792

    [WIP] FLINK-3115: Update ElasticSearch connector to 2.x

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/smarthi/flink ElasticSearch2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1792.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1792
    
----
commit b375b89491e42000b1d3efd62c4a1fe11ae61649
Author: smarthi <sm...@apache.org>
Date:   2015-12-29T09:55:57Z

    FLINK-3115: Update ElasticSearch connector to 2.x

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56822903
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,88 @@
    +<?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</artifactId>
    +		<version>1.1-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
    +	<name>flink-connector-elasticsearch2</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<!-- Allow users to pass custom connector versions -->
    +	<properties>
    +		<elasticsearch.version>2.2.1</elasticsearch.version>
    +	</properties>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_2.10</artifactId>
    +			<version>${project.version}</version>
    +			<scope>provided</scope>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.elasticsearch</groupId>
    +            <artifactId>elasticsearch</artifactId>
    +            <version>${elasticsearch.version}</version>
    +        </dependency>
    +
    +		<dependency>
    +			<groupId>com.fasterxml.jackson.core</groupId>
    +			<artifactId>jackson-core</artifactId>
    +			<version>2.7.2</version>
    --- End diff --
    
    I would suggest to use the `${jackson.version}` version here. This is bumping the jackson-dependency we're using.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56637824
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +input.addSink(new ElasticsearchSink<>(config, new ElasticSearchSinkFunction<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);
    +    }
    +    
    +    @Override
    +    public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
    +        indexer.add(createIndexRequest(element));
    --- End diff --
    
    The `createIndexRequest()` method is not called with the correct signature.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by smarthi <gi...@git.apache.org>.
Github user smarthi commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-198110028
  
    @rmetzger @mxm This PR is good to merge now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140971
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2Sink.java ---
    @@ -0,0 +1,310 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import com.google.common.collect.ImmutableList;
    +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.settings.Settings;
    +import org.elasticsearch.common.transport.InetSocketTransportAddress;
    +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.elasticsearch.node.NodeBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Sink that emits its input elements to an Elasticsearch cluster.
    + *
    + * <p>
    + * When using the first constructor {@link #ElasticSearch2Sink(java.util.Map, ElasticSearch2SinkFunction)}
    + * the sink will create a local {@link Node} for communicating with the
    + * Elasticsearch cluster. When using the second constructor
    + * {@link #ElasticSearch2Sink(java.util.Map, java.util.List, ElasticSearch2SinkFunction)} 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 RequestIndexer}. This is used to create an
    + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
    + * {@link RequestIndexer} for an example.
    + *
    + * @param <T> Type of the elements emitted by this sink
    + */
    +public class ElasticSearch2Sink<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(ElasticSearch2Sink.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<InetSocketAddress> transportAddresses;
    +
    +	/**
    +	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
    +	 */
    +	private final ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction;
    +
    +	/**
    +	 * 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;
    +
    +	/**
    +	 * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
    +	 */
    +	private transient RequestIndexer requestIndexer;
    +
    +	/**
    +	 * 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 ElasticSearch2Sink 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 elasticSearch2SinkFunction This is used to generate the IndexRequest from the incoming element
    +	 */
    +	public ElasticSearch2Sink(Map<String, String> userConfig, ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticSearch2SinkFunction = elasticSearch2SinkFunction;
    +		transportAddresses = null;
    +	}
    +
    +	/**
    +	 * Creates a new ElasticSearch2Sink 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 transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
    +	 * @param elasticSearch2SinkFunction This is used to generate the ActionRequest from the incoming element
    +	 *
    +	 */
    +	public ElasticSearch2Sink(Map<String, String> userConfig, List<InetSocketAddress> transportAddresses,
    +														ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticSearch2SinkFunction = elasticSearch2SinkFunction;
    +		this.transportAddresses = transportAddresses;
    +	}
    +
    +	/**
    +	 * 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) {
    +		List<TransportAddress> transportNodes = null;
    +		if (transportAddresses != null) {
    +			transportNodes = new ArrayList<>(transportAddresses.size());
    +			for (InetSocketAddress address : transportAddresses) {
    +				transportNodes.add(new InetSocketTransportAddress(address));
    +			}
    +		}
    +
    +		if (transportNodes == null) {
    +			// Make sure that we disable http access to our embedded node
    +			Settings settings =
    +					Settings.builder()
    +							.put(userConfig)
    +							.put("http.enabled", false)
    +							.build();
    +
    +			node = NodeBuilder.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 = Settings.settingsBuilder().put(userConfig).build();
    +
    +			TransportClient transportClient = TransportClient.builder().settings(settings).build();
    +			for (TransportAddress transport: transportNodes) {
    +				transportClient.addTransportAddress(transport);
    +			}
    +
    +			// verify that we actually are connected to a cluster
    +			ImmutableList<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes());
    +			if (nodes.isEmpty()) {
    +				throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
    +			}
    +
    +			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();
    +		requestIndexer = new BulkProcessorIndexer(bulkProcessor);
    +	}
    +
    +	@Override
    +	public void invoke(T element) {
    +		elasticSearch2SinkFunction.sink(element, getRuntimeContext(), requestIndexer);
    +	}
    +
    +	@Override
    +	public void close() {
    +		if (bulkProcessor != null) {
    +			bulkProcessor.close();
    +			bulkProcessor = null;
    +		}
    +
    +		if (client != null) {
    +			client.close();
    +		}
    +
    +		if (node != null) {
    +			node.close();
    +		}
    +
    --- End diff --
    
    are we sure that not of these 3 close() methods can throw an exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56823244
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java ---
    @@ -0,0 +1,309 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import com.google.common.collect.ImmutableList;
    +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.settings.Settings;
    +import org.elasticsearch.common.transport.InetSocketTransportAddress;
    +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.elasticsearch.node.NodeBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Sink that emits its input elements in bulk to an Elasticsearch cluster.
    + *
    + * <p>
    + * When using the first constructor {@link #ElasticsearchSink(java.util.Map, ElasticsearchSinkFunction)}
    + * the sink will create a local {@link Node} for communicating with the
    + * Elasticsearch cluster. When using the second constructor
    + * {@link #ElasticsearchSink(java.util.Map, java.util.List, ElasticsearchSinkFunction)} 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 RequestIndexer}. This is used to create an
    + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
    + * {@link RequestIndexer} 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<InetSocketAddress> transportAddresses;
    +
    +	/**
    +	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
    +	 */
    +	private final ElasticsearchSinkFunction<T> elasticsearchSinkFunction;
    +
    +	/**
    +	 * 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;
    +
    +	/**
    +	 * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
    +	 */
    +	private transient RequestIndexer requestIndexer;
    +
    +	/**
    +	 * 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 elasticsearchSinkFunction This is used to generate the IndexRequest from the incoming element
    +	 */
    +	public ElasticsearchSink(Map<String, String> userConfig, ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticsearchSinkFunction = elasticsearchSinkFunction;
    +		transportAddresses = 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 transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
    +	 * @param elasticsearchSinkFunction This is used to generate the ActionRequest from the incoming element
    +	 *
    +	 */
    +	public ElasticsearchSink(Map<String, String> userConfig, List<InetSocketAddress> transportAddresses, ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticsearchSinkFunction = elasticsearchSinkFunction;
    +		this.transportAddresses = transportAddresses;
    --- End diff --
    
    I would add a check that the transportAddresses are not null and their size > 0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-196754905
  
    Thank you for working on this Suneel! 
    I had some comments in the pull request.
    Also, I think we need to update the documentation to refer to this implementation as well!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56641925
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +input.addSink(new ElasticsearchSink<>(config, new ElasticSearchSinkFunction<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);
    +    }
    +    
    +    @Override
    +    public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
    +        indexer.add(createIndexRequest(element));
    +    }
    +}));
    +{% 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")
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +text.addSink(new ElasticsearchSink(config, new ElasticSearchSinkFunction[String] {
    +  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)
    +  }
    +  
    +  override def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) {
    +    indexer.add(element))
    +  }
    +}))
    +{% endhighlight %}
    +</div>
    +</div>
    +
    +Note 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 Action requests to the cluster.
    +This will buffer elements and Action Requests before sending 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
    +
    +The example code below does the same using a `TransportClient`:
    +
    +<div class="codetabs" markdown="1">
    +<div data-lang="java" markdown="1">
    +{% highlight java %}
    +DataStream<String> input = ...;
    +
    +Map<String, String> config = new HashMap<>();
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +List<TransportAddress> transports = new ArrayList<>();
    --- End diff --
    
    This has to be `List<InetSocketAddress>`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56823123
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java ---
    @@ -0,0 +1,309 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import com.google.common.collect.ImmutableList;
    +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.settings.Settings;
    +import org.elasticsearch.common.transport.InetSocketTransportAddress;
    +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.elasticsearch.node.NodeBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Sink that emits its input elements in bulk to an Elasticsearch cluster.
    + *
    + * <p>
    + * When using the first constructor {@link #ElasticsearchSink(java.util.Map, ElasticsearchSinkFunction)}
    + * the sink will create a local {@link Node} for communicating with the
    + * Elasticsearch cluster. When using the second constructor
    + * {@link #ElasticsearchSink(java.util.Map, java.util.List, ElasticsearchSinkFunction)} 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 RequestIndexer}. This is used to create an
    + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
    + * {@link RequestIndexer} 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<InetSocketAddress> transportAddresses;
    +
    +	/**
    +	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
    +	 */
    +	private final ElasticsearchSinkFunction<T> elasticsearchSinkFunction;
    +
    +	/**
    +	 * 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;
    +
    +	/**
    +	 * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
    +	 */
    +	private transient RequestIndexer requestIndexer;
    +
    +	/**
    +	 * 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 elasticsearchSinkFunction This is used to generate the IndexRequest from the incoming element
    +	 */
    +	public ElasticsearchSink(Map<String, String> userConfig, ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
    --- End diff --
    
    I wonder whether we should still offer the ctor variants for the embedded node variant. It won't work in practice due to our shading (some classes are duplicate in our classpath when running from an IDE).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-196751172
  
    I don't think we necessarily need the "2" in the names; since it is a module specific for ElasitSearch2 anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56637419
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    --- End diff --
    
    The variable is not defined anywhere


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140677
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java ---
    @@ -0,0 +1,28 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import org.elasticsearch.action.ActionRequest;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Function that creates an {@link ActionRequest} from an element in a Stream.
    --- End diff --
    
    javadoc doesn't seem applicable, since the functiont doesn't create an ActionRequest but receives one.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56642050
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +input.addSink(new ElasticsearchSink<>(config, new ElasticSearchSinkFunction<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);
    +    }
    +    
    +    @Override
    +    public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
    +        indexer.add(createIndexRequest(element));
    +    }
    +}));
    +{% 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")
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +text.addSink(new ElasticsearchSink(config, new ElasticSearchSinkFunction[String] {
    +  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)
    +  }
    +  
    +  override def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) {
    +    indexer.add(element))
    +  }
    +}))
    +{% endhighlight %}
    +</div>
    +</div>
    +
    +Note 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 Action requests to the cluster.
    +This will buffer elements and Action Requests before sending 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
    +
    +The example code below does the same using a `TransportClient`:
    +
    +<div class="codetabs" markdown="1">
    +<div data-lang="java" markdown="1">
    +{% highlight java %}
    +DataStream<String> input = ...;
    +
    +Map<String, String> config = new HashMap<>();
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +List<TransportAddress> transports = new ArrayList<>();
    +transports.add(new InetSocketAddress("node-1", 9300));
    +transports.add(new InetSocketAddress("node-2", 9300));
    +
    +input.addSink(new ElasticsearchSink(config, new ElasticSearchSinkFunction[String] {
    --- End diff --
    
    The transports need to be passed here as a second argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56141103
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2SinkITCase.java ---
    @@ -0,0 +1,213 @@
    +/**
    + * 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.elasticsearch2;
    +
    +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.Settings;
    +import org.elasticsearch.node.Node;
    +import org.elasticsearch.node.NodeBuilder;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.net.InetAddress;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class ElasticSearch2SinkITCase extends StreamingMultipleProgramsTestBase {
    +
    +  private static final int NUM_ELEMENTS = 20;
    --- End diff --
    
    This file is doing indentation using spaces. Flink uses tabs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56308507
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,107 @@
    +<?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</artifactId>
    +		<version>1.1-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
    +	<name>flink-connector-elasticsearch2</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<!-- Allow users to pass custom connector versions -->
    +	<properties>
    +		<elasticsearch.version>2.2.1</elasticsearch.version>
    +	</properties>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_2.10</artifactId>
    +			<version>${project.version}</version>
    +			<scope>provided</scope>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.elasticsearch</groupId>
    +            <artifactId>elasticsearch</artifactId>
    +            <version>${elasticsearch.version}</version>
    +        </dependency>
    +
    +		<dependency>
    +			<groupId>com.fasterxml.jackson.core</groupId>
    +			<artifactId>jackson-core</artifactId>
    +			<version>2.7.2</version>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-streaming-java_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +            <type>test-jar</type>
    +        </dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-tests_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +        </dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-test-utils_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +        </dependency>
    +
    +    </dependencies>
    +
    +	<build>
    +		<plugins>
    +			<plugin>
    +				<groupId>org.apache.maven.plugins</groupId>
    +				<artifactId>maven-surefire-plugin</artifactId>
    +				<configuration>
    +					<rerunFailingTestsCount>3</rerunFailingTestsCount>
    +				</configuration>
    +			</plugin>
    +			<plugin>
    +				<groupId>org.apache.maven.plugins</groupId>
    +				<artifactId>maven-failsafe-plugin</artifactId>
    +				<configuration>
    +					<rerunFailingTestsCount>3</rerunFailingTestsCount>
    --- End diff --
    
    This is necessary for the tests to complete reliably?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-200341175
  
    Merging the change ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140947
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,113 @@
    +<?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</artifactId>
    +		<version>1.0-SNAPSHOT</version>
    --- End diff --
    
    I think the wrong version here is the reason fro the failing test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140915
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2Sink.java ---
    @@ -0,0 +1,310 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import com.google.common.collect.ImmutableList;
    +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.settings.Settings;
    +import org.elasticsearch.common.transport.InetSocketTransportAddress;
    +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.elasticsearch.node.NodeBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Sink that emits its input elements to an Elasticsearch cluster.
    + *
    + * <p>
    + * When using the first constructor {@link #ElasticSearch2Sink(java.util.Map, ElasticSearch2SinkFunction)}
    + * the sink will create a local {@link Node} for communicating with the
    + * Elasticsearch cluster. When using the second constructor
    + * {@link #ElasticSearch2Sink(java.util.Map, java.util.List, ElasticSearch2SinkFunction)} 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 RequestIndexer}. This is used to create an
    + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
    + * {@link RequestIndexer} for an example.
    + *
    + * @param <T> Type of the elements emitted by this sink
    + */
    +public class ElasticSearch2Sink<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(ElasticSearch2Sink.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<InetSocketAddress> transportAddresses;
    +
    +	/**
    +	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
    +	 */
    +	private final ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction;
    +
    +	/**
    +	 * 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;
    +
    +	/**
    +	 * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
    +	 */
    +	private transient RequestIndexer requestIndexer;
    +
    +	/**
    +	 * 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 ElasticSearch2Sink 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 elasticSearch2SinkFunction This is used to generate the IndexRequest from the incoming element
    +	 */
    +	public ElasticSearch2Sink(Map<String, String> userConfig, ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticSearch2SinkFunction = elasticSearch2SinkFunction;
    +		transportAddresses = null;
    +	}
    +
    +	/**
    +	 * Creates a new ElasticSearch2Sink 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 transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
    +	 * @param elasticSearch2SinkFunction This is used to generate the ActionRequest from the incoming element
    +	 *
    +	 */
    +	public ElasticSearch2Sink(Map<String, String> userConfig, List<InetSocketAddress> transportAddresses,
    +														ElasticSearch2SinkFunction<T> elasticSearch2SinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticSearch2SinkFunction = elasticSearch2SinkFunction;
    +		this.transportAddresses = transportAddresses;
    +	}
    +
    +	/**
    +	 * 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) {
    +		List<TransportAddress> transportNodes = null;
    +		if (transportAddresses != null) {
    +			transportNodes = new ArrayList<>(transportAddresses.size());
    +			for (InetSocketAddress address : transportAddresses) {
    +				transportNodes.add(new InetSocketTransportAddress(address));
    +			}
    +		}
    +
    +		if (transportNodes == null) {
    +			// Make sure that we disable http access to our embedded node
    +			Settings settings =
    +					Settings.builder()
    +							.put(userConfig)
    +							.put("http.enabled", false)
    +							.build();
    +
    +			node = NodeBuilder.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 = Settings.settingsBuilder().put(userConfig).build();
    +
    +			TransportClient transportClient = TransportClient.builder().settings(settings).build();
    +			for (TransportAddress transport: transportNodes) {
    +				transportClient.addTransportAddress(transport);
    +			}
    +
    +			// verify that we actually are connected to a cluster
    +			ImmutableList<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes());
    +			if (nodes.isEmpty()) {
    +				throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
    +			}
    +
    +			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();
    +		requestIndexer = new BulkProcessorIndexer(bulkProcessor);
    +	}
    +
    +	@Override
    +	public void invoke(T element) {
    +		elasticSearch2SinkFunction.sink(element, getRuntimeContext(), requestIndexer);
    +	}
    +
    +	@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 ElasticSearch2Sink.", cause);
    +			} else {
    +				throw new RuntimeException("An error occured in ElasticSearch2Sink.");
    +
    --- End diff --
    
    blank line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-199286958
  
    Thanks alot for updating the pull request.
    I'm sorry that I have some more comments. We are approaching a mergable state :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56639817
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +input.addSink(new ElasticsearchSink<>(config, new ElasticSearchSinkFunction<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);
    +    }
    +    
    +    @Override
    +    public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
    +        indexer.add(createIndexRequest(element));
    +    }
    +}));
    +{% 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")
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +text.addSink(new ElasticsearchSink(config, new ElasticSearchSinkFunction[String] {
    +  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)
    +  }
    +  
    +  override def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) {
    +    indexer.add(element))
    +  }
    +}))
    +{% endhighlight %}
    +</div>
    +</div>
    +
    +Note 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 Action requests to the cluster.
    +This will buffer elements and Action Requests before sending 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
    +
    +The example code below does the same using a `TransportClient`:
    +
    +<div class="codetabs" markdown="1">
    +<div data-lang="java" markdown="1">
    +{% highlight java %}
    +DataStream<String> input = ...;
    +
    +Map<String, String> config = new HashMap<>();
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +List<TransportAddress> transports = new ArrayList<>();
    +transports.add(new InetSocketAddress("node-1", 9300));
    +transports.add(new InetSocketAddress("node-2", 9300));
    +
    +input.addSink(new ElasticsearchSink(config, new ElasticSearchSinkFunction[String] {
    +  public IndexRequest createIndexRequest(element: String, ctx: RuntimeContext): IndexRequest = {
    +    val json = new util.HashMap[String, AnyRef]
    --- End diff --
    
    This doesn't look like Java code


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zcox <gi...@git.apache.org>.
Github user zcox commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56574878
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java ---
    @@ -0,0 +1,211 @@
    +/**
    + * 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.elasticsearch2;
    +
    +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.Settings;
    +import org.elasticsearch.node.Node;
    +import org.elasticsearch.node.NodeBuilder;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +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.nodeBuilder()
    +				.settings(Settings.settingsBuilder()
    +						.put("path.home", dataDir.getParent())
    +						.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();
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
    +
    +		Map<String, String> config = new HashMap<>();
    +		// 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");
    +
    +		// need this with ElasticSearch v2.x
    +		config.put("path.home", dataDir.getParent());
    +
    +		source.addSink(new ElasticsearchSink<>(config, new TestElasticsearchSinkFunction()));
    +
    +		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.nodeBuilder()
    +				.settings(Settings.settingsBuilder()
    +						.put("path.home", dataDir.getParent())
    +						.put("http.enabled", false)
    +						.put("path.data", dataDir.getAbsolutePath()))
    +				// set a custom cluster name to verify that user config works correctly
    +				.clusterName("my-transport-client-cluster")
    +				.local(true)
    +				.node();
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
    +
    +		Map<String, String> config = new HashMap<>();
    +		// 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-transport-client-cluster");
    +
    +		// need this with ElasticSearch v2.x
    +		config.put("path.home", dataDir.getParent());
    +
    +		// connect to our local node
    +		config.put("node.local", "true");
    +
    +		// Can't use {@link TransportAddress} as its not Serializable in Elasticsearch 2.x
    +		List<InetSocketAddress> transports = new ArrayList<>();
    +		transports.add(new InetSocketAddress(9300));
    +
    +		source.addSink(new ElasticsearchSink<>(config, null, new TestElasticsearchSinkFunction()));
    --- End diff --
    
    These changes should get this test passing:
     1. remove the `.local(true)` call above from node settings
     1. remove the `config.put("node.local", "true");` line above
     1. `transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));`
     1. pass `transports` into `ElasticsearchSink` constructor instead of `null`
    
    @smarthi let me know if you want me to just PR your branch or if you want to make changes yourself.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56141058
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java ---
    @@ -0,0 +1,28 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import org.elasticsearch.action.ActionRequest;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Function that creates an {@link ActionRequest} from an element in a Stream.
    + */
    +public interface RequestIndexer extends Serializable {
    +void add(ActionRequest... actionRequests);
    --- End diff --
    
    missing indentation 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/1792


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56637157
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    --- End diff --
    
    missing `()`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56139476
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticSearch2SinkFunction.java ---
    @@ -0,0 +1,57 @@
    +/**
    + * 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.elasticsearch2;
    +
    +import org.apache.flink.api.common.functions.Function;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Function that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
    + *
    + * <p>
    + * This is used by {@link ElasticSearch2Sink}
    + * 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}
    + */
    +
    --- End diff --
    
    this javadoc is still about IndexRequestBuilder.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140658
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,113 @@
    +<?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</artifactId>
    +		<version>1.0-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
    +	<name>flink-connector-elasticsearch2</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<!-- Allow users to pass custom connector versions -->
    +	<properties>
    +		<elasticsearch.version>2.2.0</elasticsearch.version>
    +	</properties>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_2.10</artifactId>
    +			<version>${project.version}</version>
    +			<scope>provided</scope>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.elasticsearch</groupId>
    +            <artifactId>elasticsearch</artifactId>
    +            <version>${elasticsearch.version}</version>
    +        </dependency>
    +
    +        <dependency>
    +            <groupId>com.google.guava</groupId>
    +            <artifactId>guava</artifactId>
    +            <version>${guava.version}</version>
    +        </dependency>
    +
    --- End diff --
    
    have you tried this on a cluster? i recently worked on a sink that included a guava dependency, and another dependency that also required guava (in this case that would be elasticsearch). what happened was that guava was shaded away, and the other dependency failed because it could no longer find guava (or rather, only an incompatible version)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-198305510
  
    I tried running the connector, but I got the following error:
    
    ```
    11:53:19,133 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Out of order data generator -> (Flat Map, Sink: Unnamed) (1/1) switched to FAILED with exception.
    java.lang.IllegalStateException: failed to load bundle [file:/home/robert/flink-workdir/elasticsearch/elasticsearch-2.2.1/modules/lang-groovy/lang-groovy-2.2.1.jar, file:/home/robert/flink-workdir/elasticsearch/elasticsearch-2.2.1/modules/lang-groovy/groovy-all-2.4.4-indy.jar] due to jar hell
    	at org.elasticsearch.plugins.PluginsService.loadBundles(PluginsService.java:421)
    	at org.elasticsearch.plugins.PluginsService.<init>(PluginsService.java:115)
    	at org.elasticsearch.node.Node.<init>(Node.java:146)
    	at org.elasticsearch.node.Node.<init>(Node.java:128)
    	at org.elasticsearch.node.NodeBuilder.build(NodeBuilder.java:145)
    	at org.elasticsearch.node.NodeBuilder.node(NodeBuilder.java:152)
    	at org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:199)
    	at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
    	at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
    	at org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:317)
    	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:215)
    	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:564)
    	at java.lang.Thread.run(Thread.java:745)
    Caused by: java.lang.IllegalStateException: jar hell!
    class: org.apache.flink.shaded.com.google.common.annotations.Beta
    jar1: /home/robert/.m2/repository/org/apache/flink/flink-java/1.1-SNAPSHOT/flink-java-1.1-SNAPSHOT.jar
    jar2: /home/robert/.m2/repository/org/apache/flink/flink-core/1.1-SNAPSHOT/flink-core-1.1-SNAPSHOT.jar
    	at org.elasticsearch.bootstrap.JarHell.checkClass(JarHell.java:280)
    	at org.elasticsearch.bootstrap.JarHell.checkJarHell(JarHell.java:186)
    	at org.elasticsearch.plugins.PluginsService.loadBundles(PluginsService.java:419)
    	... 12 more
    ```
    How did you resolve that issue?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by smarthi <gi...@git.apache.org>.
Github user smarthi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56575375
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java ---
    @@ -0,0 +1,211 @@
    +/**
    + * 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.elasticsearch2;
    +
    +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.Settings;
    +import org.elasticsearch.node.Node;
    +import org.elasticsearch.node.NodeBuilder;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +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.nodeBuilder()
    +				.settings(Settings.settingsBuilder()
    +						.put("path.home", dataDir.getParent())
    +						.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();
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
    +
    +		Map<String, String> config = new HashMap<>();
    +		// 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");
    +
    +		// need this with ElasticSearch v2.x
    +		config.put("path.home", dataDir.getParent());
    +
    +		source.addSink(new ElasticsearchSink<>(config, new TestElasticsearchSinkFunction()));
    +
    +		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.nodeBuilder()
    +				.settings(Settings.settingsBuilder()
    +						.put("path.home", dataDir.getParent())
    +						.put("http.enabled", false)
    +						.put("path.data", dataDir.getAbsolutePath()))
    +				// set a custom cluster name to verify that user config works correctly
    +				.clusterName("my-transport-client-cluster")
    +				.local(true)
    +				.node();
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
    +
    +		Map<String, String> config = new HashMap<>();
    +		// 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-transport-client-cluster");
    +
    +		// need this with ElasticSearch v2.x
    +		config.put("path.home", dataDir.getParent());
    +
    +		// connect to our local node
    +		config.put("node.local", "true");
    +
    +		// Can't use {@link TransportAddress} as its not Serializable in Elasticsearch 2.x
    +		List<InetSocketAddress> transports = new ArrayList<>();
    +		transports.add(new InetSocketAddress(9300));
    +
    +		source.addSink(new ElasticsearchSink<>(config, null, new TestElasticsearchSinkFunction()));
    --- End diff --
    
    Thanks @zcox. the fix works, will update the PR


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-198313758
  
    I resolved above issue by not using the `embedded Node` mode. I suggest not to document it at all.
    There is currently no way of disabling the `JarHell` check of ES.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56823183
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java ---
    @@ -0,0 +1,309 @@
    +/*
    + * 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.elasticsearch2;
    +
    +import com.google.common.collect.ImmutableList;
    +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.settings.Settings;
    +import org.elasticsearch.common.transport.InetSocketTransportAddress;
    +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.elasticsearch.node.NodeBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * Sink that emits its input elements in bulk to an Elasticsearch cluster.
    + *
    + * <p>
    + * When using the first constructor {@link #ElasticsearchSink(java.util.Map, ElasticsearchSinkFunction)}
    + * the sink will create a local {@link Node} for communicating with the
    + * Elasticsearch cluster. When using the second constructor
    + * {@link #ElasticsearchSink(java.util.Map, java.util.List, ElasticsearchSinkFunction)} 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 RequestIndexer}. This is used to create an
    + * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
    + * {@link RequestIndexer} 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<InetSocketAddress> transportAddresses;
    +
    +	/**
    +	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
    +	 */
    +	private final ElasticsearchSinkFunction<T> elasticsearchSinkFunction;
    +
    +	/**
    +	 * 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;
    +
    +	/**
    +	 * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
    +	 */
    +	private transient RequestIndexer requestIndexer;
    +
    +	/**
    +	 * 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 elasticsearchSinkFunction This is used to generate the IndexRequest from the incoming element
    +	 */
    +	public ElasticsearchSink(Map<String, String> userConfig, ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticsearchSinkFunction = elasticsearchSinkFunction;
    +		transportAddresses = 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 transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
    +	 * @param elasticsearchSinkFunction This is used to generate the ActionRequest from the incoming element
    +	 *
    +	 */
    +	public ElasticsearchSink(Map<String, String> userConfig, List<InetSocketAddress> transportAddresses, ElasticsearchSinkFunction<T> elasticsearchSinkFunction) {
    +		this.userConfig = userConfig;
    +		this.elasticsearchSinkFunction = elasticsearchSinkFunction;
    +		this.transportAddresses = transportAddresses;
    +	}
    +
    +	/**
    +	 * 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) {
    +		List<TransportAddress> transportNodes = null;
    +		if (transportAddresses != null) {
    +			transportNodes = new ArrayList<>(transportAddresses.size());
    +			for (InetSocketAddress address : transportAddresses) {
    +				transportNodes.add(new InetSocketTransportAddress(address));
    +			}
    +		}
    +
    +		if (transportNodes == null || transportNodes.size() == 0) {
    --- End diff --
    
    If we ditch the embedded node variant, this case becomes irrelevant.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56140261
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticSearch2SinkFunction.java ---
    @@ -0,0 +1,57 @@
    +/**
    + * 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.elasticsearch2;
    +
    +import org.apache.flink.api.common.functions.Function;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Function that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
    + *
    + * <p>
    + * This is used by {@link ElasticSearch2Sink}
    + * 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 ElasticSearch2SinkFunction<T> extends Serializable, Function {
    +	void sink(T element, RuntimeContext ctx, RequestIndexer indexer);
    --- End diff --
    
    we should find a better name for this method, preferably consistent with other sinks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56637650
  
    --- Diff: docs/apis/streaming/connectors/elasticsearch2.md ---
    @@ -0,0 +1,207 @@
    +---
    +title: "Elasticsearch 2.x Connector"
    +
    +# Sub-level navigation
    +sub-nav-group: streaming
    +sub-nav-parent: connectors
    +sub-nav-pos: 2
    +sub-nav-title: Elasticsearch 2.x
    +---
    +<!--
    +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.
    +-->
    +
    +This connector provides a Sink that can write to an
    +[Elasticsearch 2.x](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-elasticsearch2{{ site.scala_version_suffix }}</artifactId>
    +  <version>{{site.version }}</version>
    +</dependency>
    +{% endhighlight %}
    +
    +Note that the streaming connectors are currently not part of the binary
    +distribution. See
    +[here]({{site.baseurl}}/apis/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 2.x
    +
    +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 2.x Sink
    +The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
    +
    +The sink communicates with Elasticsearch in 2 ways:
    +
    +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 = new HashMap<>;
    +// 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");
    +
    +// need this with ElasticSearch v2.x
    +config.put("path.home", dataDir.getParent());
    +
    +input.addSink(new ElasticsearchSink<>(config, new ElasticSearchSinkFunction<String>() {
    --- End diff --
    
    The function is called `ElasticsearchSinkFunction`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by smarthi <gi...@git.apache.org>.
Github user smarthi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56316199
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,107 @@
    +<?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</artifactId>
    +		<version>1.1-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
    +	<name>flink-connector-elasticsearch2</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<!-- Allow users to pass custom connector versions -->
    +	<properties>
    +		<elasticsearch.version>2.2.1</elasticsearch.version>
    +	</properties>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_2.10</artifactId>
    +			<version>${project.version}</version>
    +			<scope>provided</scope>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.elasticsearch</groupId>
    +            <artifactId>elasticsearch</artifactId>
    +            <version>${elasticsearch.version}</version>
    +        </dependency>
    +
    +		<dependency>
    +			<groupId>com.fasterxml.jackson.core</groupId>
    +			<artifactId>jackson-core</artifactId>
    +			<version>2.7.2</version>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-streaming-java_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +            <type>test-jar</type>
    +        </dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-tests_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +        </dependency>
    +
    +        <dependency>
    +            <groupId>org.apache.flink</groupId>
    +            <artifactId>flink-test-utils_2.10</artifactId>
    +            <version>${project.version}</version>
    +            <scope>test</scope>
    +        </dependency>
    +
    +    </dependencies>
    +
    +	<build>
    +		<plugins>
    +			<plugin>
    +				<groupId>org.apache.maven.plugins</groupId>
    +				<artifactId>maven-surefire-plugin</artifactId>
    +				<configuration>
    +					<rerunFailingTestsCount>3</rerunFailingTestsCount>
    +				</configuration>
    +			</plugin>
    +			<plugin>
    +				<groupId>org.apache.maven.plugins</groupId>
    +				<artifactId>maven-failsafe-plugin</artifactId>
    +				<configuration>
    +					<rerunFailingTestsCount>3</rerunFailingTestsCount>
    --- End diff --
    
    Nope, will remove


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [WIP] FLINK-3115: Update ElasticSearch connect...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the pull request:

    https://github.com/apache/flink/pull/1792#issuecomment-197245618
  
    Looks good to merge @smarthi! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: FLINK-3115: Update ElasticSearch connector to ...

Posted by smarthi <gi...@git.apache.org>.
Github user smarthi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1792#discussion_r56886134
  
    --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/pom.xml ---
    @@ -0,0 +1,88 @@
    +<?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</artifactId>
    +		<version>1.1-SNAPSHOT</version>
    +		<relativePath>..</relativePath>
    +	</parent>
    +
    +	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
    +	<name>flink-connector-elasticsearch2</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<!-- Allow users to pass custom connector versions -->
    +	<properties>
    +		<elasticsearch.version>2.2.1</elasticsearch.version>
    +	</properties>
    +
    +	<dependencies>
    +
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_2.10</artifactId>
    +			<version>${project.version}</version>
    +			<scope>provided</scope>
    +		</dependency>
    +
    +        <dependency>
    +            <groupId>org.elasticsearch</groupId>
    +            <artifactId>elasticsearch</artifactId>
    +            <version>${elasticsearch.version}</version>
    +        </dependency>
    +
    +		<dependency>
    +			<groupId>com.fasterxml.jackson.core</groupId>
    +			<artifactId>jackson-core</artifactId>
    +			<version>2.7.2</version>
    --- End diff --
    
    ElasticSearch 2.2.1 needs at least Jackson 2.6.2 or greater. Flink parent's presently at 2.4.2


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---