Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-13025] Elasticsearch 7.x support #9720

Closed
wants to merge 8 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
11 changes: 8 additions & 3 deletions docs/dev/connectors/elasticsearch.md
Expand Up @@ -53,7 +53,12 @@ of the Elasticsearch installation:
<tr>
<td>flink-connector-elasticsearch6{{ site.scala_version_suffix }}</td>
<td>1.6.0</td>
<td>6 and later versions</td>
<td>6.x</td>
</tr>
<tr>
<td>flink-connector-elasticsearch7{{ site.scala_version_suffix }}</td>
<td>1.10.0</td>
<td>7 and later versions</td>
</tr>
</tbody>
</table>
Expand Down Expand Up @@ -123,7 +128,7 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea
}
}));{% endhighlight %}
</div>
<div data-lang="java, Elasticsearch 6.x" markdown="1">
<div data-lang="java, Elasticsearch 6.x and above" markdown="1">
{% highlight java %}
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.streaming.api.datastream.DataStream;
Expand Down Expand Up @@ -226,7 +231,7 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc
}))
{% endhighlight %}
</div>
<div data-lang="scala, Elasticsearch 6.x" markdown="1">
<div data-lang="scala, Elasticsearch 6.x and above" markdown="1">
{% highlight scala %}
import org.apache.flink.api.common.functions.RuntimeContext
import org.apache.flink.streaming.api.datastream.DataStream
Expand Down
5 changes: 5 additions & 0 deletions docs/dev/connectors/elasticsearch.zh.md
Expand Up @@ -55,6 +55,11 @@ of the Elasticsearch installation:
<td>1.6.0</td>
<td>6 and later versions</td>
</tr>
<tr>
<td>flink-connector-elasticsearch7{{ site.scala_version_suffix }}</td>
<td>1.10.0</td>
<td>7 and later versions</td>
</tr>
</tbody>
</table>

Expand Down
Expand Up @@ -377,6 +377,7 @@ public interface RequestFactory extends Serializable {

/**
* Creates an update request to be added to a {@link RequestIndexer}.
* Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort.
*/
UpdateRequest createUpdateRequest(
String index,
Expand All @@ -387,6 +388,7 @@ UpdateRequest createUpdateRequest(

/**
* Creates an index request to be added to a {@link RequestIndexer}.
* Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort.
*/
IndexRequest createIndexRequest(
String index,
Expand All @@ -396,6 +398,7 @@ IndexRequest createIndexRequest(

/**
* Creates a delete request to be added to a {@link RequestIndexer}.
* Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort.
*/
DeleteRequest createDeleteRequest(
String index,
Expand Down
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.annotation.Internal;

import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Consumer;
Expand All @@ -36,6 +35,7 @@ public class ElasticsearchValidator extends ConnectorDescriptorValidator {

public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch";
public static final String CONNECTOR_VERSION_VALUE_6 = "6";
public static final String CONNECTOR_VERSION_VALUE_7 = "7";
public static final String CONNECTOR_HOSTS = "connector.hosts";
public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname";
public static final String CONNECTOR_HOSTS_PORT = "port";
Expand Down Expand Up @@ -79,7 +79,7 @@ private void validateVersion(DescriptorProperties properties) {
properties.validateEnumValues(
CONNECTOR_VERSION,
false,
Collections.singletonList(CONNECTOR_VERSION_VALUE_6));
Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7));
}

private void validateHosts(DescriptorProperties properties) {
Expand Down
8 changes: 0 additions & 8 deletions flink-connectors/flink-connector-elasticsearch6/pom.xml
Expand Up @@ -93,14 +93,6 @@ under the License.
<scope>provided</scope>
<optional>true</optional>
</dependency>
<!-- A planner dependency won't be necessary once FLIP-32 has been completed. -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<!-- test dependencies -->

Expand Down
190 changes: 190 additions & 0 deletions flink-connectors/flink-connector-elasticsearch7/pom.xml
@@ -0,0 +1,190 @@
<?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/xsd/maven-4.0.0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<artifactId>flink-connectors</artifactId>
<groupId>org.apache.flink</groupId>
<version>1.10-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-connector-elasticsearch7_${scala.binary.version}</artifactId>
<name>flink-connector-elasticsearch7</name>

<packaging>jar</packaging>

<!-- Allow users to pass custom connector versions -->
<properties>
<elasticsearch.version>7.3.2</elasticsearch.version>
</properties>

<dependencies>

<!-- core dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<exclusions>
<!-- Elasticsearch Java Client has been moved to a different module in 5.x -->
<exclusion>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- Dependency for Elasticsearch 7.x REST Client -->
<dependency>
<groupId>org.elasticsearch.client</groupId>
<artifactId>elasticsearch-rest-high-level-client</artifactId>
<version>${elasticsearch.version}</version>
</dependency>

<!-- Table ecosystem -->
<!-- Projects depending on this project won't depend on flink-table-*. -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<!-- test dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch</artifactId>
</exclusion>
</exclusions>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<!--
Including elasticsearch transport dependency for tests. Netty3 is not here anymore in 7.x
-->

<dependency>
<groupId>org.elasticsearch.client</groupId>
<artifactId>transport</artifactId>
<version>${elasticsearch.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.elasticsearch.plugin</groupId>
<artifactId>transport-netty4-client</artifactId>
<version>${elasticsearch.version}</version>
<scope>test</scope>
</dependency>

<!--
Including Log4j2 dependencies for tests is required for the
embedded Elasticsearch nodes used in tests to run correctly.
-->

<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
<version>2.9.1</version>
<scope>test</scope>
</dependency>

<!-- Elasticsearch table descriptor testing -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<!-- Elasticsearch table sink factory testing -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-json</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

</dependencies>

<build>
<plugins>
<!--
For the tests, we need to exclude the Log4j2 to slf4j adapter dependency
and let Elasticsearch directly use Log4j2, otherwise the embedded Elasticsearch node
used in tests will fail to work.

In other words, the connector jar is routing Elasticsearch 5.x's Log4j2 API's to SLF4J,
but for the test builds, we still stick to directly using Log4j2.
-->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>2.12.2</version>
<configuration>
<systemPropertyVariables>
<jna.nosys>true</jna.nosys>
</systemPropertyVariables>
<classpathDependencyExcludes>
<classpathDependencyExclude>org.apache.logging.log4j:log4j-to-slf4j</classpathDependencyExclude>
</classpathDependencyExcludes>
</configuration>
</plugin>
</plugins>
</build>

</project>