Skip to content

Commit

Permalink
[FLINK-25756] [connectors/opensearch] Dedicated Opensearch connectors
Browse files Browse the repository at this point in the history
Signed-off-by: Andriy Redko <andriy.redko@aiven.io>
  • Loading branch information
reta committed Nov 7, 2022
1 parent 3406727 commit 1f873a1
Show file tree
Hide file tree
Showing 59 changed files with 7,316 additions and 1 deletion.
192 changes: 192 additions & 0 deletions flink-connectors/flink-connector-opensearch/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,192 @@
<?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-connectors</artifactId>
<version>1.17-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-connector-opensearch</artifactId>
<name>Flink : Connectors : Opensearch</name>

<packaging>jar</packaging>

<!-- Allow users to pass custom connector versions -->
<properties>
<opensearch.version>1.3.0</opensearch.version>
</properties>

<dependencies>

<!-- Core -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</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</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<!-- Opensearch -->

<dependency>
<groupId>org.opensearch</groupId>
<artifactId>opensearch</artifactId>
<version>${opensearch.version}</version>
</dependency>

<dependency>
<groupId>org.opensearch.client</groupId>
<artifactId>opensearch-rest-high-level-client</artifactId>
<version>${opensearch.version}</version>
<exclusions>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore-nio</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- We need to include httpcore-nio again in the correct version due to the exclusion above -->
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore-nio</artifactId>
<version>4.4.12</version>
</dependency>

<!-- Tests -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

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

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

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

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

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

<!-- Table API integration tests -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-test-utils</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

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

<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
<scope>provided</scope>
</dependency>

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

<dependency>
<groupId>org.opensearch</groupId>
<artifactId>opensearch-testcontainers</artifactId>
<version>1.0.0</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<!-- Enforce single fork execution because of spawning
Opensearch cluster multiple times -->
<forkCount>1</forkCount>
</configuration>
</plugin>
</plugins>
</build>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.connector.opensearch.sink;

import org.apache.flink.annotation.Internal;
import org.apache.flink.util.function.TriFunction;

import org.opensearch.action.bulk.BulkProcessor;
import org.opensearch.client.RestHighLevelClient;

import java.io.Serializable;

@Internal
interface BulkProcessorBuilderFactory
extends Serializable,
TriFunction<
RestHighLevelClient,
BulkProcessorConfig,
BulkProcessor.Listener,
BulkProcessor.Builder> {}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.connector.opensearch.sink;

import java.io.Serializable;

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

class BulkProcessorConfig implements Serializable {

private final int bulkFlushMaxActions;
private final int bulkFlushMaxMb;
private final long bulkFlushInterval;
private final FlushBackoffType flushBackoffType;
private final int bulkFlushBackoffRetries;
private final long bulkFlushBackOffDelay;

BulkProcessorConfig(
int bulkFlushMaxActions,
int bulkFlushMaxMb,
long bulkFlushInterval,
FlushBackoffType flushBackoffType,
int bulkFlushBackoffRetries,
long bulkFlushBackOffDelay) {
this.bulkFlushMaxActions = bulkFlushMaxActions;
this.bulkFlushMaxMb = bulkFlushMaxMb;
this.bulkFlushInterval = bulkFlushInterval;
this.flushBackoffType = checkNotNull(flushBackoffType);
this.bulkFlushBackoffRetries = bulkFlushBackoffRetries;
this.bulkFlushBackOffDelay = bulkFlushBackOffDelay;
}

public int getBulkFlushMaxActions() {
return bulkFlushMaxActions;
}

public int getBulkFlushMaxMb() {
return bulkFlushMaxMb;
}

public long getBulkFlushInterval() {
return bulkFlushInterval;
}

public FlushBackoffType getFlushBackoffType() {
return flushBackoffType;
}

public int getBulkFlushBackoffRetries() {
return bulkFlushBackoffRetries;
}

public long getBulkFlushBackOffDelay() {
return bulkFlushBackOffDelay;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.connector.opensearch.sink;

import org.apache.flink.annotation.Internal;

import org.opensearch.action.ActionListener;
import org.opensearch.action.bulk.BulkRequest;
import org.opensearch.action.bulk.BulkResponse;

import java.util.function.BiConsumer;

/**
* {@link BulkRequestConsumerFactory} is used to bridge incompatible Opensearch Java API calls
* across different Opensearch versions.
*/
@Internal
interface BulkRequestConsumerFactory
extends BiConsumer<BulkRequest, ActionListener<BulkResponse>> {}
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.connector.opensearch.sink;

import org.apache.flink.annotation.PublicEvolving;

/**
* Used to control whether the sink should retry failed requests at all or with which kind back off
* strategy.
*/
@PublicEvolving
public enum FlushBackoffType {
/** After every failure, it waits a configured time until the retries are exhausted. */
CONSTANT,
/**
* After every failure, it waits initially the configured time and increases the waiting time
* exponentially until the retries are exhausted.
*/
EXPONENTIAL,
/** The failure is not retried. */
NONE,
}
Loading

0 comments on commit 1f873a1

Please sign in to comment.