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

[INLONG-10721][Sort] Add elasticsearch7 connector on flink 1.18 #10741

Merged
merged 2 commits into from
Aug 2, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
<?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>
<groupId>org.apache.inlong</groupId>
<artifactId>sort-connectors-v1.18</artifactId>
<version>1.14.0-SNAPSHOT</version>
</parent>

<artifactId>sort-connector-elasticsearch7-v1.18</artifactId>
<packaging>jar</packaging>
<name>Apache InLong - Sort-connector-elasticsearch7</name>

<properties>
<inlong.root.dir>${project.parent.parent.parent.parent.parent.basedir}</inlong.root.dir>
<elasticsearch.version>7.10.2</elasticsearch.version>
<elasticsearch.connector.version>3.0.1-1.17</elasticsearch.connector.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.inlong</groupId>
<artifactId>sort-flink-dependencies-v1.18</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
XiaoYou201 marked this conversation as resolved.
Show resolved Hide resolved
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-elasticsearch7</artifactId>
<version>${elasticsearch.connector.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-json</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.inlong</groupId>
<artifactId>sort-connector-elasticsearch-base-v1.18</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.elasticsearch.client</groupId>
<artifactId>elasticsearch-rest-client</artifactId>
<version>${elasticsearch.version}</version>
</dependency>
<dependency>
<groupId>org.elasticsearch.client</groupId>
<artifactId>elasticsearch-rest-high-level-client</artifactId>
<version>${elasticsearch.version}</version>
</dependency>
<dependency>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch</artifactId>
<version>${elasticsearch.version}</version>
<!--
FLINK-7133: Excluding all org.ow2.asm from elasticsearch dependencies because
1. from the POV of client they are optional,
2. the version configured by default at the time of writing this comment (1.7.1) depends on asm 4.1
and when it is shaded into elasticsearch-base artifact it conflicts with newer shaded versions of asm
resulting in errors at the runtime when application is executed locally, e.g. from IDE.
-->
<exclusions>
<exclusion>
<groupId>org.ow2.asm</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<!-- Shade all the dependencies to avoid conflicts -->
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>shade-flink</id>
<goals>
<goal>shade</goal>
</goals>
<phase>package</phase>
<configuration>
<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
<filters>
<filter>
<artifact>org.apache.inlong:sort-connector-*</artifact>
<includes>
<include>org/apache/inlong/**</include>
<include>META-INF/services/org.apache.flink.table.factories.Factory</include>
</includes>
</filter>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>log4j.properties</exclude>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.PluginXmlResourceTransformer" />
</transformers>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* 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.inlong.sort.elasticsearch7;

import org.apache.inlong.sort.elasticsearch.ElasticsearchApiCallBridge;
import org.apache.inlong.sort.elasticsearch.ElasticsearchSinkBase;
import org.apache.inlong.sort.elasticsearch.RequestIndexer;

import org.apache.flink.annotation.Internal;
import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory;
import org.apache.flink.util.Preconditions;
import org.apache.http.HttpHost;
import org.elasticsearch.action.bulk.BackoffPolicy;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkProcessor;
import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.client.RestClientBuilder;
import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.common.unit.TimeValue;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;

import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;

/** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. */
@Internal
public class Elasticsearch7ApiCallBridge
implements
ElasticsearchApiCallBridge<RestHighLevelClient> {

private static final long serialVersionUID = -5222683870097809633L;

private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class);

/** User-provided HTTP Host. */
private final List<HttpHost> httpHosts;

/** The factory to configure the rest client. */
private final RestClientFactory restClientFactory;

Elasticsearch7ApiCallBridge(List<HttpHost> httpHosts, RestClientFactory restClientFactory) {
Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty());
this.httpHosts = httpHosts;
this.restClientFactory = Preconditions.checkNotNull(restClientFactory);
}

@Override
public RestHighLevelClient createClient() {
RestClientBuilder builder =
RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]));
restClientFactory.configureRestClientBuilder(builder);

RestHighLevelClient rhlClient = new RestHighLevelClient(builder);

return rhlClient;
}

@Override
public BulkProcessor.Builder createBulkProcessorBuilder(
RestHighLevelClient client, BulkProcessor.Listener listener) {
return BulkProcessor.builder(
(request, bulkListener) -> client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener),
listener);
}

@Override
public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) {
if (!bulkItemResponse.isFailed()) {
return null;
} else {
return bulkItemResponse.getFailure().getCause();
}
}

@Override
public void configureBulkProcessorFlushInterval(
BulkProcessor.Builder builder, long flushIntervalMillis) {
builder.setFlushInterval(TimeValue.timeValueMillis(flushIntervalMillis));
}

@Override
public void configureBulkProcessorBackoff(
BulkProcessor.Builder builder,
@Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) {

BackoffPolicy backoffPolicy;
if (flushBackoffPolicy != null) {
switch (flushBackoffPolicy.getBackoffType()) {
case CONSTANT:
backoffPolicy =
BackoffPolicy.constantBackoff(
new TimeValue(flushBackoffPolicy.getDelayMillis()),
flushBackoffPolicy.getMaxRetryCount());
break;
case EXPONENTIAL:
default:
backoffPolicy =
BackoffPolicy.exponentialBackoff(
new TimeValue(flushBackoffPolicy.getDelayMillis()),
flushBackoffPolicy.getMaxRetryCount());
}
} else {
backoffPolicy = BackoffPolicy.noBackoff();
}

builder.setBackoffPolicy(backoffPolicy);
}

@Override
public RequestIndexer createBulkProcessorIndexer(
BulkProcessor bulkProcessor,
boolean flushOnCheckpoint,
AtomicLong numPendingRequestsRef) {
return new Elasticsearch7BulkProcessorIndexer(
bulkProcessor, flushOnCheckpoint, numPendingRequestsRef);
}

@Override
public void verifyClientConnection(RestHighLevelClient client) throws IOException {
if (LOG.isInfoEnabled()) {
LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts);
}

if (!client.ping(RequestOptions.DEFAULT)) {
throw new RuntimeException("There are no reachable Elasticsearch nodes!");
}

if (LOG.isInfoEnabled()) {
LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString());
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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.inlong.sort.elasticsearch7;

import org.apache.inlong.sort.elasticsearch.RequestIndexer;

import org.apache.flink.annotation.Internal;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.bulk.BulkProcessor;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.update.UpdateRequest;

import java.util.concurrent.atomic.AtomicLong;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest
* ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster.
*
* <p>Note: This class is binary compatible to Elasticsearch 7.
*/
@Internal
class Elasticsearch7BulkProcessorIndexer implements RequestIndexer {

private final BulkProcessor bulkProcessor;
private final boolean flushOnCheckpoint;
private final AtomicLong numPendingRequestsRef;

Elasticsearch7BulkProcessorIndexer(
BulkProcessor bulkProcessor,
boolean flushOnCheckpoint,
AtomicLong numPendingRequestsRef) {
this.bulkProcessor = checkNotNull(bulkProcessor);
this.flushOnCheckpoint = flushOnCheckpoint;
this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef);
}

@Override
public void add(DeleteRequest... deleteRequests) {
for (DeleteRequest deleteRequest : deleteRequests) {
if (flushOnCheckpoint) {
numPendingRequestsRef.getAndIncrement();
}
this.bulkProcessor.add(deleteRequest);
}
}

@Override
public void add(IndexRequest... indexRequests) {
for (IndexRequest indexRequest : indexRequests) {
if (flushOnCheckpoint) {
numPendingRequestsRef.getAndIncrement();
}
this.bulkProcessor.add(indexRequest);
}
}

@Override
public void add(UpdateRequest... updateRequests) {
for (UpdateRequest updateRequest : updateRequests) {
if (flushOnCheckpoint) {
numPendingRequestsRef.getAndIncrement();
}
this.bulkProcessor.add(updateRequest);
}
}
}
Loading
Loading