[FLINK-33859] Support OpenSearch v2
diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml
index 53cba91..91a91ae 100644
--- a/.github/workflows/push_pr.yml
+++ b/.github/workflows/push_pr.yml
@@ -28,11 +28,9 @@
compile_and_test:
strategy:
matrix:
- flink: [ 1.17-SNAPSHOT ]
- jdk: [ '8, 11' ]
+ flink: [ 1.18-SNAPSHOT ]
+ jdk: [ '8, 11, 17' ]
include:
- - flink: 1.18-SNAPSHOT
- jdk: '8, 11, 17'
- flink: 1.19-SNAPSHOT
jdk: '8, 11, 17, 21'
- flink: 1.20-SNAPSHOT
diff --git a/docs/content.zh/docs/connectors/datastream/opensearch.md b/docs/content.zh/docs/connectors/datastream/opensearch.md
index f9791af..019f427 100644
--- a/docs/content.zh/docs/connectors/datastream/opensearch.md
+++ b/docs/content.zh/docs/connectors/datastream/opensearch.md
@@ -66,7 +66,7 @@
```java
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder;
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.http.HttpHost;
diff --git a/docs/content/docs/connectors/datastream/opensearch.md b/docs/content/docs/connectors/datastream/opensearch.md
index dde159b..6498238 100644
--- a/docs/content/docs/connectors/datastream/opensearch.md
+++ b/docs/content/docs/connectors/datastream/opensearch.md
@@ -87,7 +87,7 @@
```java
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder;
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.http.HttpHost;
diff --git a/flink-connector-opensearch-base/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e b/flink-connector-opensearch-base/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
diff --git a/flink-connector-opensearch-base/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975 b/flink-connector-opensearch-base/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975
diff --git a/flink-connector-opensearch-base/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484 b/flink-connector-opensearch-base/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
diff --git a/flink-connector-opensearch-base/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967 b/flink-connector-opensearch-base/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967
new file mode 100644
index 0000000..dfcc175
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967
@@ -0,0 +1,3 @@
+Constructor <org.apache.flink.connector.opensearch.sink.BulkProcessorConfig.<init>(int, int, long, org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (BulkProcessorConfig.java:44)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchConfiguration.<init>(org.apache.flink.configuration.ReadableConfig)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchConfiguration.java:61)
+Method <org.apache.flink.connector.opensearch.table.IndexGeneratorFactory.createRuntimeIndexGenerator(java.lang.String, [Ljava.lang.String;, [Lorg.apache.flink.table.types.DataType;, org.apache.flink.connector.opensearch.table.IndexGeneratorFactory$IndexHelper, java.time.ZoneId)> has parameter of type <[Lorg.apache.flink.table.types.DataType;> in (IndexGeneratorFactory.java:0)
diff --git a/flink-connector-opensearch-base/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c b/flink-connector-opensearch-base/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c
diff --git a/flink-connector-opensearch-base/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09 b/flink-connector-opensearch-base/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
diff --git a/flink-connector-opensearch-base/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156 b/flink-connector-opensearch-base/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156
diff --git a/flink-connector-opensearch-base/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5 b/flink-connector-opensearch-base/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5
diff --git a/flink-connector-opensearch-base/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20 b/flink-connector-opensearch-base/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
diff --git a/flink-connector-opensearch-base/archunit-violations/stored.rules b/flink-connector-opensearch-base/archunit-violations/stored.rules
new file mode 100644
index 0000000..4d7b998
--- /dev/null
+++ b/flink-connector-opensearch-base/archunit-violations/stored.rules
@@ -0,0 +1,11 @@
+#
+#Thu Dec 14 21:20:35 CET 2023
+Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=879defc0-4d7e-43cb-ace6-1eb5aec9bac5
+Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=85d671bb-83fe-41ff-bc35-bd171908a156
+ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=4382f1f0-807a-45ff-97d8-42f72b6e9484
+Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=73099dba-7f06-4637-b2ad-b3c906aaaf1c
+Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
+Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=0c6c8466-9ce6-41a7-b6dd-947cc5702975
+Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
+Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
+Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=5c4a6228-f9cb-4828-9625-43c57d133967
diff --git a/flink-connector-opensearch-base/pom.xml b/flink-connector-opensearch-base/pom.xml
new file mode 100644
index 0000000..4fff4cf
--- /dev/null
+++ b/flink-connector-opensearch-base/pom.xml
@@ -0,0 +1,197 @@
+<?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-connector-opensearch-parent</artifactId>
+ <version>2.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <name>Flink : Connectors : Opensearch : Base</name>
+
+ <packaging>jar</packaging>
+
+ <!-- Allow users to pass custom connector versions -->
+ <properties>
+ <opensearch.version>1.3.14</opensearch.version>
+ <flink.connector.module.config><!-- required by
+ OpensearchSinkITCase --> --add-opens=java.base/java.lang=ALL-UNNAMED <!--
+ OpensearchSinkITCase --> --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED <!--
+ OpensearchDynamicSinkITCase --> --add-opens=java.base/java.util=ALL-UNNAMED </flink.connector.module.config>
+ </properties>
+
+ <dependencies>
+
+ <!-- Core -->
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-base</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.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>${flink.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>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-runtime</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.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>${flink.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>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Table API integration tests -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-test-utils</artifactId>
+ <version>${flink.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>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- ArchUit test dependencies -->
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-architecture-tests-test</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkResponseInspector.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/BulkResponseInspector.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkResponseInspector.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/BulkResponseInspector.java
diff --git a/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultFailureHandler.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultFailureHandler.java
new file mode 100644
index 0000000..c64c0e9
--- /dev/null
+++ b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultFailureHandler.java
@@ -0,0 +1,14 @@
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.util.FlinkRuntimeException;
+
+class DefaultFailureHandler implements FailureHandler {
+
+ @Override
+ public void onFailure(Throwable failure) {
+ if (failure instanceof FlinkRuntimeException) {
+ throw (FlinkRuntimeException) failure;
+ }
+ throw new FlinkRuntimeException(failure);
+ }
+}
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientConfig.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientConfig.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientConfig.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientConfig.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientFactory.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientFactory.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientFactory.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultRestClientFactory.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FailureHandler.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/FailureHandler.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FailureHandler.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/FailureHandler.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RestClientFactory.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/RestClientFactory.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RestClientFactory.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/sink/RestClientFactory.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java b/flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java
similarity index 100%
rename from flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java
rename to flink-connector-opensearch-base/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java
new file mode 100644
index 0000000..5b9811c
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** product code Architecture tests. */
+@AnalyzeClasses(
+ packages = "org.apache.flink.connector",
+ importOptions = {
+ ImportOption.DoNotIncludeTests.class,
+ ImportOption.DoNotIncludeArchives.class,
+ ImportOptions.ExcludeScalaImportOption.class,
+ ImportOptions.ExcludeShadedImportOption.class
+ })
+public class ProductionCodeArchitectureTest {
+
+ @ArchTest
+ public static final ArchTests COMMON_TESTS = ArchTests.in(ProductionCodeArchitectureBase.class);
+}
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
new file mode 100644
index 0000000..7e44a8c
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+ packages = {
+ "org.apache.flink.connector.opensearch",
+ "org.apache.flink.streaming.connectors.opensearch"
+ },
+ importOptions = {
+ ImportOption.OnlyIncludeTests.class,
+ ImportOptions.ExcludeScalaImportOption.class,
+ ImportOptions.ExcludeShadedImportOption.class
+ })
+public class TestCodeArchitectureTest {
+
+ @ArchTest
+ public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java
similarity index 100%
rename from flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java
rename to flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java
new file mode 100644
index 0000000..322ffc1
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchTestClient.java
@@ -0,0 +1,74 @@
+/*
+ * 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.opensearch.OpenSearchStatusException;
+import org.opensearch.action.get.GetRequest;
+import org.opensearch.action.get.GetResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestHighLevelClient;
+
+import java.io.IOException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class OpensearchTestClient {
+ private static final String DATA_FIELD_NAME = "data";
+ private final RestHighLevelClient client;
+
+ OpensearchTestClient(RestHighLevelClient client) {
+ this.client = client;
+ }
+
+ GetResponse getResponse(String index, int id) throws IOException {
+ return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT);
+ }
+
+ void assertThatIdsAreNotWritten(String index, int... ids) throws IOException {
+ for (final int id : ids) {
+ try {
+ final GetResponse response = getResponse(index, id);
+ assertThat(response.isExists())
+ .isFalse()
+ .as(String.format("Id %s is unexpectedly present.", id));
+ } catch (OpenSearchStatusException e) {
+ assertThat(e.status().getStatus()).isEqualTo(404);
+ }
+ }
+ }
+
+ void assertThatIdsAreWritten(String index, int... ids)
+ throws IOException, InterruptedException {
+ for (final int id : ids) {
+ GetResponse response;
+ do {
+ response = getResponse(index, id);
+ Thread.sleep(10);
+ } while (response.isSourceEmpty());
+ assertThat(response.getSource().get(DATA_FIELD_NAME)).isEqualTo(buildMessage(id));
+ }
+ }
+
+ String getDataFieldName() {
+ return DATA_FIELD_NAME;
+ }
+
+ static String buildMessage(int id) {
+ return "test-" + id;
+ }
+}
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/TestingSinkWriterMetricGroup.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/sink/TestingSinkWriterMetricGroup.java
similarity index 100%
rename from flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/TestingSinkWriterMetricGroup.java
rename to flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/sink/TestingSinkWriterMetricGroup.java
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java
new file mode 100644
index 0000000..7e88328
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java
@@ -0,0 +1,335 @@
+/*
+ * 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.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+
+import org.junit.jupiter.api.Test;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Suite tests for {@link IndexGenerator}. */
+class IndexGeneratorTest {
+
+ private static final List<String> fieldNames =
+ Arrays.asList(
+ "id",
+ "item",
+ "log_ts",
+ "log_date",
+ "order_timestamp",
+ "log_time",
+ "local_datetime",
+ "local_date",
+ "local_time",
+ "note",
+ "status");
+
+ private static final List<DataType> dataTypes =
+ Arrays.asList(
+ DataTypes.INT(),
+ DataTypes.STRING(),
+ DataTypes.BIGINT(),
+ DataTypes.DATE().bridgedTo(Date.class),
+ DataTypes.TIMESTAMP().bridgedTo(Timestamp.class),
+ DataTypes.TIME().bridgedTo(Time.class),
+ DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class),
+ DataTypes.DATE().bridgedTo(LocalDate.class),
+ DataTypes.TIME().bridgedTo(LocalTime.class),
+ DataTypes.STRING(),
+ DataTypes.BOOLEAN());
+
+ private static final List<RowData> rows =
+ Arrays.asList(
+ GenericRowData.of(
+ 1,
+ StringData.fromString("apple"),
+ Timestamp.valueOf("2020-03-18 12:12:14").getTime(),
+ (int) Date.valueOf("2020-03-18").toLocalDate().toEpochDay(),
+ TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-18 12:12:14")),
+ (int)
+ (Time.valueOf("12:12:14").toLocalTime().toNanoOfDay()
+ / 1_000_000L),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)),
+ (int) LocalDate.of(2020, 3, 18).toEpochDay(),
+ (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L),
+ "test1",
+ true),
+ GenericRowData.of(
+ 2,
+ StringData.fromString("peanut"),
+ Timestamp.valueOf("2020-03-19 12:22:14").getTime(),
+ (int) Date.valueOf("2020-03-19").toLocalDate().toEpochDay(),
+ TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-19 12:22:21")),
+ (int)
+ (Time.valueOf("12:22:21").toLocalTime().toNanoOfDay()
+ / 1_000_000L),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)),
+ (int) LocalDate.of(2020, 3, 19).toEpochDay(),
+ (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L),
+ "test2",
+ false));
+
+ @Test
+ public void testDynamicIndexFromTimestamp() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{order_timestamp|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("2020_03_18_12-14_index");
+ IndexGenerator indexGenerator1 =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{order_timestamp|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes);
+ indexGenerator1.open();
+ assertThat(indexGenerator1.generate(rows.get(1))).isEqualTo("2020_03_19_12_22_index");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalDateTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{local_datetime|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("2020_03_18_12-14_index");
+ IndexGenerator indexGenerator1 =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{local_datetime|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes);
+ indexGenerator1.open();
+ assertThat(indexGenerator1.generate(rows.get(1))).isEqualTo("2020_03_19_12_22_index");
+ }
+
+ @Test
+ public void testDynamicIndexFromDate() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{log_date|yyyy/MM/dd}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-2020/03/18");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-2020/03/19");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalDate() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date|yyyy/MM/dd}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-2020/03/18");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-2020/03/19");
+ }
+
+ @Test
+ public void testDynamicIndexFromTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{log_time|HH-mm}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12-12");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12-22");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_time|HH-mm}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12-13");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12-13");
+ }
+
+ @Test
+ public void testDynamicIndexDefaultFormat() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_time|}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12_13_14");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12_13_14");
+ }
+
+ @Test
+ public void testGeneralDynamicIndex() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator("index_{item}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("index_apple");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("index_peanut");
+ }
+
+ @Test
+ public void testStaticIndex() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator("my-index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index");
+ }
+
+ @Test
+ public void testUnknownField() {
+ String expectedExceptionMsg =
+ "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}',"
+ + " please check the field name.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{unknown_ts|yyyy-MM-dd}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedTimeType() {
+ String expectedExceptionMsg =
+ "Unsupported type 'INT' found in Opensearch dynamic index field, "
+ + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{id|yyyy-MM-dd}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedMultiParametersType() {
+ String expectedExceptionMsg =
+ "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported,"
+ + " only support single dynamic index pattern.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date}-{local_time}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testDynamicIndexUnsupportedFormat() {
+ String expectedExceptionMsg = "Unsupported field: HourOfDay";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date|yyyy/MM/dd HH:mm}", fieldNames, dataTypes);
+ } catch (UnsupportedTemporalTypeException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedIndexFieldType() {
+ String expectedExceptionMsg =
+ "Unsupported type BOOLEAN of index field, Supported types are:"
+ + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE,"
+ + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]";
+ try {
+ IndexGeneratorFactory.createIndexGenerator("index_{status}", fieldNames, dataTypes);
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testDynamicIndexFromSystemTime() {
+ List<String> supportedUseCases =
+ Arrays.asList(
+ "now()",
+ "NOW()",
+ "now( )",
+ "NOW(\t)",
+ "\t NOW( ) \t",
+ "current_timestamp",
+ "CURRENT_TIMESTAMP",
+ "\tcurrent_timestamp\t",
+ " current_timestamp ");
+
+ supportedUseCases.stream()
+ .forEach(
+ f -> {
+ DateTimeFormatter dateTimeFormatter =
+ DateTimeFormatter.ofPattern("yyyy_MM_dd");
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ String.format("my-index-{%s|yyyy_MM_dd}", f),
+ fieldNames,
+ dataTypes);
+ indexGenerator.open();
+ // The date may change during the running of the unit test.
+ // Generate expected index-name based on the current time
+ // before and after calling the generate method.
+ String expectedIndex1 =
+ "my-index-" + LocalDateTime.now().format(dateTimeFormatter);
+ String actualIndex = indexGenerator.generate(rows.get(1));
+ String expectedIndex2 =
+ "my-index-" + LocalDateTime.now().format(dateTimeFormatter);
+ assertThat(
+ actualIndex.equals(expectedIndex1)
+ || actualIndex.equals(expectedIndex2))
+ .isTrue();
+ });
+
+ List<String> invalidUseCases =
+ Arrays.asList(
+ "now",
+ "now(",
+ "NOW",
+ "NOW)",
+ "current_timestamp()",
+ "CURRENT_TIMESTAMP()",
+ "CURRENT_timestamp");
+ invalidUseCases.stream()
+ .forEach(
+ f -> {
+ String expectedExceptionMsg =
+ String.format(
+ "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}',"
+ + " please check the field name.",
+ f, f);
+ try {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ String.format("my-index-{%s|yyyy_MM_dd}", f),
+ fieldNames,
+ dataTypes);
+ indexGenerator.open();
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ });
+ }
+}
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java
new file mode 100644
index 0000000..31ef457
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+
+import org.junit.jupiter.api.Test;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link KeyExtractor}. */
+class KeyExtractorTest {
+ @Test
+ public void testSimpleKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.BIGINT().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+ assertThat(key).isEqualTo("12");
+ }
+
+ @Test
+ public void testNoPrimaryKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex = Collections.emptyList();
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+ assertThat(key).isNull();
+ }
+
+ @Test
+ public void testTwoFieldsKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.BIGINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 2, DataTypes.TIMESTAMP().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key =
+ keyExtractor.apply(
+ GenericRowData.of(
+ 12L,
+ StringData.fromString("ABCD"),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2012-12-12T12:12:12"))));
+ assertThat(key).isEqualTo("12_2012-12-12T12:12:12");
+ }
+
+ @Test
+ public void testAllTypesKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.TINYINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 1, DataTypes.SMALLINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 2, DataTypes.INT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 3, DataTypes.BIGINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 4, DataTypes.BOOLEAN().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 5, DataTypes.FLOAT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 6, DataTypes.DOUBLE().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 7, DataTypes.STRING().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 8, DataTypes.TIMESTAMP().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 9,
+ DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()
+ .notNull()
+ .getLogicalType()),
+ new LogicalTypeWithIndex(
+ 10, DataTypes.TIME().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 11, DataTypes.DATE().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key =
+ keyExtractor.apply(
+ GenericRowData.of(
+ (byte) 1,
+ (short) 2,
+ 3,
+ (long) 4,
+ true,
+ 1.0f,
+ 2.0d,
+ StringData.fromString("ABCD"),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2012-12-12T12:12:12")),
+ TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")),
+ (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000),
+ (int) LocalDate.parse("2015-05-15").toEpochDay()));
+ assertThat(key)
+ .isEqualTo(
+ "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15");
+ }
+}
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java
new file mode 100644
index 0000000..50da696
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java
@@ -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.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** A utility class for mocking {@link DynamicTableFactory.Context}. */
+class TestContext {
+
+ private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME()));
+
+ private final Map<String, String> options = new HashMap<>();
+
+ public static TestContext context() {
+ return new TestContext();
+ }
+
+ public TestContext withSchema(ResolvedSchema schema) {
+ this.schema = schema;
+ return this;
+ }
+
+ DynamicTableFactory.Context build() {
+ return new FactoryUtil.DefaultDynamicTableContext(
+ ObjectIdentifier.of("default", "default", "t1"),
+ new ResolvedCatalogTable(
+ CatalogTable.of(
+ Schema.newBuilder().fromResolvedSchema(schema).build(),
+ "mock context",
+ Collections.emptyList(),
+ options),
+ schema),
+ Collections.emptyMap(),
+ new Configuration(),
+ TestContext.class.getClassLoader(),
+ false);
+ }
+
+ public TestContext withOption(String key, String value) {
+ options.put(key, value);
+ return this;
+ }
+}
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
similarity index 90%
rename from flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
rename to flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
index 00b4b13..97f9c2b 100644
--- a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
@@ -22,5 +22,6 @@
* integration tests.
*/
public class DockerImageVersions {
- public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.3.6";
+ public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.3.14";
+ public static final String OPENSEARCH_2 = "opensearchproject/opensearch:2.11.1";
}
diff --git a/flink-connector-opensearch-base/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java b/flink-connector-opensearch-base/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java
new file mode 100644
index 0000000..293484d
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java
@@ -0,0 +1,144 @@
+/*
+ * 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.opensearch.testutils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.opensearch.OpensearchSinkFunction;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+
+import org.opensearch.action.get.GetRequest;
+import org.opensearch.action.get.GetResponse;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.common.xcontent.XContentBuilder;
+import org.opensearch.common.xcontent.XContentFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * This class contains utilities and a pre-defined source function and Opensearch Sink function used
+ * to simulate and verify data used in tests.
+ */
+public class SourceSinkDataTestKit {
+
+ private static final int NUM_ELEMENTS = 20;
+
+ private static final String DATA_PREFIX = "message #";
+ private static final String DATA_FIELD_NAME = "data";
+
+ /**
+ * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 -
+ * 20.
+ */
+ public static class TestDataSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+
+ private volatile boolean running = true;
+
+ @Override
+ public void run(SourceFunction.SourceContext<Tuple2<Integer, String>> ctx)
+ throws Exception {
+ for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+ ctx.collect(Tuple2.of(i, DATA_PREFIX + i));
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ }
+
+ public static OpensearchSinkFunction<Tuple2<Integer, String>> getJsonSinkFunction(
+ String index) {
+ return new TestOpensearchSinkFunction(index, XContentFactory::jsonBuilder);
+ }
+
+ public static OpensearchSinkFunction<Tuple2<Integer, String>> getSmileSinkFunction(
+ String index) {
+ return new TestOpensearchSinkFunction(index, XContentFactory::smileBuilder);
+ }
+
+ private static class TestOpensearchSinkFunction
+ implements OpensearchSinkFunction<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+
+ private final String index;
+ private final XContentBuilderProvider contentBuilderProvider;
+
+ /**
+ * Create the sink function, specifying a target Opensearch index.
+ *
+ * @param index Name of the target Opensearch index.
+ */
+ public TestOpensearchSinkFunction(
+ String index, XContentBuilderProvider contentBuilderProvider) {
+ this.index = index;
+ this.contentBuilderProvider = contentBuilderProvider;
+ }
+
+ public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
+ Map<String, Object> document = new HashMap<>();
+ document.put(DATA_FIELD_NAME, element.f1);
+
+ try {
+ return new IndexRequest(index)
+ .id(element.f0.toString())
+ .source(contentBuilderProvider.getBuilder().map(document));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void process(
+ Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
+ indexer.add(createIndexRequest(element));
+ }
+ }
+
+ /**
+ * Verify the results in an Opensearch index. The results must first be produced into the index
+ * using a {@link TestOpensearchSinkFunction};
+ *
+ * @param client The client to use to connect to Opensearch
+ * @param index The index to check
+ * @throws IOException IOException
+ */
+ public static void verifyProducedSinkData(RestHighLevelClient client, String index)
+ throws IOException {
+ for (int i = 0; i < NUM_ELEMENTS; i++) {
+ GetResponse response =
+ client.get(new GetRequest(index, Integer.toString(i)), RequestOptions.DEFAULT);
+ assertThat(response.getSource().get(DATA_FIELD_NAME)).isEqualTo(DATA_PREFIX + i);
+ }
+ }
+
+ @FunctionalInterface
+ private interface XContentBuilderProvider extends Serializable {
+ XContentBuilder getBuilder() throws IOException;
+ }
+}
diff --git a/flink-connector-opensearch-base/src/test/resources/archunit.properties b/flink-connector-opensearch-base/src/test/resources/archunit.properties
new file mode 100644
index 0000000..48011f9
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/resources/archunit.properties
@@ -0,0 +1,33 @@
+#
+# 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.
+#
+
+# By default we allow removing existing violations, but fail when new violations are added.
+freeze.store.default.allowStoreUpdate=true
+
+# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
+#freeze.store.default.allowStoreCreation=true
+
+# Enable this to add allow new violations to be recorded.
+# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
+# violation, please try to avoid creating the violation. If the violation was created due to a
+# shortcoming of the rule, file a JIRA issue so the rule can be improved.
+#freeze.refreeze=true
+
+freeze.store.default.path=archunit-violations
+
+archRule.failOnEmptyShould = false
diff --git a/flink-connector-opensearch-base/src/test/resources/log4j2-test.properties b/flink-connector-opensearch-base/src/test/resources/log4j2-test.properties
new file mode 100644
index 0000000..835c2ec
--- /dev/null
+++ b/flink-connector-opensearch-base/src/test/resources/log4j2-test.properties
@@ -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.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+rootLogger.level = OFF
+rootLogger.appenderRef.test.ref = TestLogger
+
+appender.testlogger.name = TestLogger
+appender.testlogger.type = CONSOLE
+appender.testlogger.target = SYSTEM_ERR
+appender.testlogger.layout.type = PatternLayout
+appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-connector-opensearch-e2e-tests/pom.xml b/flink-connector-opensearch-e2e-tests/pom.xml
index dd74292..95346ac 100644
--- a/flink-connector-opensearch-e2e-tests/pom.xml
+++ b/flink-connector-opensearch-e2e-tests/pom.xml
@@ -77,6 +77,23 @@
<artifactId>opensearch-testcontainers</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.opensearch.client</groupId>
+ <artifactId>opensearch-rest-high-level-client</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
</dependencies>
<build>
diff --git a/flink-connector-opensearch/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484 b/flink-connector-opensearch/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
index f346712..4f444b0 100644
--- a/flink-connector-opensearch/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
+++ b/flink-connector-opensearch/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
@@ -1,23 +1,5 @@
org.apache.flink.connector.opensearch.sink.OpensearchSinkITCase does not satisfy: only one of the following predicates match:\
* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.opensearch.sink.OpensearchWriterITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.opensearch.table.OpensearchDynamicSinkITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.opensearch.sink.OpensearchSinkITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
@@ -33,4 +15,4 @@
* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
\ No newline at end of file
+ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
diff --git a/flink-connector-opensearch/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967 b/flink-connector-opensearch/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967
new file mode 100644
index 0000000..4cfbaa4
--- /dev/null
+++ b/flink-connector-opensearch/archunit-violations/5c4a6228-f9cb-4828-9625-43c57d133967
@@ -0,0 +1,57 @@
+Constructor <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.<init>(org.apache.flink.connector.opensearch.sink.FailureHandler)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (DefaultBulkResponseInspector.java:45)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (OpensearchSink.java:75)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:74)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:76)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:77)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:78)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:79)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:80)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchSink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSink.java:81)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter$DefaultRequestIndexer.<init>(org.apache.flink.connector.opensearch.sink.OpensearchWriter, org.apache.flink.metrics.Counter)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:295)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:100)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:102)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:105)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:111)
+Constructor <org.apache.flink.connector.opensearch.sink.OpensearchWriter.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchWriter.java:113)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchDynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchDynamicSink.java:69)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchDynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchDynamicSink.java:70)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchDynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchDynamicSink.java:71)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchDynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchDynamicSink.java:72)
+Constructor <org.apache.flink.connector.opensearch.table.OpensearchDynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchDynamicSink.java:73)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearchEmitter.java:56)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearchEmitter.java:57)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearchEmitter.java:58)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearchEmitter.java:59)
+Field <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.failureHandler> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (DefaultBulkResponseInspector.java:0)
+Method <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.inspect(org.opensearch.action.bulk.BulkRequest, org.opensearch.action.bulk.BulkResponse)> calls method <org.apache.flink.util.ExceptionUtils.firstOrSuppressed(java.lang.Throwable, java.lang.Throwable)> in (DefaultBulkResponseInspector.java:68)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSink.getBulkResponseInspectorFactory()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (OpensearchSink.java:0)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSink.getDeliveryGuarantee()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (OpensearchSink.java:0)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.build()> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:339)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.build()> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:340)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.buildNetworkClientConfig()> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:362)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:194)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:195)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:198)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:200)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushInterval(long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:172)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushMaxActions(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:143)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkFlushMaxSizeMb(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:158)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setBulkResponseInspectorFactory(org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:329)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setConnectionPassword(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:227)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setConnectionPathPrefix(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:239)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setConnectionRequestTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:252)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setConnectionTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:264)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setConnectionUsername(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:215)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setDeliveryGuarantee(org.apache.flink.connector.base.DeliveryGuarantee)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:131)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setDeliveryGuarantee(org.apache.flink.connector.base.DeliveryGuarantee)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:128)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.InstantiationUtil.isSerializable(java.lang.Object)> in (OpensearchSinkBuilder.java:99)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:97)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:98)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setFailureHandler(org.apache.flink.connector.opensearch.sink.FailureHandler)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:314)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setHosts([Lorg.apache.http.HttpHost;)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:114)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setHosts([Lorg.apache.http.HttpHost;)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:115)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setRestClientFactory(org.apache.flink.connector.opensearch.sink.RestClientFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (OpensearchSinkBuilder.java:302)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder.setSocketTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (OpensearchSinkBuilder.java:277)
+Method <org.apache.flink.connector.opensearch.sink.OpensearchWriter.blockingFlushAllActions()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (OpensearchWriter.java:0)
+Method <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter$1.getMetricGroup()> calls constructor <org.apache.flink.metrics.groups.UnregisteredMetricsGroup.<init>()> in (RowOpensearchEmitter.java:69)
+Method <org.apache.flink.connector.opensearch.table.RowOpensearchEmitter$1.getUserCodeClassLoader()> calls method <org.apache.flink.util.SimpleUserCodeClassLoader.create(java.lang.ClassLoader)> in (RowOpensearchEmitter.java:74)
diff --git a/flink-connector-opensearch/archunit-violations/stored.rules b/flink-connector-opensearch/archunit-violations/stored.rules
index ce6d53d..4d7b998 100644
--- a/flink-connector-opensearch/archunit-violations/stored.rules
+++ b/flink-connector-opensearch/archunit-violations/stored.rules
@@ -1,5 +1,5 @@
#
-#Tue Nov 22 12:58:19 EST 2022
+#Thu Dec 14 21:20:35 CET 2023
Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=879defc0-4d7e-43cb-ace6-1eb5aec9bac5
Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=85d671bb-83fe-41ff-bc35-bd171908a156
ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=4382f1f0-807a-45ff-97d8-42f72b6e9484
@@ -8,3 +8,4 @@
Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=0c6c8466-9ce6-41a7-b6dd-947cc5702975
Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
+Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=5c4a6228-f9cb-4828-9625-43c57d133967
diff --git a/flink-connector-opensearch/pom.xml b/flink-connector-opensearch/pom.xml
index ffd40d7..f3e6edd 100644
--- a/flink-connector-opensearch/pom.xml
+++ b/flink-connector-opensearch/pom.xml
@@ -59,6 +59,18 @@
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
<!-- Table ecosystem -->
@@ -91,29 +103,6 @@
</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>${flink.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-runtime</artifactId>
- <version>${flink.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
-
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
@@ -169,7 +158,7 @@
<artifactId>opensearch-testcontainers</artifactId>
<scope>test</scope>
</dependency>
-
+
<!-- ArchUit test dependencies -->
<dependency>
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java
deleted file mode 100644
index 0eb3f68..0000000
--- a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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>> {}
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java
new file mode 100644
index 0000000..b23eadd
--- /dev/null
+++ b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java
@@ -0,0 +1,91 @@
+/*
+ * 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.VisibleForTesting;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.rest.RestStatus;
+
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A strict implementation that fails if either the whole bulk request failed or any of its actions.
+ */
+class DefaultBulkResponseInspector implements BulkResponseInspector {
+
+ @VisibleForTesting final FailureHandler failureHandler;
+
+ DefaultBulkResponseInspector() {
+ this(new DefaultFailureHandler());
+ }
+
+ DefaultBulkResponseInspector(FailureHandler failureHandler) {
+ this.failureHandler = checkNotNull(failureHandler);
+ }
+
+ @Override
+ public void inspect(BulkRequest request, BulkResponse response) {
+ if (!response.hasFailures()) {
+ return;
+ }
+
+ Throwable chainedFailures = null;
+ for (int i = 0; i < response.getItems().length; i++) {
+ final BulkItemResponse itemResponse = response.getItems()[i];
+ if (!itemResponse.isFailed()) {
+ continue;
+ }
+ final Throwable failure = itemResponse.getFailure().getCause();
+ if (failure == null) {
+ continue;
+ }
+ final RestStatus restStatus = itemResponse.getFailure().getStatus();
+ final DocWriteRequest<?> actionRequest = request.requests().get(i);
+
+ chainedFailures =
+ firstOrSuppressed(
+ wrapException(restStatus, failure, actionRequest), chainedFailures);
+ }
+ if (chainedFailures == null) {
+ return;
+ }
+ failureHandler.onFailure(chainedFailures);
+ }
+
+ private static Throwable wrapException(
+ RestStatus restStatus, Throwable rootFailure, DocWriteRequest<?> actionRequest) {
+ if (restStatus == null) {
+ return new FlinkRuntimeException(
+ String.format("Single action %s of bulk request failed.", actionRequest),
+ rootFailure);
+ } else {
+ return new FlinkRuntimeException(
+ String.format(
+ "Single action %s of bulk request failed with status %s.",
+ actionRequest, restStatus.getStatus()),
+ rootFailure);
+ }
+ }
+}
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java
index ce7b8ae..4fad113 100644
--- a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java
+++ b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java
@@ -21,8 +21,6 @@
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.connector.base.DeliveryGuarantee;
import org.apache.flink.connector.opensearch.sink.BulkResponseInspector.BulkResponseInspectorFactory;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultBulkResponseInspector;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultFailureHandler;
import org.apache.flink.util.InstantiationUtil;
import org.apache.http.HttpHost;
diff --git a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java
index d13973d..56ad900 100644
--- a/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java
+++ b/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java
@@ -18,6 +18,7 @@
package org.apache.flink.connector.opensearch.sink;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.connector.sink2.SinkWriter;
@@ -30,7 +31,6 @@
import org.opensearch.action.ActionListener;
import org.opensearch.action.DocWriteRequest;
import org.opensearch.action.bulk.BackoffPolicy;
-import org.opensearch.action.bulk.BulkItemResponse;
import org.opensearch.action.bulk.BulkProcessor;
import org.opensearch.action.bulk.BulkRequest;
import org.opensearch.action.bulk.BulkResponse;
@@ -50,8 +50,8 @@
import java.io.IOException;
import java.util.List;
+import java.util.function.BiConsumer;
-import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
import static org.apache.flink.util.Preconditions.checkNotNull;
class OpensearchWriter<IN> implements SinkWriter<IN> {
@@ -323,60 +323,7 @@
}
}
- /**
- * A strict implementation that fails if either the whole bulk request failed or any of its
- * actions.
- */
- static class DefaultBulkResponseInspector implements BulkResponseInspector {
-
- @VisibleForTesting final FailureHandler failureHandler;
-
- DefaultBulkResponseInspector() {
- this(new DefaultFailureHandler());
- }
-
- DefaultBulkResponseInspector(FailureHandler failureHandler) {
- this.failureHandler = checkNotNull(failureHandler);
- }
-
- @Override
- public void inspect(BulkRequest request, BulkResponse response) {
- if (!response.hasFailures()) {
- return;
- }
-
- Throwable chainedFailures = null;
- for (int i = 0; i < response.getItems().length; i++) {
- final BulkItemResponse itemResponse = response.getItems()[i];
- if (!itemResponse.isFailed()) {
- continue;
- }
- final Throwable failure = itemResponse.getFailure().getCause();
- if (failure == null) {
- continue;
- }
- final RestStatus restStatus = itemResponse.getFailure().getStatus();
- final DocWriteRequest<?> actionRequest = request.requests().get(i);
-
- chainedFailures =
- firstOrSuppressed(
- wrapException(restStatus, failure, actionRequest), chainedFailures);
- }
- if (chainedFailures == null) {
- return;
- }
- failureHandler.onFailure(chainedFailures);
- }
- }
-
- static class DefaultFailureHandler implements FailureHandler {
-
- @Override
- public void onFailure(Throwable failure) {
- if (failure instanceof FlinkRuntimeException) {
- throw (FlinkRuntimeException) failure;
- }
- throw new FlinkRuntimeException(failure);
- }
- }
+ @Internal
+ interface BulkRequestConsumerFactory
+ extends BiConsumer<BulkRequest, ActionListener<BulkResponse>> {}
}
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkInspectorTest.java b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkInspectorTest.java
index e230da7..f7ecae0 100644
--- a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkInspectorTest.java
+++ b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkInspectorTest.java
@@ -18,7 +18,6 @@
package org.apache.flink.connector.opensearch.sink;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultBulkResponseInspector;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.TestLoggerExtension;
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java
index ce4278b..cf35411 100644
--- a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java
+++ b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java
@@ -21,7 +21,6 @@
import org.apache.flink.api.connector.sink2.Sink.InitContext;
import org.apache.flink.connector.base.DeliveryGuarantee;
import org.apache.flink.connector.opensearch.sink.BulkResponseInspector.BulkResponseInspectorFactory;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultBulkResponseInspector;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java
index 838c6bd..fa7fc16 100644
--- a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java
+++ b/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchWriterITCase.java
@@ -21,8 +21,6 @@
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.opensearch.OpensearchUtil;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultBulkResponseInspector;
-import org.apache.flink.connector.opensearch.sink.OpensearchWriter.DefaultFailureHandler;
import org.apache.flink.connector.opensearch.test.DockerImageVersions;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.Gauge;
diff --git a/flink-connector-opensearch2-e2e-tests/pom.xml b/flink-connector-opensearch2-e2e-tests/pom.xml
new file mode 100644
index 0000000..e294ad7
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/pom.xml
@@ -0,0 +1,197 @@
+<?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.flink</groupId>
+ <artifactId>flink-connector-opensearch-parent</artifactId>
+ <version>2.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>flink-connector-opensearch2-e2e-tests</artifactId>
+ <name>Flink : E2E Tests : Opensearch : 2</name>
+ <packaging>jar</packaging>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-base</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch2</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch2</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-api</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch-testcontainers</artifactId>
+ <version>2.0.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.opensearch.client</groupId>
+ <artifactId>opensearch-rest-high-level-client</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-deploy-plugin</artifactId>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <finalName>opensearch-end-to-end-test</finalName>
+ <outputDirectory>dependencies</outputDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy</id>
+ <phase>pre-integration-test</phase>
+ <goals>
+ <goal>copy</goal>
+ </goals>
+ <configuration>
+ <artifactItems>
+ <artifactItem>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <destFileName>flink-connector-test-utils.jar</destFileName>
+ <type>jar</type>
+ <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
+ </artifactItem>
+ </artifactItems>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>default-test</id>
+ <phase>none</phase>
+ </execution>
+ <execution>
+ <id>integration-tests</id>
+ <phase>none</phase>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+ <profiles>
+ <profile>
+ <id>run-end-to-end-tests</id>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>end-to-end-tests</id>
+ <phase>integration-test</phase>
+ <goals>
+ <goal>test</goal>
+ </goals>
+ <configuration>
+ <includes>
+ <include>**/*.*</include>
+ </includes>
+ <systemPropertyVariables>
+ <moduleDir>${project.basedir}</moduleDir>
+ </systemPropertyVariables>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ </profiles>
+</project>
diff --git a/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/ComparableTuple2.java b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/ComparableTuple2.java
new file mode 100644
index 0000000..5b0d1c0
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/ComparableTuple2.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/** Variation of the {@link Tuple2} which implements {@link Comparable}. */
+public class ComparableTuple2<T0 extends Comparable<? super T0>, T1 extends Comparable<? super T1>>
+ extends Tuple2<T0, T1> implements Comparable<ComparableTuple2<T0, T1>> {
+ private static final long serialVersionUID = 1L;
+
+ public ComparableTuple2(T0 f0, T1 f1) {
+ super(f0, f1);
+ }
+
+ @Override
+ public int compareTo(ComparableTuple2<T0, T1> other) {
+ int d = this.f0.compareTo(other.f0);
+ if (d == 0) {
+ return this.f1.compareTo(other.f1);
+ }
+ return d;
+ }
+
+ /** Creates a new key-value pair. */
+ public static <K extends Comparable<? super K>, T1 extends Comparable<? super T1>>
+ ComparableTuple2<K, T1> of(K key, T1 value) {
+ return new ComparableTuple2<>(key, value);
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java
new file mode 100644
index 0000000..ade2f7b
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchSinkExample.java
@@ -0,0 +1,148 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler;
+import org.apache.flink.streaming.connectors.opensearch.Opensearch2Sink;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+import org.apache.flink.util.Collector;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.client.Requests;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** End to end test for OpensearchSink. */
+public class OpensearchSinkExample {
+
+ public static void main(String[] args) throws Exception {
+
+ final ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+ if (parameterTool.getNumberOfParameters() < 2) {
+ System.out.println(
+ "Missing parameters!\n" + "Usage: --numRecords <numRecords> --index <index>");
+ return;
+ }
+
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.enableCheckpointing(5000);
+
+ DataStream<Tuple2<String, String>> source =
+ env.generateSequence(0, parameterTool.getInt("numRecords") - 1)
+ .flatMap(
+ new FlatMapFunction<Long, Tuple2<String, String>>() {
+ @Override
+ public void flatMap(
+ Long value, Collector<Tuple2<String, String>> out) {
+ final String key = String.valueOf(value);
+ final String message = "message #" + value;
+ out.collect(Tuple2.of(key, message + "update #1"));
+ out.collect(Tuple2.of(key, message + "update #2"));
+ }
+ });
+
+ List<HttpHost> httpHosts = new ArrayList<>();
+ httpHosts.add(new HttpHost("127.0.0.1", 9200, "http"));
+
+ Opensearch2Sink.Builder<Tuple2<String, String>> osSinkBuilder =
+ new Opensearch2Sink.Builder<>(
+ httpHosts,
+ (Tuple2<String, String> element,
+ RuntimeContext ctx,
+ RequestIndexer indexer) -> {
+ indexer.add(createIndexRequest(element.f1, parameterTool));
+ indexer.add(createUpdateRequest(element, parameterTool));
+ });
+
+ osSinkBuilder.setFailureHandler(
+ new CustomFailureHandler(parameterTool.getRequired("index")));
+
+ // this instructs the sink to emit after every element, otherwise they would be buffered
+ osSinkBuilder.setBulkFlushMaxActions(1);
+
+ source.addSink(osSinkBuilder.build());
+
+ env.execute("Opensearch end to end sink test example");
+ }
+
+ private static class CustomFailureHandler implements ActionRequestFailureHandler {
+
+ private static final long serialVersionUID = 942269087742453482L;
+
+ private final String index;
+
+ CustomFailureHandler(String index) {
+ this.index = index;
+ }
+
+ @Override
+ public void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer)
+ throws Throwable {
+ if (action instanceof IndexRequest) {
+ Map<String, Object> json = new HashMap<>();
+ json.put("data", ((IndexRequest) action).source());
+
+ indexer.add(
+ Requests.indexRequest()
+ .index(index)
+ .id(((IndexRequest) action).id())
+ .source(json));
+ } else {
+ throw new IllegalStateException("unexpected");
+ }
+ }
+ }
+
+ private static IndexRequest createIndexRequest(String element, ParameterTool parameterTool) {
+ Map<String, Object> json = new HashMap<>();
+ json.put("data", element);
+
+ String index;
+ if (element.startsWith("message #15")) {
+ index = ":intentional invalid index:";
+ } else {
+ index = parameterTool.getRequired("index");
+ }
+
+ return Requests.indexRequest().index(index).id(element).source(json);
+ }
+
+ private static UpdateRequest createUpdateRequest(
+ Tuple2<String, String> element, ParameterTool parameterTool) {
+ Map<String, Object> json = new HashMap<>();
+ json.put("data", element.f1);
+
+ return new UpdateRequest(parameterTool.getRequired("index"), element.f0)
+ .doc(json)
+ .upsert(json);
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchTestEmitter.java b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchTestEmitter.java
new file mode 100644
index 0000000..be1048a
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchTestEmitter.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.opensearch.sink.OpensearchEmitter;
+import org.apache.flink.connector.opensearch.sink.RequestIndexer;
+
+import org.opensearch.action.update.UpdateRequest;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/** Test emitter for performing Opensearch indexing requests. */
+public class OpensearchTestEmitter implements OpensearchEmitter<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+ private final String indexName;
+
+ OpensearchTestEmitter(String indexName) {
+ this.indexName = indexName;
+ }
+
+ @Override
+ public void emit(
+ Tuple2<Integer, String> element, SinkWriter.Context context, RequestIndexer indexer) {
+ final Map<String, Object> json = new HashMap<>();
+ json.put("key", element.f0);
+ json.put("value", element.f1);
+
+ final UpdateRequest updateRequest =
+ new UpdateRequest(indexName, String.valueOf(element.f0)).doc(json).upsert(json);
+ indexer.add(updateRequest);
+ }
+}
diff --git a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/core/execution/CheckpointingMode.java
similarity index 63%
copy from flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
copy to flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/core/execution/CheckpointingMode.java
index 00b4b13..f003dd5 100644
--- a/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/test/DockerImageVersions.java
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/core/execution/CheckpointingMode.java
@@ -15,12 +15,18 @@
* limitations under the License.
*/
-package org.apache.flink.connector.opensearch.test;
+package org.apache.flink.core.execution;
+
+import org.apache.flink.streaming.tests.OpensearchSinkE2ECase;
/**
- * Utility class for defining the image names and versions of Docker containers used during the
- * integration tests.
+ * This is a copy of {@link CheckpointingMode} from flink-core module introduced in Flink 1.20. We
+ * need it here to make {@link OpensearchSinkE2ECase} compatible with earlier releases. Could be
+ * removed together with dropping support of Flink 1.19.
*/
-public class DockerImageVersions {
- public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.3.6";
+public enum CheckpointingMode {
+ EXACTLY_ONCE,
+ AT_LEAST_ONCE;
+
+ private CheckpointingMode() {}
}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchDataReader.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchDataReader.java
new file mode 100644
index 0000000..da353d1
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchDataReader.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import java.time.Duration;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Opensearch data reader. */
+public class OpensearchDataReader implements ExternalSystemDataReader<Tuple2<Integer, String>> {
+ private final OpensearchTestClient client;
+ private final String indexName;
+ private final int pageLength;
+
+ public OpensearchDataReader(OpensearchTestClient client, String indexName, int pageLength) {
+ this.client = checkNotNull(client);
+ this.indexName = checkNotNull(indexName);
+ this.pageLength = pageLength;
+ }
+
+ @Override
+ public List<Tuple2<Integer, String>> poll(Duration timeout) {
+ client.refreshIndex(indexName);
+ return client.fetchAll(indexName, "key", 0, pageLength, true);
+ }
+
+ @Override
+ public void close() throws Exception {
+ client.close();
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkE2ECase.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkE2ECase.java
new file mode 100644
index 0000000..eb14b0f
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkE2ECase.java
@@ -0,0 +1,155 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.connector.opensearch.test.DockerImageVersions;
+import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment;
+import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.test.resources.ResourceTestUtils;
+
+import org.opensearch.testcontainers.OpensearchContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.utility.DockerImageName;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.connector.testframe.utils.CollectIteratorAssertions.assertThat;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition;
+
+/** End to end test for OpensearchSink based on connector testing framework. */
+@SuppressWarnings("unused")
+public class OpensearchSinkE2ECase extends SinkTestSuiteBase<ComparableTuple2<Integer, String>> {
+ private static final Logger LOG = LoggerFactory.getLogger(OpensearchSinkE2ECase.class);
+ private static final int READER_RETRY_ATTEMPTS = 10;
+ private static final int READER_TIMEOUT = -1; // Not used
+
+ @TestSemantics
+ CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+ @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment(1, 6);
+
+ public OpensearchSinkE2ECase() throws Exception {}
+
+ @TestExternalSystem
+ DefaultContainerizedExternalSystem<?> opensearch =
+ DefaultContainerizedExternalSystem.builder()
+ .fromContainer(
+ new OpensearchContainer(
+ DockerImageName.parse(DockerImageVersions.OPENSEARCH_2))
+ .withEnv(
+ "cluster.routing.allocation.disk.threshold_enabled",
+ "false")
+ .withNetworkAliases("opensearch"))
+ .bindWithFlinkContainer(flink.getFlinkContainers().getJobManager())
+ .build();
+
+ @TestContext
+ OpensearchSinkExternalContextFactory contextFactory =
+ new OpensearchSinkExternalContextFactory(
+ (OpensearchContainer) opensearch.getContainer(),
+ Arrays.asList(
+ ResourceTestUtils.getResource(
+ "dependencies/opensearch-end-to-end-test.jar")
+ .toAbsolutePath()
+ .toUri()
+ .toURL(),
+ ResourceTestUtils.getResource(
+ "dependencies/flink-connector-test-utils.jar")
+ .toAbsolutePath()
+ .toUri()
+ .toURL()));
+
+ /** Could be removed together with dropping support of Flink 1.19. */
+ @Deprecated
+ protected void checkResultWithSemantic(
+ ExternalSystemDataReader<ComparableTuple2<Integer, String>> reader,
+ List<ComparableTuple2<Integer, String>> testData,
+ CheckpointingMode semantic)
+ throws Exception {
+ waitUntilCondition(
+ () -> {
+ try {
+ List<ComparableTuple2<Integer, String>> result =
+ reader.poll(Duration.ofMillis(READER_TIMEOUT));
+ assertThat(sort(result).iterator())
+ .matchesRecordsFromSource(
+ Collections.singletonList(sort(testData)), semantic);
+ return true;
+ } catch (Throwable t) {
+ LOG.warn("Polled results not as expected", t);
+ return false;
+ }
+ },
+ 5000,
+ READER_RETRY_ATTEMPTS);
+ }
+
+ protected void checkResultWithSemantic(
+ ExternalSystemDataReader<ComparableTuple2<Integer, String>> reader,
+ List<ComparableTuple2<Integer, String>> testData,
+ org.apache.flink.core.execution.CheckpointingMode semantic)
+ throws Exception {
+ waitUntilCondition(
+ () -> {
+ try {
+ List<ComparableTuple2<Integer, String>> result =
+ reader.poll(Duration.ofMillis(READER_TIMEOUT));
+ assertThat(sort(result).iterator())
+ .matchesRecordsFromSource(
+ Collections.singletonList(sort(testData)),
+ convertFromCheckpointingMode(semantic));
+ return true;
+ } catch (Throwable t) {
+ LOG.warn("Polled results not as expected", t);
+ return false;
+ }
+ },
+ 5000,
+ READER_RETRY_ATTEMPTS);
+ }
+
+ private static <T extends Comparable<T>> List<T> sort(List<T> list) {
+ Collections.sort(list);
+ return list;
+ }
+
+ /** Could be removed together with dropping support of Flink 1.19. */
+ @Deprecated
+ private static org.apache.flink.streaming.api.CheckpointingMode convertFromCheckpointingMode(
+ org.apache.flink.core.execution.CheckpointingMode semantic) {
+ switch (semantic) {
+ case EXACTLY_ONCE:
+ return org.apache.flink.streaming.api.CheckpointingMode.EXACTLY_ONCE;
+ case AT_LEAST_ONCE:
+ return org.apache.flink.streaming.api.CheckpointingMode.AT_LEAST_ONCE;
+ default:
+ throw new IllegalArgumentException("Unsupported semantic: " + semantic);
+ }
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContext.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContext.java
new file mode 100644
index 0000000..5d1c064
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContext.java
@@ -0,0 +1,138 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.http.HttpHost;
+
+import java.net.URL;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class OpensearchSinkExternalContext
+ implements DataStreamSinkV2ExternalContext<Tuple2<Integer, String>> {
+ /** The constant INDEX_NAME_PREFIX. */
+ private static final String INDEX_NAME_PREFIX = "os-index";
+
+ private static final int RANDOM_STRING_MAX_LENGTH = 50;
+ private static final int NUM_RECORDS_UPPER_BOUND = 500;
+ private static final int NUM_RECORDS_LOWER_BOUND = 100;
+ private static final int BULK_BUFFER = 100;
+ private static final int PAGE_LENGTH = NUM_RECORDS_UPPER_BOUND + 1;
+ /** The index name. */
+ private final String indexName;
+
+ /** The address reachable from Flink (internal to the testing environment). */
+ private final String addressInternal;
+
+ /** The connector jar paths. */
+ private final List<URL> connectorJarPaths;
+
+ /** The client. */
+ private final OpensearchTestClient client;
+
+ /**
+ * Instantiates a new Opensearch sink context base.
+ *
+ * @param addressExternal The address to access Opensearch from the host machine (outside of the
+ * containerized environment).
+ * @param addressInternal The address to access Opensearch from Flink. When running in a
+ * containerized environment, should correspond to the network alias that resolves within
+ * the environment's network together with the exposed port.
+ * @param connectorJarPaths The connector jar paths.
+ */
+ OpensearchSinkExternalContext(
+ String addressExternal, String addressInternal, List<URL> connectorJarPaths) {
+ this.addressInternal = checkNotNull(addressInternal);
+ this.connectorJarPaths = checkNotNull(connectorJarPaths);
+ this.client = new OpensearchTestClient(addressExternal);
+ this.indexName =
+ INDEX_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+ }
+
+ @Override
+ public Sink<Tuple2<Integer, String>> createSink(TestingSinkSettings sinkSettings)
+ throws UnsupportedOperationException {
+ client.createIndexIfDoesNotExist(indexName, 1, 0);
+ return new Opensearch2SinkBuilder<Tuple2<Integer, String>>()
+ .setHosts(HttpHost.create(addressInternal))
+ .setEmitter(new OpensearchTestEmitter(indexName))
+ .setBulkFlushMaxActions(BULK_BUFFER)
+ .build();
+ }
+
+ @Override
+ public ExternalSystemDataReader<Tuple2<Integer, String>> createSinkDataReader(
+ TestingSinkSettings sinkSettings) {
+ return new OpensearchDataReader(client, indexName, PAGE_LENGTH);
+ }
+
+ @Override
+ public List<Tuple2<Integer, String>> generateTestData(
+ TestingSinkSettings sinkSettings, long seed) {
+ Random random = new Random(seed);
+ int recordNum =
+ random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND)
+ + NUM_RECORDS_LOWER_BOUND;
+
+ return IntStream.range(0, recordNum)
+ .boxed()
+ .map(
+ i -> {
+ int valueLength = random.nextInt(RANDOM_STRING_MAX_LENGTH) + 1;
+ String value = RandomStringUtils.random(valueLength, true, true);
+ return ComparableTuple2.of(i, value);
+ })
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public void close() throws Exception {
+ client.deleteIndex(indexName);
+ }
+
+ @Override
+ public List<URL> getConnectorJarPaths() {
+ return connectorJarPaths;
+ }
+
+ @Override
+ public TypeInformation<Tuple2<Integer, String>> getProducedType() {
+ return TypeInformation.of(new TypeHint<Tuple2<Integer, String>>() {});
+ }
+
+ @Override
+ public String toString() {
+ return "Opensearch sink context.";
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContextFactory.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContextFactory.java
new file mode 100644
index 0000000..9e5efc0
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchSinkExternalContextFactory.java
@@ -0,0 +1,60 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalContextFactory;
+
+import org.opensearch.testcontainers.OpensearchContainer;
+
+import java.net.URL;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Opensearch sink external context factory. */
+class OpensearchSinkExternalContextFactory
+ implements ExternalContextFactory<OpensearchSinkExternalContext> {
+ /** The OpensearchContainer container. */
+ private final OpensearchContainer opensearchContainer;
+
+ /** The connector jars. */
+ private final List<URL> connectorJars;
+
+ /**
+ * Instantiates a new Opensearch sink external context factory.
+ *
+ * @param opensearchContainer The Opensearch container.
+ * @param connectorJars The connector jars.
+ */
+ OpensearchSinkExternalContextFactory(
+ OpensearchContainer opensearchContainer, List<URL> connectorJars) {
+ this.opensearchContainer = checkNotNull(opensearchContainer);
+ this.connectorJars = checkNotNull(connectorJars);
+ }
+
+ @Override
+ public OpensearchSinkExternalContext createExternalContext(String testName) {
+ return new OpensearchSinkExternalContext(
+ opensearchContainer.getHttpHostAddress(),
+ opensearchContainer.getNetworkAliases().get(0)
+ + ":"
+ + opensearchContainer.getExposedPorts().get(0),
+ connectorJars);
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchTestClient.java b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchTestClient.java
new file mode 100644
index 0000000..c18d1bd
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/java/org/apache/flink/streaming/tests/OpensearchTestClient.java
@@ -0,0 +1,141 @@
+/*
+ * 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.tests;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import org.apache.http.HttpHost;
+import org.opensearch.OpenSearchException;
+import org.opensearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.opensearch.action.admin.indices.refresh.RefreshRequest;
+import org.opensearch.action.search.SearchRequest;
+import org.opensearch.action.search.SearchResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.client.indices.CreateIndexRequest;
+import org.opensearch.client.indices.GetIndexRequest;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.search.SearchHit;
+import org.opensearch.search.builder.SearchSourceBuilder;
+import org.opensearch.search.sort.SortOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The type Opensearch test client. */
+public class OpensearchTestClient implements AutoCloseable {
+ private static final Logger LOG = LoggerFactory.getLogger(OpensearchTestClient.class);
+
+ private final RestHighLevelClient restClient;
+
+ /**
+ * Instantiates a new Opensearch client.
+ *
+ * @param address The address to access Opensearch from the host machine (outside of the
+ * containerized environment).
+ */
+ public OpensearchTestClient(String address) {
+ checkNotNull(address);
+ this.restClient = new RestHighLevelClient(RestClient.builder(HttpHost.create(address)));
+ }
+
+ public void deleteIndex(String indexName) {
+ DeleteIndexRequest request = new DeleteIndexRequest(indexName);
+ try {
+ restClient.indices().delete(request, RequestOptions.DEFAULT);
+ } catch (IOException e) {
+ LOG.error("Cannot delete index {}", indexName, e);
+ }
+ // This is needed to avoid race conditions between tests that reuse the same index
+ refreshIndex(indexName);
+ }
+
+ public void refreshIndex(String indexName) {
+ RefreshRequest refresh = new RefreshRequest(indexName);
+ try {
+ restClient.indices().refresh(refresh, RequestOptions.DEFAULT);
+ } catch (IOException e) {
+ LOG.error("Cannot delete index {}", indexName, e);
+ } catch (OpenSearchException e) {
+ if (e.status() == RestStatus.NOT_FOUND) {
+ LOG.info("Index {} not found", indexName);
+ }
+ }
+ }
+
+ public void createIndexIfDoesNotExist(String indexName, int shards, int replicas) {
+ GetIndexRequest request = new GetIndexRequest(indexName);
+ CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName);
+ createIndexRequest.settings(
+ Settings.builder()
+ .put("index.number_of_shards", shards)
+ .put("index.number_of_replicas", replicas));
+ try {
+ boolean exists = restClient.indices().exists(request, RequestOptions.DEFAULT);
+ if (!exists) {
+ restClient.indices().create(createIndexRequest, RequestOptions.DEFAULT);
+ } else {
+ LOG.info("Index already exists {}", indexName);
+ }
+ } catch (IOException e) {
+ LOG.error("Cannot create index {}", indexName, e);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ restClient.close();
+ }
+
+ public List<Tuple2<Integer, String>> fetchAll(
+ String indexName, String sortField, int from, int pageLength, boolean trackTotalHits) {
+ try {
+ SearchResponse response =
+ restClient.search(
+ new SearchRequest(indexName)
+ .source(
+ new SearchSourceBuilder()
+ .sort(sortField, SortOrder.ASC)
+ .from(from)
+ .size(pageLength)
+ .trackTotalHits(trackTotalHits)),
+ RequestOptions.DEFAULT);
+ SearchHit[] searchHits = response.getHits().getHits();
+ return Arrays.stream(searchHits)
+ .map(
+ searchHit ->
+ ComparableTuple2.of(
+ Integer.valueOf(searchHit.getId()),
+ searchHit.getSourceAsMap().get("value").toString()))
+ .collect(Collectors.toList());
+ } catch (IOException e) {
+ LOG.error("Fetching records failed", e);
+ return Collections.emptyList();
+ }
+ }
+}
diff --git a/flink-connector-opensearch2-e2e-tests/src/test/resources/log4j2-test.properties b/flink-connector-opensearch2-e2e-tests/src/test/resources/log4j2-test.properties
new file mode 100644
index 0000000..e48d6c0
--- /dev/null
+++ b/flink-connector-opensearch2-e2e-tests/src/test/resources/log4j2-test.properties
@@ -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.
+################################################################################
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+rootLogger.level=OFF
+rootLogger.appenderRef.test.ref=TestLogger
+appender.testlogger.name=TestLogger
+appender.testlogger.type=CONSOLE
+appender.testlogger.target=SYSTEM_ERR
+appender.testlogger.layout.type=PatternLayout
+appender.testlogger.layout.pattern=DOCKER> %m%n
+# It is recommended to uncomment these lines when enabling the logger. The below package used
+# by testcontainers is quite verbose
+logger.yarn.name=org.testcontainers.shaded.com.github.dockerjava.core
+logger.yarn.level=WARN
+logger.yarn.appenderRef.console.ref=TestLogger
+logger.testutils.name=org.apache.flink.runtime.testutils.CommonTestUtils
+logger.testutils.level=WARN
+logger.testutils.appenderRef.console.ref=TestLogger
+
diff --git a/flink-connector-opensearch2/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e b/flink-connector-opensearch2/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
diff --git a/flink-connector-opensearch2/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975 b/flink-connector-opensearch2/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/0c6c8466-9ce6-41a7-b6dd-947cc5702975
diff --git a/flink-connector-opensearch2/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484 b/flink-connector-opensearch2/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
new file mode 100644
index 0000000..b1649b2
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/4382f1f0-807a-45ff-97d8-42f72b6e9484
@@ -0,0 +1,18 @@
+org.apache.flink.connector.opensearch.sink.Opensearch2SinkITCase does not satisfy: only one of the following predicates match:\
+* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
+* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
+ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
+org.apache.flink.connector.opensearch.sink.Opensearch2WriterITCase does not satisfy: only one of the following predicates match:\
+* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
+* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
+ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
+org.apache.flink.connector.opensearch.table.Opensearch2DynamicSinkITCase does not satisfy: only one of the following predicates match:\
+* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
+* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
+ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
diff --git a/flink-connector-opensearch2/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c b/flink-connector-opensearch2/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/73099dba-7f06-4637-b2ad-b3c906aaaf1c
diff --git a/flink-connector-opensearch2/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09 b/flink-connector-opensearch2/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
diff --git a/flink-connector-opensearch2/archunit-violations/7f6b6846-355a-45b6-9c14-b65accd8d850 b/flink-connector-opensearch2/archunit-violations/7f6b6846-355a-45b6-9c14-b65accd8d850
new file mode 100644
index 0000000..9a63c4e
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/7f6b6846-355a-45b6-9c14-b65accd8d850
@@ -0,0 +1,60 @@
+Constructor <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.<init>(org.apache.flink.connector.opensearch.sink.FailureHandler)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (DefaultBulkResponseInspector.java:42)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (Opensearch2Sink.java:75)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:74)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:76)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:77)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:78)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:79)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:80)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, org.apache.flink.connector.base.DeliveryGuarantee, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Sink.java:81)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer$DefaultBulkResponseInspector.<init>(org.apache.flink.connector.opensearch.sink.FailureHandler)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:341)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer$DefaultRequestIndexer.<init>(org.apache.flink.connector.opensearch.sink.Opensearch2Writer, org.apache.flink.metrics.Counter)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:297)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:102)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:104)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:107)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:113)
+Constructor <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.<init>(java.util.List, org.apache.flink.connector.opensearch.sink.OpensearchEmitter, boolean, org.apache.flink.connector.opensearch.sink.BulkProcessorConfig, org.apache.flink.connector.opensearch.sink.NetworkClientConfig, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.connector.opensearch.sink.RestClientFactory, org.apache.flink.connector.opensearch.sink.BulkResponseInspector)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2Writer.java:115)
+Constructor <org.apache.flink.connector.opensearch.table.Opensearch2DynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2DynamicSink.java:69)
+Constructor <org.apache.flink.connector.opensearch.table.Opensearch2DynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2DynamicSink.java:70)
+Constructor <org.apache.flink.connector.opensearch.table.Opensearch2DynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2DynamicSink.java:71)
+Constructor <org.apache.flink.connector.opensearch.table.Opensearch2DynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2DynamicSink.java:72)
+Constructor <org.apache.flink.connector.opensearch.table.Opensearch2DynamicSink.<init>(org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.connector.opensearch.table.OpensearchConfiguration, java.util.List, org.apache.flink.table.types.DataType, java.lang.String, java.time.ZoneId)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2DynamicSink.java:73)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearch2Emitter.java:56)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearch2Emitter.java:57)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearch2Emitter.java:58)
+Constructor <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter.<init>(org.apache.flink.connector.opensearch.table.IndexGenerator, org.apache.flink.api.common.serialization.SerializationSchema, org.opensearch.common.xcontent.XContentType, java.util.function.Function)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (RowOpensearch2Emitter.java:59)
+Field <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.failureHandler> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (DefaultBulkResponseInspector.java:0)
+Field <org.apache.flink.connector.opensearch.sink.Opensearch2Writer$DefaultBulkResponseInspector.failureHandler> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (Opensearch2Writer.java:0)
+Method <org.apache.flink.connector.opensearch.sink.DefaultBulkResponseInspector.inspect(org.opensearch.action.bulk.BulkRequest, org.opensearch.action.bulk.BulkResponse)> calls method <org.apache.flink.util.ExceptionUtils.firstOrSuppressed(java.lang.Throwable, java.lang.Throwable)> in (DefaultBulkResponseInspector.java:65)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.getBulkResponseInspectorFactory()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (Opensearch2Sink.java:0)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2Sink.getDeliveryGuarantee()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (Opensearch2Sink.java:0)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.build()> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:339)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.build()> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:340)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.buildNetworkClientConfig()> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:362)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:194)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:195)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:198)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushBackoffStrategy(org.apache.flink.connector.opensearch.sink.FlushBackoffType, int, long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:200)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushInterval(long)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:172)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushMaxActions(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:143)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkFlushMaxSizeMb(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:158)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setBulkResponseInspectorFactory(org.apache.flink.connector.opensearch.sink.BulkResponseInspector$BulkResponseInspectorFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:329)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setConnectionPassword(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:227)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setConnectionPathPrefix(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:239)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setConnectionRequestTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:252)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setConnectionTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:264)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setConnectionUsername(java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:215)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setDeliveryGuarantee(org.apache.flink.connector.base.DeliveryGuarantee)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:131)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setDeliveryGuarantee(org.apache.flink.connector.base.DeliveryGuarantee)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:128)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.InstantiationUtil.isSerializable(java.lang.Object)> in (Opensearch2SinkBuilder.java:99)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:97)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setEmitter(org.apache.flink.connector.opensearch.sink.OpensearchEmitter)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:98)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setFailureHandler(org.apache.flink.connector.opensearch.sink.FailureHandler)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:314)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setHosts([Lorg.apache.http.HttpHost;)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:114)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setHosts([Lorg.apache.http.HttpHost;)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:115)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setRestClientFactory(org.apache.flink.connector.opensearch.sink.RestClientFactory)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (Opensearch2SinkBuilder.java:302)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder.setSocketTimeout(int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.Object)> in (Opensearch2SinkBuilder.java:277)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2Writer$DefaultBulkResponseInspector.inspect(org.opensearch.action.bulk.BulkRequest, org.opensearch.action.bulk.BulkResponse)> calls method <org.apache.flink.util.ExceptionUtils.firstOrSuppressed(java.lang.Throwable, java.lang.Throwable)> in (Opensearch2Writer.java:364)
+Method <org.apache.flink.connector.opensearch.sink.Opensearch2Writer.blockingFlushAllActions()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (Opensearch2Writer.java:0)
+Method <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter$1.getMetricGroup()> calls constructor <org.apache.flink.metrics.groups.UnregisteredMetricsGroup.<init>()> in (RowOpensearch2Emitter.java:69)
+Method <org.apache.flink.connector.opensearch.table.RowOpensearch2Emitter$1.getUserCodeClassLoader()> calls method <org.apache.flink.util.SimpleUserCodeClassLoader.create(java.lang.ClassLoader)> in (RowOpensearch2Emitter.java:74)
diff --git a/flink-connector-opensearch2/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156 b/flink-connector-opensearch2/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/85d671bb-83fe-41ff-bc35-bd171908a156
diff --git a/flink-connector-opensearch2/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5 b/flink-connector-opensearch2/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/879defc0-4d7e-43cb-ace6-1eb5aec9bac5
diff --git a/flink-connector-opensearch2/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20 b/flink-connector-opensearch2/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
diff --git a/flink-connector-opensearch2/archunit-violations/stored.rules b/flink-connector-opensearch2/archunit-violations/stored.rules
new file mode 100644
index 0000000..39c4b39
--- /dev/null
+++ b/flink-connector-opensearch2/archunit-violations/stored.rules
@@ -0,0 +1,11 @@
+#
+#Thu Dec 14 21:22:47 CET 2023
+Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=879defc0-4d7e-43cb-ace6-1eb5aec9bac5
+Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=85d671bb-83fe-41ff-bc35-bd171908a156
+ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=4382f1f0-807a-45ff-97d8-42f72b6e9484
+Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=73099dba-7f06-4637-b2ad-b3c906aaaf1c
+Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=7c5588a1-e67a-4f85-a1c2-6be9dfe44c09
+Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=0c6c8466-9ce6-41a7-b6dd-947cc5702975
+Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=0c16f106-1632-4ba5-aa26-eb3ab7c7d43e
+Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=c921f20b-c7ee-4fd5-a8e9-c19ac7a59f20
+Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=7f6b6846-355a-45b6-9c14-b65accd8d850
diff --git a/flink-connector-opensearch2/pom.xml b/flink-connector-opensearch2/pom.xml
new file mode 100644
index 0000000..47bb566
--- /dev/null
+++ b/flink-connector-opensearch2/pom.xml
@@ -0,0 +1,252 @@
+<?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-connector-opensearch-parent</artifactId>
+ <version>2.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>flink-connector-opensearch2</artifactId>
+ <name>Flink : Connectors : Opensearch : 2</name>
+
+ <packaging>jar</packaging>
+
+ <!-- Allow users to pass custom connector versions -->
+ <properties>
+ <opensearch.version>2.11.1</opensearch.version>
+ <flink.connector.module.config><!-- required by
+ OpensearchSinkITCase --> --add-opens=java.base/java.lang=ALL-UNNAMED <!--
+ OpensearchSinkITCase --> --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED <!--
+ OpensearchDynamicSinkITCase --> --add-opens=java.base/java.util=ALL-UNNAMED </flink.connector.module.config>
+ </properties>
+
+ <dependencies>
+
+ <!-- Core -->
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-base</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.opensearch.client</groupId>
+ <artifactId>opensearch-rest-high-level-client</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch-base</artifactId>
+ <version>${project.parent.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.opensearch.client</groupId>
+ <artifactId>opensearch-rest-high-level-client</artifactId>
+ </exclusion>
+ </exclusions>
+ </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>${flink.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <!-- Opensearch -->
+
+ <dependency>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch</artifactId>
+ <version>${opensearch.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.google.protobuf</groupId>
+ <artifactId>protobuf-java</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>jakarta.annotation</groupId>
+ <artifactId>jakarta.annotation-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.jcraft</groupId>
+ <artifactId>jzlib</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch-core</artifactId>
+ <version>${opensearch.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.opensearch</groupId>
+ <artifactId>opensearch-x-content</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>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-runtime</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.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>${flink.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>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Table API integration tests -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-test-utils</artifactId>
+ <version>${flink.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>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- ArchUit test dependencies -->
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-architecture-tests-test</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java
new file mode 100644
index 0000000..356e297
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspector.java
@@ -0,0 +1,88 @@
+/*
+ * 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.VisibleForTesting;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.core.rest.RestStatus;
+
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class DefaultBulkResponseInspector implements BulkResponseInspector {
+
+ @VisibleForTesting final FailureHandler failureHandler;
+
+ DefaultBulkResponseInspector() {
+ this(new DefaultFailureHandler());
+ }
+
+ DefaultBulkResponseInspector(FailureHandler failureHandler) {
+ this.failureHandler = checkNotNull(failureHandler);
+ }
+
+ @Override
+ public void inspect(BulkRequest request, BulkResponse response) {
+ if (!response.hasFailures()) {
+ return;
+ }
+
+ Throwable chainedFailures = null;
+ for (int i = 0; i < response.getItems().length; i++) {
+ final BulkItemResponse itemResponse = response.getItems()[i];
+ if (!itemResponse.isFailed()) {
+ continue;
+ }
+ final Throwable failure = itemResponse.getFailure().getCause();
+ if (failure == null) {
+ continue;
+ }
+ final RestStatus restStatus = itemResponse.getFailure().getStatus();
+ final DocWriteRequest<?> actionRequest = request.requests().get(i);
+
+ chainedFailures =
+ firstOrSuppressed(
+ wrapException(restStatus, failure, actionRequest), chainedFailures);
+ }
+ if (chainedFailures == null) {
+ return;
+ }
+ failureHandler.onFailure(chainedFailures);
+ }
+
+ private static Throwable wrapException(
+ RestStatus restStatus, Throwable rootFailure, DocWriteRequest<?> actionRequest) {
+ if (restStatus == null) {
+ return new FlinkRuntimeException(
+ String.format("Single action %s of bulk request failed.", actionRequest),
+ rootFailure);
+ } else {
+ return new FlinkRuntimeException(
+ String.format(
+ "Single action %s of bulk request failed with status %s.",
+ actionRequest, restStatus.getStatus()),
+ rootFailure);
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Sink.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Sink.java
new file mode 100644
index 0000000..9ca342e
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Sink.java
@@ -0,0 +1,107 @@
+/*
+ * 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;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.opensearch.sink.BulkResponseInspector.BulkResponseInspectorFactory;
+
+import org.apache.http.HttpHost;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Flink Sink to insert or update data in an Opensearch index. The sink supports the following
+ * delivery guarantees.
+ *
+ * <ul>
+ * <li>{@link DeliveryGuarantee#NONE} does not provide any guarantees: actions are flushed to
+ * Opensearch only depending on the configurations of the bulk processor. In case of a
+ * failure, it might happen that actions are lost if the bulk processor still has buffered
+ * actions.
+ * <li>{@link DeliveryGuarantee#AT_LEAST_ONCE} on a checkpoint the sink will wait until all
+ * buffered actions are flushed to and acknowledged by Opensearch. No actions will be lost but
+ * actions might be sent to Opensearch multiple times when Flink restarts. These additional
+ * requests may cause inconsistent data in Opensearch right after the restart, but eventually
+ * everything will be consistent again.
+ * </ul>
+ *
+ * @param <IN> type of the records converted to Opensearch actions
+ * @see Opensearch2SinkBuilder on how to construct a OpensearchSink
+ */
+@PublicEvolving
+public class Opensearch2Sink<IN> implements Sink<IN> {
+
+ private final List<HttpHost> hosts;
+ private final OpensearchEmitter<? super IN> emitter;
+ private final BulkProcessorConfig buildBulkProcessorConfig;
+ private final NetworkClientConfig networkClientConfig;
+ private final DeliveryGuarantee deliveryGuarantee;
+ private final RestClientFactory restClientFactory;
+ private final BulkResponseInspectorFactory bulkResponseInspectorFactory;
+
+ Opensearch2Sink(
+ List<HttpHost> hosts,
+ OpensearchEmitter<? super IN> emitter,
+ DeliveryGuarantee deliveryGuarantee,
+ BulkProcessorConfig buildBulkProcessorConfig,
+ NetworkClientConfig networkClientConfig,
+ RestClientFactory restClientFactory,
+ BulkResponseInspectorFactory bulkResponseInspectorFactory) {
+ this.hosts = checkNotNull(hosts);
+ checkArgument(!hosts.isEmpty(), "Hosts cannot be empty.");
+ this.emitter = checkNotNull(emitter);
+ this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+ this.buildBulkProcessorConfig = checkNotNull(buildBulkProcessorConfig);
+ this.networkClientConfig = checkNotNull(networkClientConfig);
+ this.restClientFactory = checkNotNull(restClientFactory);
+ this.bulkResponseInspectorFactory = checkNotNull(bulkResponseInspectorFactory);
+ }
+
+ @Override
+ public SinkWriter<IN> createWriter(InitContext context) throws IOException {
+ return new Opensearch2Writer<>(
+ hosts,
+ emitter,
+ deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE,
+ buildBulkProcessorConfig,
+ networkClientConfig,
+ context.metricGroup(),
+ context.getMailboxExecutor(),
+ restClientFactory,
+ bulkResponseInspectorFactory.apply(context::metricGroup));
+ }
+
+ @VisibleForTesting
+ DeliveryGuarantee getDeliveryGuarantee() {
+ return deliveryGuarantee;
+ }
+
+ @VisibleForTesting
+ BulkResponseInspectorFactory getBulkResponseInspectorFactory() {
+ return bulkResponseInspectorFactory;
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilder.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilder.java
new file mode 100644
index 0000000..9b42f01
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilder.java
@@ -0,0 +1,438 @@
+/*
+ * 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;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.opensearch.sink.BulkResponseInspector.BulkResponseInspectorFactory;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.apache.http.HttpHost;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Builder to construct an Opensearch2 compatible {@link Opensearch2Sink}.
+ *
+ * <p>The following example shows the minimal setup to create a Opensearch2Sink that submits actions
+ * on checkpoint or the default number of actions was buffered (1000).
+ *
+ * <pre>{@code
+ * Opensearch2Sink<String> sink = new Opensearch2SinkBuilder<String>()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setEmitter((element, context, indexer) -> {
+ * indexer.add(
+ * new IndexRequest("my-index")
+ * .id(element.f0.toString())
+ * .source(element.f1)
+ * );
+ * })
+ * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+ * .build();
+ * }</pre>
+ *
+ * @param <IN> type of the records converted to Opensearch2 actions
+ */
+@PublicEvolving
+public class Opensearch2SinkBuilder<IN> {
+
+ private int bulkFlushMaxActions = 1000;
+ private int bulkFlushMaxMb = -1;
+ private long bulkFlushInterval = -1;
+ private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE;
+ private int bulkFlushBackoffRetries = -1;
+ private long bulkFlushBackOffDelay = -1;
+ private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+ private List<HttpHost> hosts;
+ protected OpensearchEmitter<? super IN> emitter;
+ private String username;
+ private String password;
+ private String connectionPathPrefix;
+ private Integer connectionTimeout;
+ private Integer connectionRequestTimeout;
+ private Integer socketTimeout;
+ private Boolean allowInsecure;
+ private RestClientFactory restClientFactory;
+ private FailureHandler failureHandler = new DefaultFailureHandler();
+ private BulkResponseInspectorFactory bulkResponseInspectorFactory;
+
+ public Opensearch2SinkBuilder() {
+ restClientFactory = new DefaultRestClientFactory();
+ }
+
+ @SuppressWarnings("unchecked")
+ protected <S extends Opensearch2SinkBuilder<?>> S self() {
+ return (S) this;
+ }
+
+ /**
+ * Sets the emitter which is invoked on every record to convert it to Opensearch actions.
+ *
+ * @param emitter to process records into Opensearch actions.
+ * @return this builder
+ */
+ public <T extends IN> Opensearch2SinkBuilder<T> setEmitter(
+ OpensearchEmitter<? super T> emitter) {
+ checkNotNull(emitter);
+ checkState(
+ InstantiationUtil.isSerializable(emitter),
+ "The Opensearch emitter must be serializable.");
+
+ final Opensearch2SinkBuilder<T> self = self();
+ self.emitter = emitter;
+ return self;
+ }
+
+ /**
+ * Sets the hosts where the Opensearch cluster nodes are reachable.
+ *
+ * @param hosts http addresses describing the node locations
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setHosts(HttpHost... hosts) {
+ checkNotNull(hosts);
+ checkState(hosts.length > 0, "Hosts cannot be empty.");
+ this.hosts = Arrays.asList(hosts);
+ return self();
+ }
+
+ /**
+ * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+ * DeliveryGuarantee#NONE}
+ *
+ * @param deliveryGuarantee which describes the record emission behaviour
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+ checkState(
+ deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+ "Opensearch sink does not support the EXACTLY_ONCE guarantee.");
+ this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+ return self();
+ }
+
+ /**
+ * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+ * disable it. The default flush size 1000.
+ *
+ * @param numMaxActions the maximum number of actions to buffer per bulk request.
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setBulkFlushMaxActions(int numMaxActions) {
+ checkState(
+ numMaxActions == -1 || numMaxActions > 0,
+ "Max number of buffered actions must be larger than 0.");
+ this.bulkFlushMaxActions = numMaxActions;
+ return self();
+ }
+
+ /**
+ * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to
+ * disable it.
+ *
+ * @param maxSizeMb the maximum size of buffered actions, in mb.
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setBulkFlushMaxSizeMb(int maxSizeMb) {
+ checkState(
+ maxSizeMb == -1 || maxSizeMb > 0,
+ "Max size of buffered actions must be larger than 0.");
+ this.bulkFlushMaxMb = maxSizeMb;
+ return self();
+ }
+
+ /**
+ * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+ *
+ * @param intervalMillis the bulk flush interval, in milliseconds.
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setBulkFlushInterval(long intervalMillis) {
+ checkState(
+ intervalMillis == -1 || intervalMillis >= 0,
+ "Interval (in milliseconds) between each flush must be larger than "
+ + "or equal to 0.");
+ this.bulkFlushInterval = intervalMillis;
+ return self();
+ }
+
+ /**
+ * Sets the type of back off to use when flushing bulk requests. The default bulk flush back off
+ * type is {@link FlushBackoffType#NONE}.
+ *
+ * <p>Sets the amount of delay between each backoff attempt when flushing bulk requests, in
+ * milliseconds.
+ *
+ * <p>Sets the maximum number of retries for a backoff attempt when flushing bulk requests.
+ *
+ * @param flushBackoffType the backoff type to use.
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setBulkFlushBackoffStrategy(
+ FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) {
+ this.bulkFlushBackoffType = checkNotNull(flushBackoffType);
+ checkState(
+ flushBackoffType != FlushBackoffType.NONE,
+ "FlushBackoffType#NONE does not require a configuration it is the default, retries and delay are ignored.");
+ checkState(maxRetries > 0, "Max number of backoff attempts must be larger than 0.");
+ this.bulkFlushBackoffRetries = maxRetries;
+ checkState(
+ delayMillis >= 0,
+ "Delay (in milliseconds) between each backoff attempt must be larger "
+ + "than or equal to 0.");
+ this.bulkFlushBackOffDelay = delayMillis;
+ return self();
+ }
+
+ /**
+ * Sets the username used to authenticate the connection with the Opensearch cluster.
+ *
+ * @param username of the Opensearch cluster user
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setConnectionUsername(String username) {
+ checkNotNull(username);
+ this.username = username;
+ return self();
+ }
+
+ /**
+ * Sets the password used to authenticate the conection with the Opensearch cluster.
+ *
+ * @param password of the Opensearch cluster user
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setConnectionPassword(String password) {
+ checkNotNull(password);
+ this.password = password;
+ return self();
+ }
+
+ /**
+ * Sets a prefix which used for every REST communication to the Opensearch cluster.
+ *
+ * @param prefix for the communication
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setConnectionPathPrefix(String prefix) {
+ checkNotNull(prefix);
+ this.connectionPathPrefix = prefix;
+ return self();
+ }
+
+ /**
+ * Sets the timeout for requesting the connection of the Opensearch cluster from the connection
+ * manager.
+ *
+ * @param timeout for the connection request
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setConnectionRequestTimeout(int timeout) {
+ checkState(timeout >= 0, "Connection request timeout must be larger than or equal to 0.");
+ this.connectionRequestTimeout = timeout;
+ return self();
+ }
+
+ /**
+ * Sets the timeout for establishing a connection of the Opensearch cluster.
+ *
+ * @param timeout for the connection
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setConnectionTimeout(int timeout) {
+ checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0.");
+ this.connectionTimeout = timeout;
+ return self();
+ }
+
+ /**
+ * Sets the timeout for waiting for data or, put differently, a maximum period inactivity
+ * between two consecutive data packets.
+ *
+ * @param timeout for the socket
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setSocketTimeout(int timeout) {
+ checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0.");
+ this.socketTimeout = timeout;
+ return self();
+ }
+
+ /**
+ * Allows to bypass the certificates chain validation and connect to insecure network endpoints
+ * (for example, servers which use self-signed certificates).
+ *
+ * @param allowInsecure allow or not to insecure network endpoints
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setAllowInsecure(boolean allowInsecure) {
+ this.allowInsecure = allowInsecure;
+ return self();
+ }
+
+ /**
+ * Sets the {@link RestClientFactory} to be used for configuring the instance of the OpenSearch
+ * REST client.
+ *
+ * @param restClientFactory the {@link RestClientFactory} instance
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setRestClientFactory(RestClientFactory restClientFactory) {
+ this.restClientFactory = checkNotNull(restClientFactory);
+ return self();
+ }
+
+ /**
+ * Allows to set custom failure handler. If not set, then the DEFAULT_FAILURE_HANDLER will be
+ * used which throws a runtime exception upon receiving a failure.
+ *
+ * @param failureHandler the custom handler
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setFailureHandler(FailureHandler failureHandler) {
+ checkNotNull(failureHandler);
+ this.failureHandler = failureHandler;
+ return self();
+ }
+
+ /**
+ * Overrides the default {@link BulkResponseInspectorFactory}. A custom {@link
+ * BulkResponseInspector}, for example, can change the failure handling and capture additional
+ * metrics. See {@link #failureHandler} for a simpler way of handling failures.
+ *
+ * @param bulkResponseInspectorFactory the factory
+ * @return this builder
+ */
+ public Opensearch2SinkBuilder<IN> setBulkResponseInspectorFactory(
+ BulkResponseInspectorFactory bulkResponseInspectorFactory) {
+ this.bulkResponseInspectorFactory = checkNotNull(bulkResponseInspectorFactory);
+ return self();
+ }
+
+ /**
+ * Constructs the {@link Opensearch2Sink} with the properties configured this builder.
+ *
+ * @return {@link Opensearch2Sink}
+ */
+ public Opensearch2Sink<IN> build() {
+ checkNotNull(emitter);
+ checkNotNull(hosts);
+
+ NetworkClientConfig networkClientConfig = buildNetworkClientConfig();
+ BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig();
+
+ return new Opensearch2Sink<>(
+ hosts,
+ emitter,
+ deliveryGuarantee,
+ bulkProcessorConfig,
+ networkClientConfig,
+ restClientFactory,
+ getBulkResponseInspectorFactory());
+ }
+
+ protected BulkResponseInspectorFactory getBulkResponseInspectorFactory() {
+ return this.bulkResponseInspectorFactory == null
+ ? new DefaultBulkResponseInspectorFactory(failureHandler)
+ : this.bulkResponseInspectorFactory;
+ }
+
+ private NetworkClientConfig buildNetworkClientConfig() {
+ checkArgument(!hosts.isEmpty(), "Hosts cannot be empty.");
+
+ return new NetworkClientConfig(
+ username,
+ password,
+ connectionPathPrefix,
+ connectionRequestTimeout,
+ connectionTimeout,
+ socketTimeout,
+ allowInsecure);
+ }
+
+ private BulkProcessorConfig buildBulkProcessorConfig() {
+ return new BulkProcessorConfig(
+ bulkFlushMaxActions,
+ bulkFlushMaxMb,
+ bulkFlushInterval,
+ bulkFlushBackoffType,
+ bulkFlushBackoffRetries,
+ bulkFlushBackOffDelay);
+ }
+
+ @Override
+ public String toString() {
+ return "OpensearchSinkBuilder{"
+ + "bulkFlushMaxActions="
+ + bulkFlushMaxActions
+ + ", bulkFlushMaxMb="
+ + bulkFlushMaxMb
+ + ", bulkFlushInterval="
+ + bulkFlushInterval
+ + ", bulkFlushBackoffType="
+ + bulkFlushBackoffType
+ + ", bulkFlushBackoffRetries="
+ + bulkFlushBackoffRetries
+ + ", bulkFlushBackOffDelay="
+ + bulkFlushBackOffDelay
+ + ", deliveryGuarantee="
+ + deliveryGuarantee
+ + ", hosts="
+ + hosts
+ + ", emitter="
+ + emitter
+ + ", username='"
+ + username
+ + '\''
+ + ", password='"
+ + password
+ + '\''
+ + ", connectionPathPrefix='"
+ + connectionPathPrefix
+ + '\''
+ + ", allowInsecure='"
+ + allowInsecure
+ + '\''
+ + '}';
+ }
+
+ /**
+ * Default factory for {@link FailureHandler}-bound {@link BulkResponseInspector
+ * BulkResponseInspectors}. A Static class is used instead of anonymous/lambda to avoid
+ * non-serializable references to {@link Opensearch2SinkBuilder}.
+ */
+ static class DefaultBulkResponseInspectorFactory implements BulkResponseInspectorFactory {
+
+ private final FailureHandler failureHandler;
+
+ DefaultBulkResponseInspectorFactory(FailureHandler failureHandler) {
+ this.failureHandler = failureHandler;
+ }
+
+ @Override
+ public BulkResponseInspector apply(InitContext context) {
+ return new DefaultBulkResponseInspector(failureHandler);
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Writer.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Writer.java
new file mode 100644
index 0000000..0bf0997
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/sink/Opensearch2Writer.java
@@ -0,0 +1,388 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.bulk.BackoffPolicy;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkProcessor;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestClientBuilder;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.action.ActionListener;
+import org.opensearch.core.common.unit.ByteSizeUnit;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.core.rest.RestStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class Opensearch2Writer<IN> implements SinkWriter<IN> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Opensearch2Writer.class);
+
+ private final OpensearchEmitter<? super IN> emitter;
+ private final MailboxExecutor mailboxExecutor;
+ private final boolean flushOnCheckpoint;
+ private final BulkProcessor bulkProcessor;
+ private final RestHighLevelClient client;
+ private final RequestIndexer requestIndexer;
+ private final Counter numBytesOutCounter;
+
+ private long pendingActions = 0;
+ private boolean checkpointInProgress = false;
+ private volatile long lastSendTime = 0;
+ private volatile long ackTime = Long.MAX_VALUE;
+ private volatile boolean closed = false;
+
+ /**
+ * Constructor creating an Opensearch writer.
+ *
+ * @param hosts the reachable Opensearch cluster nodes
+ * @param emitter converting incoming records to Opensearch actions
+ * @param flushOnCheckpoint if true all until now received records are flushed after every
+ * checkpoint
+ * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link
+ * BulkProcessor}
+ * @param networkClientConfig describing properties of the network connection used to connect to
+ * the Opensearch cluster
+ * @param metricGroup for the sink writer
+ * @param mailboxExecutor Flink's mailbox executor
+ * @param restClientFactory Flink's mailbox executor
+ */
+ Opensearch2Writer(
+ List<HttpHost> hosts,
+ OpensearchEmitter<? super IN> emitter,
+ boolean flushOnCheckpoint,
+ BulkProcessorConfig bulkProcessorConfig,
+ NetworkClientConfig networkClientConfig,
+ SinkWriterMetricGroup metricGroup,
+ MailboxExecutor mailboxExecutor,
+ RestClientFactory restClientFactory,
+ BulkResponseInspector bulkResponseInspector) {
+ this.emitter = checkNotNull(emitter);
+ this.flushOnCheckpoint = flushOnCheckpoint;
+ this.mailboxExecutor = checkNotNull(mailboxExecutor);
+
+ final RestClientBuilder builder = RestClient.builder(hosts.toArray(new HttpHost[0]));
+ checkNotNull(restClientFactory)
+ .configureRestClientBuilder(
+ builder, new DefaultRestClientConfig(networkClientConfig));
+
+ this.client = new RestHighLevelClient(builder);
+ this.bulkProcessor =
+ createBulkProcessor(bulkProcessorConfig, checkNotNull(bulkResponseInspector));
+ this.requestIndexer = new DefaultRequestIndexer(metricGroup.getNumRecordsSendCounter());
+ checkNotNull(metricGroup);
+ metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+ this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter();
+ try {
+ emitter.open();
+ } catch (Exception e) {
+ throw new FlinkRuntimeException("Failed to open the OpensearchEmitter", e);
+ }
+ }
+
+ @Override
+ public void write(IN element, Context context) throws IOException, InterruptedException {
+ // do not allow new bulk writes until all actions are flushed
+ while (checkpointInProgress) {
+ mailboxExecutor.yield();
+ }
+ emitter.emit(element, context, requestIndexer);
+ }
+
+ @Override
+ public void flush(boolean endOfInput) throws IOException, InterruptedException {
+ checkpointInProgress = true;
+ while (pendingActions != 0 && (flushOnCheckpoint || endOfInput)) {
+ bulkProcessor.flush();
+ LOG.info("Waiting for the response of {} pending actions.", pendingActions);
+ mailboxExecutor.yield();
+ }
+ checkpointInProgress = false;
+ }
+
+ @VisibleForTesting
+ void blockingFlushAllActions() throws InterruptedException {
+ while (pendingActions != 0) {
+ bulkProcessor.flush();
+ LOG.info("Waiting for the response of {} pending actions.", pendingActions);
+ mailboxExecutor.yield();
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ closed = true;
+ emitter.close();
+ bulkProcessor.close();
+ client.close();
+ }
+
+ private BulkProcessor createBulkProcessor(
+ BulkProcessorConfig bulkProcessorConfig, BulkResponseInspector bulkResponseInspector) {
+
+ final BulkProcessor.Builder builder =
+ BulkProcessor.builder(
+ new BulkRequestConsumerFactory() { // This cannot be inlined as a
+ // lambda because then
+ // deserialization fails
+ @Override
+ public void accept(
+ BulkRequest bulkRequest,
+ ActionListener<BulkResponse> bulkResponseActionListener) {
+ client.bulkAsync(
+ bulkRequest,
+ RequestOptions.DEFAULT,
+ bulkResponseActionListener);
+ }
+ },
+ new BulkListener(bulkResponseInspector));
+
+ if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) {
+ builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions());
+ }
+
+ if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) {
+ builder.setBulkSize(
+ new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB));
+ }
+
+ if (bulkProcessorConfig.getBulkFlushInterval() != -1) {
+ builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval()));
+ }
+
+ BackoffPolicy backoffPolicy;
+ final TimeValue backoffDelay =
+ new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay());
+ final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries();
+ switch (bulkProcessorConfig.getFlushBackoffType()) {
+ case CONSTANT:
+ backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount);
+ break;
+ case EXPONENTIAL:
+ backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount);
+ break;
+ case NONE:
+ backoffPolicy = BackoffPolicy.noBackoff();
+ break;
+ default:
+ throw new IllegalArgumentException(
+ "Received unknown backoff policy type "
+ + bulkProcessorConfig.getFlushBackoffType());
+ }
+ builder.setBackoffPolicy(backoffPolicy);
+ // This makes flush() blocking
+ builder.setConcurrentRequests(0);
+
+ return builder.build();
+ }
+
+ private class BulkListener implements BulkProcessor.Listener {
+
+ private final BulkResponseInspector bulkResponseInspector;
+
+ public BulkListener(BulkResponseInspector bulkResponseInspector) {
+ this.bulkResponseInspector = bulkResponseInspector;
+ }
+
+ @Override
+ public void beforeBulk(long executionId, BulkRequest request) {
+ LOG.info("Sending bulk of {} actions to Opensearch.", request.numberOfActions());
+ lastSendTime = System.currentTimeMillis();
+ numBytesOutCounter.inc(request.estimatedSizeInBytes());
+ }
+
+ @Override
+ public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
+ ackTime = System.currentTimeMillis();
+ enqueueActionInMailbox(
+ () -> extractFailures(request, response), "opensearchSuccessCallback");
+ }
+
+ @Override
+ public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
+ enqueueActionInMailbox(
+ () -> {
+ throw new FlinkRuntimeException("Complete bulk has failed.", failure);
+ },
+ "opensearchErrorCallback");
+ }
+
+ private void extractFailures(BulkRequest request, BulkResponse response) {
+ bulkResponseInspector.inspect(request, response);
+ pendingActions -= request.numberOfActions();
+ }
+ }
+
+ private void enqueueActionInMailbox(
+ ThrowingRunnable<? extends Exception> action, String actionName) {
+ // If the writer is cancelled before the last bulk response (i.e. no flush on checkpoint
+ // configured or shutdown without a final
+ // checkpoint) the mailbox might already be shutdown, so we should not enqueue any
+ // actions.
+ if (isClosed()) {
+ return;
+ }
+ mailboxExecutor.execute(action, actionName);
+ }
+
+ private static Throwable wrapException(
+ RestStatus restStatus, Throwable rootFailure, DocWriteRequest<?> actionRequest) {
+ if (restStatus == null) {
+ return new FlinkRuntimeException(
+ String.format("Single action %s of bulk request failed.", actionRequest),
+ rootFailure);
+ } else {
+ return new FlinkRuntimeException(
+ String.format(
+ "Single action %s of bulk request failed with status %s.",
+ actionRequest, restStatus.getStatus()),
+ rootFailure);
+ }
+ }
+
+ private boolean isClosed() {
+ if (closed) {
+ LOG.warn("Writer was closed before all records were acknowledged by Opensearch.");
+ }
+ return closed;
+ }
+
+ private class DefaultRequestIndexer implements RequestIndexer {
+
+ private final Counter numRecordsSendCounter;
+
+ public DefaultRequestIndexer(Counter numRecordsSendCounter) {
+ this.numRecordsSendCounter = checkNotNull(numRecordsSendCounter);
+ }
+
+ @Override
+ public void add(DeleteRequest... deleteRequests) {
+ for (final DeleteRequest deleteRequest : deleteRequests) {
+ numRecordsSendCounter.inc();
+ pendingActions++;
+ bulkProcessor.add(deleteRequest);
+ }
+ }
+
+ @Override
+ public void add(IndexRequest... indexRequests) {
+ for (final IndexRequest indexRequest : indexRequests) {
+ numRecordsSendCounter.inc();
+ pendingActions++;
+ bulkProcessor.add(indexRequest);
+ }
+ }
+
+ @Override
+ public void add(UpdateRequest... updateRequests) {
+ for (final UpdateRequest updateRequest : updateRequests) {
+ numRecordsSendCounter.inc();
+ pendingActions++;
+ bulkProcessor.add(updateRequest);
+ }
+ }
+ }
+
+ /**
+ * A strict implementation that fails if either the whole bulk request failed or any of its
+ * actions.
+ */
+ static class DefaultBulkResponseInspector implements BulkResponseInspector {
+
+ @VisibleForTesting final FailureHandler failureHandler;
+
+ DefaultBulkResponseInspector() {
+ this(new DefaultFailureHandler());
+ }
+
+ DefaultBulkResponseInspector(FailureHandler failureHandler) {
+ this.failureHandler = checkNotNull(failureHandler);
+ }
+
+ @Override
+ public void inspect(BulkRequest request, BulkResponse response) {
+ if (!response.hasFailures()) {
+ return;
+ }
+
+ Throwable chainedFailures = null;
+ for (int i = 0; i < response.getItems().length; i++) {
+ final BulkItemResponse itemResponse = response.getItems()[i];
+ if (!itemResponse.isFailed()) {
+ continue;
+ }
+ final Throwable failure = itemResponse.getFailure().getCause();
+ if (failure == null) {
+ continue;
+ }
+ final RestStatus restStatus = itemResponse.getFailure().getStatus();
+ final DocWriteRequest<?> actionRequest = request.requests().get(i);
+
+ chainedFailures =
+ firstOrSuppressed(
+ wrapException(restStatus, failure, actionRequest), chainedFailures);
+ }
+ if (chainedFailures == null) {
+ return;
+ }
+ failureHandler.onFailure(chainedFailures);
+ }
+ }
+
+ static class DefaultFailureHandler implements FailureHandler {
+
+ @Override
+ public void onFailure(Throwable failure) {
+ if (failure instanceof FlinkRuntimeException) {
+ throw (FlinkRuntimeException) failure;
+ }
+ throw new FlinkRuntimeException(failure);
+ }
+ }
+
+ @Internal
+ interface BulkRequestConsumerFactory
+ extends BiConsumer<BulkRequest, ActionListener<BulkResponse>> {}
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSink.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSink.java
new file mode 100644
index 0000000..106f98d
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSink.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.connector.opensearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.opensearch.sink.FlushBackoffType;
+import org.apache.flink.connector.opensearch.sink.Opensearch2Sink;
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.http.HttpHost;
+import org.opensearch.common.xcontent.XContentType;
+
+import java.time.ZoneId;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link DynamicTableSink} that describes how to create a {@link Opensearch2Sink} from a logical
+ * description.
+ */
+@Internal
+class Opensearch2DynamicSink implements DynamicTableSink {
+
+ final EncodingFormat<SerializationSchema<RowData>> format;
+ final DataType physicalRowDataType;
+ final List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex;
+ final OpensearchConfiguration config;
+ final ZoneId localTimeZoneId;
+
+ final String summaryString;
+ final boolean isDynamicIndexWithSystemTime;
+
+ Opensearch2DynamicSink(
+ EncodingFormat<SerializationSchema<RowData>> format,
+ OpensearchConfiguration config,
+ List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex,
+ DataType physicalRowDataType,
+ String summaryString,
+ ZoneId localTimeZoneId) {
+ this.format = checkNotNull(format);
+ this.physicalRowDataType = checkNotNull(physicalRowDataType);
+ this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex);
+ this.config = checkNotNull(config);
+ this.summaryString = checkNotNull(summaryString);
+ this.localTimeZoneId = localTimeZoneId;
+ this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime();
+ }
+
+ public boolean isDynamicIndexWithSystemTime() {
+ IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper();
+ return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex());
+ }
+
+ Function<RowData, String> createKeyExtractor() {
+ return KeyExtractor.createKeyExtractor(
+ primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+ }
+
+ IndexGenerator createIndexGenerator() {
+ return IndexGeneratorFactory.createIndexGenerator(
+ config.getIndex(),
+ DataType.getFieldNames(physicalRowDataType),
+ DataType.getFieldDataTypes(physicalRowDataType),
+ localTimeZoneId);
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+ ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+ for (RowKind kind : requestedMode.getContainedKinds()) {
+ if (kind != RowKind.UPDATE_BEFORE) {
+ builder.addContainedKind(kind);
+ }
+ }
+ if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) {
+ throw new ValidationException(
+ "Dynamic indexing based on system time only works on append only stream.");
+ }
+ return builder.build();
+ }
+
+ @Override
+ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+ SerializationSchema<RowData> format =
+ this.format.createRuntimeEncoder(context, physicalRowDataType);
+
+ final RowOpensearch2Emitter rowOpensearchEmitter =
+ new RowOpensearch2Emitter(
+ createIndexGenerator(), format, XContentType.JSON, createKeyExtractor());
+
+ final Opensearch2SinkBuilder<RowData> builder = new Opensearch2SinkBuilder<>();
+ builder.setEmitter(rowOpensearchEmitter);
+ builder.setHosts(config.getHosts().toArray(new HttpHost[0]));
+ builder.setDeliveryGuarantee(config.getDeliveryGuarantee());
+ builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions());
+ builder.setBulkFlushMaxSizeMb(config.getBulkFlushMaxByteSize().getMebiBytes());
+ builder.setBulkFlushInterval(config.getBulkFlushInterval());
+
+ if (config.getBulkFlushBackoffType().isPresent()) {
+ FlushBackoffType backoffType = config.getBulkFlushBackoffType().get();
+ int backoffMaxRetries = config.getBulkFlushBackoffRetries().get();
+ long backoffDelayMs = config.getBulkFlushBackoffDelay().get();
+
+ builder.setBulkFlushBackoffStrategy(backoffType, backoffMaxRetries, backoffDelayMs);
+ }
+
+ if (config.getUsername().isPresent()
+ && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+ builder.setConnectionUsername(config.getUsername().get());
+ }
+
+ if (config.getPassword().isPresent()
+ && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) {
+ builder.setConnectionPassword(config.getPassword().get());
+ }
+
+ if (config.getPathPrefix().isPresent()
+ && !StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) {
+ builder.setConnectionPathPrefix(config.getPathPrefix().get());
+ }
+
+ if (config.getConnectionRequestTimeout().isPresent()) {
+ builder.setConnectionRequestTimeout(
+ (int) config.getConnectionRequestTimeout().get().getSeconds());
+ }
+
+ if (config.getConnectionTimeout().isPresent()) {
+ builder.setConnectionTimeout((int) config.getConnectionTimeout().get().getSeconds());
+ }
+
+ if (config.getSocketTimeout().isPresent()) {
+ builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds());
+ }
+
+ if (config.isAllowInsecure().isPresent()) {
+ builder.setAllowInsecure(config.isAllowInsecure().get());
+ }
+
+ return SinkV2Provider.of(builder.build(), config.getParallelism().orElse(null));
+ }
+
+ @Override
+ public DynamicTableSink copy() {
+ return new Opensearch2DynamicSink(
+ format,
+ config,
+ primaryKeyLogicalTypesWithIndex,
+ physicalRowDataType,
+ summaryString,
+ localTimeZoneId);
+ }
+
+ @Override
+ public String asSummaryString() {
+ return summaryString;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Opensearch2DynamicSink that = (Opensearch2DynamicSink) o;
+ return Objects.equals(format, that.format)
+ && Objects.equals(physicalRowDataType, that.physicalRowDataType)
+ && Objects.equals(
+ primaryKeyLogicalTypesWithIndex, that.primaryKeyLogicalTypesWithIndex)
+ && Objects.equals(config, that.config)
+ && Objects.equals(summaryString, that.summaryString);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ format,
+ physicalRowDataType,
+ primaryKeyLogicalTypesWithIndex,
+ config,
+ summaryString);
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactory.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactory.java
new file mode 100644
index 0000000..2e02b08
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactory.java
@@ -0,0 +1,241 @@
+/*
+ * 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.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.StringUtils;
+
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.commons.lang3.StringUtils.capitalize;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.ALLOW_INSECURE;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.CONNECTION_TIMEOUT;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.DELIVERY_GUARANTEE_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.FORMAT_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.HOSTS_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.INDEX_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.KEY_DELIMITER_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.PASSWORD_OPTION;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.SOCKET_TIMEOUT;
+import static org.apache.flink.connector.opensearch.table.OpensearchConnectorOptions.USERNAME_OPTION;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */
+@Internal
+public class Opensearch2DynamicSinkFactory implements DynamicTableSinkFactory {
+ private static final String FACTORY_IDENTIFIER = "opensearch-2";
+
+ @Override
+ public DynamicTableSink createDynamicTableSink(Context context) {
+ List<LogicalTypeWithIndex> primaryKeyLogicalTypesWithIndex =
+ getPrimaryKeyLogicalTypesWithIndex(context);
+
+ final FactoryUtil.TableFactoryHelper helper =
+ FactoryUtil.createTableFactoryHelper(this, context);
+ EncodingFormat<SerializationSchema<RowData>> format =
+ helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION);
+
+ OpensearchConfiguration config = getConfiguration(helper);
+ helper.validate();
+ validateConfiguration(config);
+
+ return new Opensearch2DynamicSink(
+ format,
+ config,
+ primaryKeyLogicalTypesWithIndex,
+ context.getPhysicalRowDataType(),
+ capitalize(FACTORY_IDENTIFIER),
+ getLocalTimeZoneId(context.getConfiguration()));
+ }
+
+ private static ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) {
+ final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE);
+ final ZoneId zoneId =
+ TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone)
+ ? ZoneId.systemDefault()
+ : ZoneId.of(zone);
+
+ return zoneId;
+ }
+
+ private static OpensearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) {
+ return new OpensearchConfiguration(helper.getOptions());
+ }
+
+ private static void validateConfiguration(OpensearchConfiguration config) {
+ config.getHosts(); // validate hosts
+ validate(
+ config.getIndex().length() >= 1,
+ () -> String.format("'%s' must not be empty", INDEX_OPTION.key()));
+ int maxActions = config.getBulkFlushMaxActions();
+ validate(
+ maxActions == -1 || maxActions >= 1,
+ () ->
+ String.format(
+ "'%s' must be at least 1. Got: %s",
+ BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions));
+ long maxSize = config.getBulkFlushMaxByteSize().getBytes();
+ long mb1 = 1024 * 1024;
+ validate(
+ maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0),
+ () ->
+ String.format(
+ "'%s' must be in MB granularity. Got: %s",
+ BULK_FLUSH_MAX_SIZE_OPTION.key(),
+ config.getBulkFlushMaxByteSize().toHumanReadableString()));
+ validate(
+ config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true),
+ () ->
+ String.format(
+ "'%s' must be at least 1. Got: %s",
+ BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(),
+ config.getBulkFlushBackoffRetries().get()));
+ if (config.getUsername().isPresent()
+ && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) {
+ validate(
+ config.getPassword().isPresent()
+ && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()),
+ () ->
+ String.format(
+ "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'",
+ USERNAME_OPTION.key(),
+ PASSWORD_OPTION.key(),
+ config.getUsername().get(),
+ config.getPassword().orElse("")));
+ }
+ }
+
+ private static void validate(boolean condition, Supplier<String> message) {
+ if (!condition) {
+ throw new ValidationException(message.get());
+ }
+ }
+
+ private static List<LogicalTypeWithIndex> getPrimaryKeyLogicalTypesWithIndex(Context context) {
+ DataType physicalRowDataType = context.getPhysicalRowDataType();
+ int[] primaryKeyIndexes = context.getPrimaryKeyIndexes();
+ if (primaryKeyIndexes.length != 0) {
+ DataType pkDataType = Projection.of(primaryKeyIndexes).project(physicalRowDataType);
+
+ OpensearchValidationUtils.validatePrimaryKey(pkDataType);
+ }
+
+ ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema();
+ return Arrays.stream(primaryKeyIndexes)
+ .mapToObj(
+ index -> {
+ Optional<Column> column = resolvedSchema.getColumn(index);
+ if (!column.isPresent()) {
+ throw new IllegalStateException(
+ String.format(
+ "No primary key column found with index '%s'.",
+ index));
+ }
+ LogicalType logicalType = column.get().getDataType().getLogicalType();
+ return new LogicalTypeWithIndex(index, logicalType);
+ })
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set<ConfigOption<?>> forwardOptions() {
+ return Stream.of(
+ HOSTS_OPTION,
+ INDEX_OPTION,
+ PASSWORD_OPTION,
+ USERNAME_OPTION,
+ KEY_DELIMITER_OPTION,
+ BULK_FLUSH_MAX_ACTIONS_OPTION,
+ BULK_FLUSH_MAX_SIZE_OPTION,
+ BULK_FLUSH_INTERVAL_OPTION,
+ BULK_FLUSH_BACKOFF_TYPE_OPTION,
+ BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION,
+ BULK_FLUSH_BACKOFF_DELAY_OPTION,
+ CONNECTION_PATH_PREFIX_OPTION,
+ CONNECTION_REQUEST_TIMEOUT,
+ CONNECTION_TIMEOUT,
+ SOCKET_TIMEOUT,
+ ALLOW_INSECURE)
+ .collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ return Stream.of(
+ KEY_DELIMITER_OPTION,
+ BULK_FLUSH_MAX_SIZE_OPTION,
+ BULK_FLUSH_MAX_ACTIONS_OPTION,
+ BULK_FLUSH_INTERVAL_OPTION,
+ BULK_FLUSH_BACKOFF_TYPE_OPTION,
+ BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION,
+ BULK_FLUSH_BACKOFF_DELAY_OPTION,
+ CONNECTION_PATH_PREFIX_OPTION,
+ CONNECTION_REQUEST_TIMEOUT,
+ CONNECTION_TIMEOUT,
+ SOCKET_TIMEOUT,
+ FORMAT_OPTION,
+ DELIVERY_GUARANTEE_OPTION,
+ PASSWORD_OPTION,
+ USERNAME_OPTION,
+ SINK_PARALLELISM,
+ ALLOW_INSECURE)
+ .collect(Collectors.toSet());
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return FACTORY_IDENTIFIER;
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2SinkBuilderSupplier.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2SinkBuilderSupplier.java
new file mode 100644
index 0000000..5845e58
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/Opensearch2SinkBuilderSupplier.java
@@ -0,0 +1,25 @@
+/*
+ * 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.table;
+
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
+
+import java.util.function.Supplier;
+
+interface Opensearch2SinkBuilderSupplier<T> extends Supplier<Opensearch2SinkBuilder<T>> {}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearch2Emitter.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearch2Emitter.java
new file mode 100644
index 0000000..3eb5b27
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearch2Emitter.java
@@ -0,0 +1,123 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.opensearch.sink.OpensearchEmitter;
+import org.apache.flink.connector.opensearch.sink.RequestIndexer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.SimpleUserCodeClassLoader;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.common.xcontent.XContentType;
+
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Sink function for converting upserts into Opensearch {@link ActionRequest}s. */
+class RowOpensearch2Emitter implements OpensearchEmitter<RowData> {
+
+ private final IndexGenerator indexGenerator;
+ private final SerializationSchema<RowData> serializationSchema;
+ private final XContentType contentType;
+ private final Function<RowData, String> createKey;
+
+ public RowOpensearch2Emitter(
+ IndexGenerator indexGenerator,
+ SerializationSchema<RowData> serializationSchema,
+ XContentType contentType,
+ Function<RowData, String> createKey) {
+ this.indexGenerator = checkNotNull(indexGenerator);
+ this.serializationSchema = checkNotNull(serializationSchema);
+ this.contentType = checkNotNull(contentType);
+ this.createKey = checkNotNull(createKey);
+ }
+
+ @Override
+ public void open() throws Exception {
+ try {
+ serializationSchema.open(
+ new SerializationSchema.InitializationContext() {
+ @Override
+ public MetricGroup getMetricGroup() {
+ return new UnregisteredMetricsGroup();
+ }
+
+ @Override
+ public UserCodeClassLoader getUserCodeClassLoader() {
+ return SimpleUserCodeClassLoader.create(
+ RowOpensearch2Emitter.class.getClassLoader());
+ }
+ });
+ } catch (Exception e) {
+ throw new FlinkRuntimeException("Failed to initialize serialization schema.", e);
+ }
+ indexGenerator.open();
+ }
+
+ @Override
+ public void emit(RowData element, SinkWriter.Context context, RequestIndexer indexer) {
+ switch (element.getRowKind()) {
+ case INSERT:
+ case UPDATE_AFTER:
+ processUpsert(element, indexer);
+ break;
+ case UPDATE_BEFORE:
+ case DELETE:
+ processDelete(element, indexer);
+ break;
+ default:
+ throw new TableException("Unsupported message kind: " + element.getRowKind());
+ }
+ }
+
+ private void processUpsert(RowData row, RequestIndexer indexer) {
+ final byte[] document = serializationSchema.serialize(row);
+ final String key = createKey.apply(row);
+ if (key != null) {
+ final UpdateRequest updateRequest =
+ new UpdateRequest(indexGenerator.generate(row), key)
+ .doc(document, contentType)
+ .upsert(document, contentType);
+ indexer.add(updateRequest);
+ } else {
+ final IndexRequest indexRequest =
+ new IndexRequest(indexGenerator.generate(row))
+ .id(key)
+ .source(document, contentType);
+ indexer.add(indexRequest);
+ }
+ }
+
+ private void processDelete(RowData row, RequestIndexer indexer) {
+ final String key = createKey.apply(row);
+ final DeleteRequest deleteRequest = new DeleteRequest(indexGenerator.generate(row), key);
+ indexer.add(deleteRequest);
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2Sink.java b/flink-connector-opensearch2/src/main/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2Sink.java
new file mode 100644
index 0000000..151b14e
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2Sink.java
@@ -0,0 +1,809 @@
+/*
+ * 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.opensearch;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.opensearch.sink.Opensearch2SinkBuilder;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.opensearch.util.NoOpFailureHandler;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.bulk.BackoffPolicy;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkProcessor;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestClientBuilder;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.common.unit.ByteSizeUnit;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.core.rest.RestStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Base class for all Flink Opensearch Sinks.
+ *
+ * <p>This class implements the common behaviour across Opensearch versions, such as the use of an
+ * internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before sending the
+ * requests to the cluster, as well as passing input records to the user provided {@link
+ * OpensearchSinkFunction} for processing.
+ *
+ * @param <T> Type of the elements handled by this sink
+ * @deprecated This sink has been deprecated in favor of {@link
+ * org.apache.flink.connector.opensearch.sink.Opensearch2Sink}
+ */
+@Deprecated
+@PublicEvolving
+public class Opensearch2Sink<T> extends RichSinkFunction<T> implements CheckpointedFunction {
+ private static final long serialVersionUID = -1007596293618451942L;
+ private static final Logger LOG = LoggerFactory.getLogger(Opensearch2Sink.class);
+
+ // ------------------------------------------------------------------------
+ // Internal bulk processor configuration
+ // ------------------------------------------------------------------------
+
+ 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";
+ public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable";
+ public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type";
+ public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries";
+ public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay";
+
+ /** Used to control whether the retry delay should increase exponentially or remain constant. */
+ @PublicEvolving
+ public enum FlushBackoffType {
+ CONSTANT,
+ EXPONENTIAL
+ }
+
+ /**
+ * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to
+ * resource constraints (i.e. the client's internal thread pool is full), the backoff policy
+ * decides how long the bulk processor will wait before the operation is retried internally.
+ *
+ * <p>This is a proxy for version specific backoff policies.
+ */
+ public static class BulkFlushBackoffPolicy implements Serializable {
+
+ private static final long serialVersionUID = -6022851996101826049L;
+
+ // the default values follow the Opensearch default settings for BulkProcessor
+ private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL;
+ private int maxRetryCount = 8;
+ private long delayMillis = 50;
+
+ public FlushBackoffType getBackoffType() {
+ return backoffType;
+ }
+
+ public int getMaxRetryCount() {
+ return maxRetryCount;
+ }
+
+ public long getDelayMillis() {
+ return delayMillis;
+ }
+
+ public void setBackoffType(FlushBackoffType backoffType) {
+ this.backoffType = checkNotNull(backoffType);
+ }
+
+ public void setMaxRetryCount(int maxRetryCount) {
+ checkArgument(maxRetryCount >= 0);
+ this.maxRetryCount = maxRetryCount;
+ }
+
+ public void setDelayMillis(long delayMillis) {
+ checkArgument(delayMillis >= 0);
+ this.delayMillis = delayMillis;
+ }
+ }
+
+ private final Integer bulkProcessorFlushMaxActions;
+ private final Integer bulkProcessorFlushMaxSizeMb;
+ private final Long bulkProcessorFlushIntervalMillis;
+ private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy;
+
+ // ------------------------------------------------------------------------
+ // User-facing API and configuration
+ // ------------------------------------------------------------------------
+
+ /**
+ * The config map that contains configuration for the bulk flushing behaviours.
+ *
+ * <p>For {@link org.opensearch.client.transport.TransportClient} based implementations, this
+ * config map would also contain Opensearch-shipped configuration, and therefore this config map
+ * would also be forwarded when creating the Opensearch client.
+ */
+ private final Map<String, String> userConfig;
+
+ /**
+ * The function that is used to construct multiple {@link ActionRequest ActionRequests} from
+ * each incoming element.
+ */
+ private final OpensearchSinkFunction<T> opensearchSinkFunction;
+
+ /** User-provided handler for failed {@link ActionRequest ActionRequests}. */
+ private final ActionRequestFailureHandler failureHandler;
+
+ /**
+ * If true, the producer will wait until all outstanding action requests have been sent to
+ * Opensearch.
+ */
+ private boolean flushOnCheckpoint = true;
+
+ /**
+ * Provided to the user via the {@link OpensearchSinkFunction} to add {@link ActionRequest
+ * ActionRequests}.
+ */
+ private transient RequestIndexer requestIndexer;
+
+ /**
+ * Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed
+ * requests.
+ */
+ private transient BufferingNoOpRequestIndexer failureRequestIndexer;
+
+ // ------------------------------------------------------------------------
+ // Internals for the Flink Opensearch Sink
+ // ------------------------------------------------------------------------
+
+ /** Opensearch client instance. */
+ private transient RestHighLevelClient client;
+
+ /**
+ * Number of pending action requests not yet acknowledged by Opensearch. This value is
+ * maintained only if {@link Opensearch2Sink#flushOnCheckpoint} is {@code true}.
+ *
+ * <p>This is incremented whenever the user adds (or re-adds through the {@link
+ * ActionRequestFailureHandler}) requests to the {@link RequestIndexer}. It is decremented for
+ * each completed request of a bulk request, in {@link BulkProcessor.Listener#afterBulk(long,
+ * BulkRequest, BulkResponse)} and {@link BulkProcessor.Listener#afterBulk(long, BulkRequest,
+ * Throwable)}.
+ */
+ private AtomicLong numPendingRequests = new AtomicLong(0);
+
+ /** User-provided HTTP Host. */
+ private final List<HttpHost> httpHosts;
+
+ /** The factory to configure the rest client. */
+ private final RestClientFactory restClientFactory;
+
+ /** Bulk processor to buffer and send requests to Opensearch, created using the client. */
+ private transient BulkProcessor bulkProcessor;
+
+ /**
+ * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown
+ * in callbacks and the user considered it should fail the sink via the {@link
+ * ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method.
+ *
+ * <p>Errors will be checked and rethrown before processing each input element, and when the
+ * sink is closed.
+ */
+ private final AtomicReference<Throwable> failureThrowable = new AtomicReference<>();
+
+ private Opensearch2Sink(
+ Map<String, String> userConfig,
+ List<HttpHost> httpHosts,
+ OpensearchSinkFunction<T> opensearchSinkFunction,
+ ActionRequestFailureHandler failureHandler,
+ RestClientFactory restClientFactory) {
+ checkArgument(httpHosts != null && !httpHosts.isEmpty());
+ this.httpHosts = httpHosts;
+ this.restClientFactory = checkNotNull(restClientFactory);
+ this.opensearchSinkFunction = checkNotNull(opensearchSinkFunction);
+ this.failureHandler = checkNotNull(failureHandler);
+ // we eagerly check if the user-provided sink function and failure handler is serializable;
+ // otherwise, if they aren't serializable, users will merely get a non-informative error
+ // message
+ // "OpensearchSink is not serializable"
+
+ checkArgument(
+ InstantiationUtil.isSerializable(opensearchSinkFunction),
+ "The implementation of the provided OpensearchSinkFunction is not serializable. "
+ + "The object probably contains or references non-serializable fields.");
+
+ checkArgument(
+ InstantiationUtil.isSerializable(failureHandler),
+ "The implementation of the provided ActionRequestFailureHandler is not serializable. "
+ + "The object probably contains or references non-serializable fields.");
+
+ // extract and remove bulk processor related configuration from the user-provided config,
+ // so that the resulting user config only contains configuration related to the
+ // Opensearch client.
+
+ checkNotNull(userConfig);
+
+ // copy config so we can remove entries without side-effects
+ userConfig = new HashMap<>(userConfig);
+
+ ParameterTool params = ParameterTool.fromMap(userConfig);
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
+ bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS);
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS);
+ } else {
+ bulkProcessorFlushMaxActions = null;
+ }
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
+ bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB);
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB);
+ } else {
+ bulkProcessorFlushMaxSizeMb = null;
+ }
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
+ bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS);
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS);
+ } else {
+ bulkProcessorFlushIntervalMillis = null;
+ }
+
+ boolean bulkProcessorFlushBackoffEnable =
+ params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true);
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE);
+
+ if (bulkProcessorFlushBackoffEnable) {
+ this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy();
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) {
+ bulkProcessorFlushBackoffPolicy.setBackoffType(
+ FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)));
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE);
+ }
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) {
+ bulkProcessorFlushBackoffPolicy.setMaxRetryCount(
+ params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES));
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES);
+ }
+
+ if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) {
+ bulkProcessorFlushBackoffPolicy.setDelayMillis(
+ params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY));
+ userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY);
+ }
+
+ } else {
+ bulkProcessorFlushBackoffPolicy = null;
+ }
+
+ this.userConfig = userConfig;
+ }
+
+ /**
+ * Disable flushing on checkpoint. When disabled, the sink will not wait for all pending action
+ * requests to be acknowledged by Opensearch on checkpoints.
+ *
+ * <p>NOTE: If flushing on checkpoint is disabled, the Flink Opensearch Sink does NOT provide
+ * any strong guarantees for at-least-once delivery of action requests.
+ */
+ public void disableFlushOnCheckpoint() {
+ this.flushOnCheckpoint = false;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ client = createClient();
+ bulkProcessor = buildBulkProcessor(new BulkProcessorListener());
+ requestIndexer =
+ new OpensearchBulkProcessorIndexer(
+ bulkProcessor, flushOnCheckpoint, numPendingRequests);
+ failureRequestIndexer = new BufferingNoOpRequestIndexer();
+ opensearchSinkFunction.open();
+ }
+
+ @Override
+ public void invoke(T value, Context context) throws Exception {
+ checkAsyncErrorsAndRequests();
+ opensearchSinkFunction.process(value, getRuntimeContext(), requestIndexer);
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ // no initialization needed
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ checkAsyncErrorsAndRequests();
+
+ if (flushOnCheckpoint) {
+ while (numPendingRequests.get() != 0) {
+ bulkProcessor.flush();
+ checkAsyncErrorsAndRequests();
+ }
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ opensearchSinkFunction.close();
+ if (bulkProcessor != null) {
+ bulkProcessor.close();
+ bulkProcessor = null;
+ }
+
+ if (client != null) {
+ client.close();
+ client = null;
+ }
+
+ // make sure any errors from callbacks are rethrown
+ checkErrorAndRethrow();
+ }
+
+ /** Build the {@link BulkProcessor}. */
+ private BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) {
+ checkNotNull(listener);
+
+ BulkProcessor.Builder bulkProcessorBuilder =
+ BulkProcessor.builder(
+ (request, bulkListener) ->
+ client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener),
+ listener);
+
+ // This makes flush() blocking
+ bulkProcessorBuilder.setConcurrentRequests(0);
+
+ if (bulkProcessorFlushMaxActions != null) {
+ bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions);
+ }
+
+ if (bulkProcessorFlushMaxSizeMb != null) {
+ configureBulkSize(bulkProcessorBuilder);
+ }
+
+ if (bulkProcessorFlushIntervalMillis != null) {
+ configureFlushInterval(bulkProcessorBuilder);
+ }
+
+ // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null
+ configureBulkProcessorBackoff(bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy);
+
+ return bulkProcessorBuilder.build();
+ }
+
+ /**
+ * Creates an Opensearch client implementing {@link AutoCloseable}.
+ *
+ * @param clientConfig The configuration to use when constructing the client.
+ * @return The created client.
+ * @throws IOException IOException
+ */
+ private RestHighLevelClient createClient() throws IOException {
+ RestClientBuilder builder =
+ RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]));
+ restClientFactory.configureRestClientBuilder(builder);
+
+ RestHighLevelClient rhlClient = new RestHighLevelClient(builder);
+ verifyClientConnection(rhlClient);
+
+ return rhlClient;
+ }
+
+ /**
+ * Verify the client connection by making a test request/ping to the Opensearch cluster.
+ *
+ * <p>Called by {@link Opensearch2Sink#open(org.apache.flink.configuration.Configuration)} after
+ * creating the client. This makes sure the underlying client is closed if the connection is not
+ * successful and preventing thread leak.
+ *
+ * @param client the Opensearch client.
+ */
+ private void verifyClientConnection(RestHighLevelClient client) throws IOException {
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Pinging Opensearch cluster via hosts {} ...", httpHosts);
+ }
+
+ if (!client.ping(RequestOptions.DEFAULT)) {
+ throw new RuntimeException("There are no reachable Opensearch nodes!");
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Opensearch RestHighLevelClient is connected to {}", httpHosts.toString());
+ }
+ }
+
+ /**
+ * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. The builder
+ * will be later on used to instantiate the actual {@link BulkProcessor}.
+ *
+ * @param builder the {@link BulkProcessor.Builder} to configure.
+ * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user
+ * disabled backoff retries).
+ */
+ private static void configureBulkProcessorBackoff(
+ BulkProcessor.Builder builder,
+ @Nullable Opensearch2Sink.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);
+ }
+
+ private void configureBulkSize(BulkProcessor.Builder bulkProcessorBuilder) {
+ final ByteSizeUnit sizeUnit;
+ if (bulkProcessorFlushMaxSizeMb == -1) {
+ // bulk size can be disabled with -1, however the ByteSizeValue constructor accepts -1
+ // only with BYTES as the size unit
+ sizeUnit = ByteSizeUnit.BYTES;
+ } else {
+ sizeUnit = ByteSizeUnit.MB;
+ }
+ bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, sizeUnit));
+ }
+
+ private void configureFlushInterval(BulkProcessor.Builder bulkProcessorBuilder) {
+ if (bulkProcessorFlushIntervalMillis == -1) {
+ bulkProcessorBuilder.setFlushInterval(null);
+ } else {
+ bulkProcessorBuilder.setFlushInterval(
+ TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis));
+ }
+ }
+
+ private void checkErrorAndRethrow() {
+ Throwable cause = failureThrowable.get();
+ if (cause != null) {
+ throw new RuntimeException("An error occurred in OpensearchSink.", cause);
+ }
+ }
+
+ private void checkAsyncErrorsAndRequests() {
+ checkErrorAndRethrow();
+ failureRequestIndexer.processBufferedRequests(requestIndexer);
+ }
+
+ private class BulkProcessorListener implements BulkProcessor.Listener {
+
+ @Override
+ public void beforeBulk(long executionId, BulkRequest request) {}
+
+ @Override
+ public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
+ if (response.hasFailures()) {
+ BulkItemResponse itemResponse;
+ Throwable failure;
+ RestStatus restStatus;
+ DocWriteRequest actionRequest;
+
+ try {
+ for (int i = 0; i < response.getItems().length; i++) {
+ itemResponse = response.getItems()[i];
+ failure = extractFailureCauseFromBulkItemResponse(itemResponse);
+ if (failure != null) {
+ restStatus = itemResponse.getFailure().getStatus();
+ actionRequest = request.requests().get(i);
+ if (restStatus == null) {
+ if (actionRequest instanceof ActionRequest) {
+ failureHandler.onFailure(
+ (ActionRequest) actionRequest,
+ failure,
+ -1,
+ failureRequestIndexer);
+ } else {
+ throw new UnsupportedOperationException(
+ "The sink currently only supports ActionRequests");
+ }
+ } else {
+ if (actionRequest instanceof ActionRequest) {
+ failureHandler.onFailure(
+ (ActionRequest) actionRequest,
+ failure,
+ restStatus.getStatus(),
+ failureRequestIndexer);
+ } else {
+ throw new UnsupportedOperationException(
+ "The sink currently only supports ActionRequests");
+ }
+ }
+ }
+ }
+ } catch (Throwable t) {
+ // fail the sink and skip the rest of the items
+ // if the failure handler decides to throw an exception
+ failureThrowable.compareAndSet(null, t);
+ }
+ }
+
+ if (flushOnCheckpoint) {
+ numPendingRequests.getAndAdd(-request.numberOfActions());
+ }
+ }
+
+ @Override
+ public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
+ try {
+ for (DocWriteRequest writeRequest : request.requests()) {
+ if (writeRequest instanceof ActionRequest) {
+ failureHandler.onFailure(
+ (ActionRequest) writeRequest, failure, -1, failureRequestIndexer);
+ } else {
+ throw new UnsupportedOperationException(
+ "The sink currently only supports ActionRequests");
+ }
+ }
+ } catch (Throwable t) {
+ // fail the sink and skip the rest of the items
+ // if the failure handler decides to throw an exception
+ failureThrowable.compareAndSet(null, t);
+ }
+
+ if (flushOnCheckpoint) {
+ numPendingRequests.getAndAdd(-request.numberOfActions());
+ }
+ }
+ }
+
+ /**
+ * Extracts the cause of failure of a bulk item action.
+ *
+ * @param bulkItemResponse the bulk item response to extract cause of failure
+ * @return the extracted {@link Throwable} from the response ({@code null} is the response is
+ * successful).
+ */
+ private static Throwable extractFailureCauseFromBulkItemResponse(
+ BulkItemResponse bulkItemResponse) {
+ if (!bulkItemResponse.isFailed()) {
+ return null;
+ } else {
+ return bulkItemResponse.getFailure().getCause();
+ }
+ }
+
+ long getNumPendingRequests() {
+ if (flushOnCheckpoint) {
+ return numPendingRequests.get();
+ } else {
+ throw new UnsupportedOperationException(
+ "The number of pending requests is not maintained when flushing on checkpoint is disabled.");
+ }
+ }
+
+ /**
+ * A builder for creating an {@link Opensearch2Sink}.
+ *
+ * @param <T> Type of the elements handled by the sink this builder creates.
+ * @deprecated This has been deprecated, please use {@link Opensearch2SinkBuilder}.
+ */
+ @Deprecated
+ @PublicEvolving
+ public static class Builder<T> {
+
+ private final List<HttpHost> httpHosts;
+ private final OpensearchSinkFunction<T> opensearchSinkFunction;
+
+ private Map<String, String> bulkRequestsConfig = new HashMap<>();
+ private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler();
+ private RestClientFactory restClientFactory = restClientBuilder -> {};
+
+ /**
+ * Creates a new {@code OpensearchSink} that connects to the cluster using a {@link
+ * RestHighLevelClient}.
+ *
+ * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient}
+ * connects to.
+ * @param opensearchSinkFunction This is used to generate multiple {@link ActionRequest}
+ * from the incoming element.
+ */
+ public Builder(List<HttpHost> httpHosts, OpensearchSinkFunction<T> opensearchSinkFunction) {
+ this.httpHosts = Preconditions.checkNotNull(httpHosts);
+ this.opensearchSinkFunction = Preconditions.checkNotNull(opensearchSinkFunction);
+ }
+
+ /**
+ * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+ * disable it.
+ *
+ * @param numMaxActions the maximum number of actions to buffer per bulk request.
+ */
+ public void setBulkFlushMaxActions(int numMaxActions) {
+ Preconditions.checkArgument(
+ numMaxActions == -1 || numMaxActions > 0,
+ "Max number of buffered actions must be larger than 0.");
+
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions));
+ }
+
+ /**
+ * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to
+ * disable it.
+ *
+ * @param maxSizeMb the maximum size of buffered actions, in mb.
+ */
+ public void setBulkFlushMaxSizeMb(int maxSizeMb) {
+ Preconditions.checkArgument(
+ maxSizeMb == -1 || maxSizeMb > 0,
+ "Max size of buffered actions must be larger than 0.");
+
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb));
+ }
+
+ /**
+ * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+ *
+ * @param intervalMillis the bulk flush interval, in milliseconds.
+ */
+ public void setBulkFlushInterval(long intervalMillis) {
+ Preconditions.checkArgument(
+ intervalMillis == -1 || intervalMillis >= 0,
+ "Interval (in milliseconds) between each flush must be larger than or equal to 0.");
+
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis));
+ }
+
+ /**
+ * Sets whether or not to enable bulk flush backoff behaviour.
+ *
+ * @param enabled whether or not to enable backoffs.
+ */
+ public void setBulkFlushBackoff(boolean enabled) {
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled));
+ }
+
+ /**
+ * Sets the type of back of to use when flushing bulk requests.
+ *
+ * @param flushBackoffType the backoff type to use.
+ */
+ public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) {
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE,
+ Preconditions.checkNotNull(flushBackoffType).toString());
+ }
+
+ /**
+ * Sets the maximum number of retries for a backoff attempt when flushing bulk requests.
+ *
+ * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk
+ * requests
+ */
+ public void setBulkFlushBackoffRetries(int maxRetries) {
+ Preconditions.checkArgument(
+ maxRetries > 0, "Max number of backoff attempts must be larger than 0.");
+
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries));
+ }
+
+ /**
+ * Sets the amount of delay between each backoff attempt when flushing bulk requests, in
+ * milliseconds.
+ *
+ * @param delayMillis the amount of delay between each backoff attempt when flushing bulk
+ * requests, in milliseconds.
+ */
+ public void setBulkFlushBackoffDelay(long delayMillis) {
+ Preconditions.checkArgument(
+ delayMillis >= 0,
+ "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0.");
+ this.bulkRequestsConfig.put(
+ CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis));
+ }
+
+ /**
+ * Sets a failure handler for action requests.
+ *
+ * @param failureHandler This is used to handle failed {@link ActionRequest}.
+ */
+ public void setFailureHandler(ActionRequestFailureHandler failureHandler) {
+ this.failureHandler = Preconditions.checkNotNull(failureHandler);
+ }
+
+ /**
+ * Sets a REST client factory for custom client configuration.
+ *
+ * @param restClientFactory the factory that configures the rest client.
+ */
+ public void setRestClientFactory(RestClientFactory restClientFactory) {
+ this.restClientFactory = Preconditions.checkNotNull(restClientFactory);
+ }
+
+ /**
+ * Creates the Opensearch sink.
+ *
+ * @return the created Opensearch sink.
+ */
+ public Opensearch2Sink<T> build() {
+ return new Opensearch2Sink<>(
+ bulkRequestsConfig,
+ httpHosts,
+ opensearchSinkFunction,
+ failureHandler,
+ restClientFactory);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Builder<?> builder = (Builder<?>) o;
+ return Objects.equals(httpHosts, builder.httpHosts)
+ && Objects.equals(opensearchSinkFunction, builder.opensearchSinkFunction)
+ && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig)
+ && Objects.equals(failureHandler, builder.failureHandler)
+ && Objects.equals(restClientFactory, builder.restClientFactory);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ httpHosts,
+ opensearchSinkFunction,
+ bulkRequestsConfig,
+ failureHandler,
+ restClientFactory);
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-opensearch2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 0000000..32c4a6d
--- /dev/null
+++ b/flink-connector-opensearch2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.flink.connector.opensearch.table.Opensearch2DynamicSinkFactory
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java
new file mode 100644
index 0000000..5b9811c
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** product code Architecture tests. */
+@AnalyzeClasses(
+ packages = "org.apache.flink.connector",
+ importOptions = {
+ ImportOption.DoNotIncludeTests.class,
+ ImportOption.DoNotIncludeArchives.class,
+ ImportOptions.ExcludeScalaImportOption.class,
+ ImportOptions.ExcludeShadedImportOption.class
+ })
+public class ProductionCodeArchitectureTest {
+
+ @ArchTest
+ public static final ArchTests COMMON_TESTS = ArchTests.in(ProductionCodeArchitectureBase.class);
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
new file mode 100644
index 0000000..7e44a8c
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+ packages = {
+ "org.apache.flink.connector.opensearch",
+ "org.apache.flink.streaming.connectors.opensearch"
+ },
+ importOptions = {
+ ImportOption.OnlyIncludeTests.class,
+ ImportOptions.ExcludeScalaImportOption.class,
+ ImportOptions.ExcludeShadedImportOption.class
+ })
+public class TestCodeArchitectureTest {
+
+ @ArchTest
+ public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspectorTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspectorTest.java
new file mode 100644
index 0000000..bc56e1e
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/DefaultBulkResponseInspectorTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.util.FlinkRuntimeException;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.opensearch.action.DocWriteRequest.OpType;
+import org.opensearch.action.DocWriteResponse;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkItemResponse.Failure;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+
+import java.io.IOException;
+
+/** Tests for {@link DefaultBulkResponseInspector}. */
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+class DefaultBulkResponseInspectorTest {
+
+ @Test
+ void testPassWithoutFailures() {
+ final DefaultBulkResponseInspector inspector = new DefaultBulkResponseInspector();
+ Assertions.assertThatCode(
+ () ->
+ inspector.inspect(
+ new BulkRequest(),
+ new BulkResponse(new BulkItemResponse[] {}, 0)))
+ .doesNotThrowAnyException();
+ }
+
+ @Test
+ void testPassesDespiteChainedFailure() {
+ final DefaultBulkResponseInspector inspector =
+ new DefaultBulkResponseInspector((failure) -> {});
+ Assertions.assertThatCode(
+ () -> {
+ final BulkRequest request = new BulkRequest();
+ request.add(
+ new IndexRequest(), new DeleteRequest(), new DeleteRequest());
+
+ inspector.inspect(
+ request,
+ new BulkResponse(
+ new BulkItemResponse[] {
+ new BulkItemResponse(
+ 0, OpType.CREATE, (DocWriteResponse) null),
+ new BulkItemResponse(
+ 1,
+ OpType.DELETE,
+ new Failure(
+ "index",
+ "id",
+ new IOException("A"))),
+ new BulkItemResponse(
+ 2,
+ OpType.DELETE,
+ new Failure(
+ "index",
+ "id",
+ new IOException("B")))
+ },
+ 0));
+ })
+ .doesNotThrowAnyException();
+ }
+
+ @Test
+ void testThrowsChainedFailure() {
+ final IOException failureCause0 = new IOException("A");
+ final IOException failureCause1 = new IOException("B");
+ final DefaultBulkResponseInspector inspector = new DefaultBulkResponseInspector();
+ Assertions.assertThatExceptionOfType(FlinkRuntimeException.class)
+ .isThrownBy(
+ () -> {
+ final BulkRequest request = new BulkRequest();
+ request.add(
+ new IndexRequest(), new DeleteRequest(), new DeleteRequest());
+
+ inspector.inspect(
+ request,
+ new BulkResponse(
+ new BulkItemResponse[] {
+ new BulkItemResponse(
+ 0, OpType.CREATE, (DocWriteResponse) null),
+ new BulkItemResponse(
+ 1,
+ OpType.DELETE,
+ new Failure("index", "id", failureCause0)),
+ new BulkItemResponse(
+ 2,
+ OpType.DELETE,
+ new Failure("index", "id", failureCause1))
+ },
+ 0));
+ })
+ .withCause(failureCause0);
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilderTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilderTest.java
new file mode 100644
index 0000000..fd85a38
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkBuilderTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.connector.base.DeliveryGuarantee;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.http.HttpHost;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatNoException;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link Opensearch2SinkBuilder}. */
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+class Opensearch2SinkBuilderTest {
+
+ @TestFactory
+ Stream<DynamicTest> testValidBuilders() {
+ Stream<Opensearch2SinkBuilder<Object>> validBuilders =
+ Stream.of(
+ createMinimalBuilder(),
+ createMinimalBuilder()
+ .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE),
+ createMinimalBuilder()
+ .setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1),
+ createMinimalBuilder()
+ .setConnectionUsername("username")
+ .setConnectionPassword("password"));
+
+ return DynamicTest.stream(
+ validBuilders,
+ Opensearch2SinkBuilder::toString,
+ builder -> assertThatNoException().isThrownBy(builder::build));
+ }
+
+ @Test
+ void testDefaultDeliveryGuarantee() {
+ assertThat(createMinimalBuilder().build().getDeliveryGuarantee())
+ .isEqualTo(DeliveryGuarantee.AT_LEAST_ONCE);
+ }
+
+ @Test
+ void testThrowIfExactlyOnceConfigured() {
+ assertThatThrownBy(
+ () ->
+ createMinimalBuilder()
+ .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE))
+ .isInstanceOf(IllegalStateException.class);
+ }
+
+ @Test
+ void testThrowIfHostsNotSet() {
+ assertThatThrownBy(
+ () ->
+ createEmptyBuilder()
+ .setEmitter((element, indexer, context) -> {})
+ .build())
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void testThrowIfEmitterNotSet() {
+ assertThatThrownBy(
+ () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build())
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void testThrowIfSetInvalidTimeouts() {
+ assertThatThrownBy(() -> createEmptyBuilder().setConnectionRequestTimeout(-1).build())
+ .isInstanceOf(IllegalStateException.class);
+ assertThatThrownBy(() -> createEmptyBuilder().setConnectionTimeout(-1).build())
+ .isInstanceOf(IllegalStateException.class);
+ assertThatThrownBy(() -> createEmptyBuilder().setSocketTimeout(-1).build())
+ .isInstanceOf(IllegalStateException.class);
+ }
+
+ @Test
+ void testThrowIfRestClientFactoryNotSet() {
+ assertThatThrownBy(() -> createEmptyBuilder().setRestClientFactory(null).build())
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void testThrowIfConnectionPathPrefixNotSet() {
+ assertThatThrownBy(() -> createEmptyBuilder().setConnectionPathPrefix(null).build())
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ private Opensearch2SinkBuilder<Object> createEmptyBuilder() {
+ return new Opensearch2SinkBuilder<>();
+ }
+
+ private Opensearch2SinkBuilder<Object> createMinimalBuilder() {
+ return new Opensearch2SinkBuilder<>()
+ .setEmitter((element, indexer, context) -> {})
+ .setHosts(new HttpHost("localhost:3000"));
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkITCase.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkITCase.java
new file mode 100644
index 0000000..b8d66e3
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2SinkITCase.java
@@ -0,0 +1,199 @@
+/*
+ * 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.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.opensearch.OpensearchUtil;
+import org.apache.flink.connector.opensearch.test.DockerImageVersions;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.apache.http.HttpHost;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.testcontainers.OpensearchContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.BiFunction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link Opensearch2Sink}. */
+@Testcontainers
+@ExtendWith(TestLoggerExtension.class)
+class Opensearch2SinkITCase {
+ protected static final Logger LOG = LoggerFactory.getLogger(Opensearch2SinkITCase.class);
+ private static boolean failed;
+
+ private RestHighLevelClient client;
+ private Opensearch2TestClient context;
+
+ @Container
+ private static final OpensearchContainer OS_CONTAINER =
+ OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_2, LOG);
+
+ @BeforeEach
+ void setUp() {
+ failed = false;
+ client = OpensearchUtil.createClient(OS_CONTAINER);
+ context = new Opensearch2TestClient(client);
+ }
+
+ @AfterEach
+ void tearDown() throws IOException {
+ if (client != null) {
+ client.close();
+ }
+ }
+
+ @ParameterizedTest
+ @EnumSource(DeliveryGuarantee.class)
+ void testWriteToOpensearchWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee)
+ throws Exception {
+ final String index = "test-opensearch-with-delivery-" + deliveryGuarantee;
+ boolean failure = false;
+ try {
+ runTest(index, false, TestEmitter::jsonEmitter, deliveryGuarantee, null);
+ } catch (IllegalStateException e) {
+ failure = true;
+ assertThat(deliveryGuarantee).isSameAs(DeliveryGuarantee.EXACTLY_ONCE);
+ } finally {
+ assertThat(failure).isEqualTo(deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE);
+ }
+ }
+
+ @ParameterizedTest
+ @MethodSource("opensearchEmitters")
+ void testWriteJsonToOpensearch(
+ BiFunction<String, String, OpensearchEmitter<Tuple2<Integer, String>>> emitterProvider)
+ throws Exception {
+ final String index = "test-opensearch-sink-" + UUID.randomUUID();
+ runTest(index, false, emitterProvider, null);
+ }
+
+ @Test
+ void testRecovery() throws Exception {
+ final String index = "test-recovery-opensearch-sink";
+ runTest(index, true, TestEmitter::jsonEmitter, new FailingMapper());
+ assertThat(failed).isTrue();
+ }
+
+ private void runTest(
+ String index,
+ boolean allowRestarts,
+ BiFunction<String, String, OpensearchEmitter<Tuple2<Integer, String>>> emitterProvider,
+ @Nullable MapFunction<Long, Long> additionalMapper)
+ throws Exception {
+ runTest(
+ index,
+ allowRestarts,
+ emitterProvider,
+ DeliveryGuarantee.AT_LEAST_ONCE,
+ additionalMapper);
+ }
+
+ private void runTest(
+ String index,
+ boolean allowRestarts,
+ BiFunction<String, String, OpensearchEmitter<Tuple2<Integer, String>>> emitterProvider,
+ DeliveryGuarantee deliveryGuarantee,
+ @Nullable MapFunction<Long, Long> additionalMapper)
+ throws Exception {
+ final Opensearch2Sink<Tuple2<Integer, String>> sink =
+ new Opensearch2SinkBuilder<>()
+ .setHosts(HttpHost.create(OS_CONTAINER.getHttpHostAddress()))
+ .setEmitter(emitterProvider.apply(index, context.getDataFieldName()))
+ .setBulkFlushMaxActions(5)
+ .setConnectionUsername(OS_CONTAINER.getUsername())
+ .setConnectionPassword(OS_CONTAINER.getPassword())
+ .setDeliveryGuarantee(deliveryGuarantee)
+ .setAllowInsecure(true)
+ .build();
+
+ final StreamExecutionEnvironment env = new LocalStreamEnvironment();
+ env.enableCheckpointing(100L);
+ if (!allowRestarts) {
+ env.setRestartStrategy(RestartStrategies.noRestart());
+ }
+ DataStream<Long> stream = env.fromSequence(1, 5);
+
+ if (additionalMapper != null) {
+ stream = stream.map(additionalMapper);
+ }
+
+ stream.map(
+ new MapFunction<Long, Tuple2<Integer, String>>() {
+ @Override
+ public Tuple2<Integer, String> map(Long value) throws Exception {
+ return Tuple2.of(
+ value.intValue(),
+ Opensearch2TestClient.buildMessage(value.intValue()));
+ }
+ })
+ .sinkTo(sink);
+ env.execute();
+ context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5);
+ }
+
+ private static List<BiFunction<String, String, OpensearchEmitter<Tuple2<Integer, String>>>>
+ opensearchEmitters() {
+ return Arrays.asList(TestEmitter::jsonEmitter, TestEmitter::smileEmitter);
+ }
+
+ private static class FailingMapper implements MapFunction<Long, Long>, CheckpointListener {
+
+ private int emittedRecords = 0;
+
+ @Override
+ public Long map(Long value) throws Exception {
+ Thread.sleep(50);
+ emittedRecords++;
+ return value;
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) throws Exception {
+ if (failed || emittedRecords == 0) {
+ return;
+ }
+ failed = true;
+ throw new Exception("Expected failure");
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2TestClient.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2TestClient.java
new file mode 100644
index 0000000..5f981c6
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2TestClient.java
@@ -0,0 +1,74 @@
+/*
+ * 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.opensearch.OpenSearchStatusException;
+import org.opensearch.action.get.GetRequest;
+import org.opensearch.action.get.GetResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestHighLevelClient;
+
+import java.io.IOException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class Opensearch2TestClient {
+ private static final String DATA_FIELD_NAME = "data";
+ private final RestHighLevelClient client;
+
+ Opensearch2TestClient(RestHighLevelClient client) {
+ this.client = client;
+ }
+
+ GetResponse getResponse(String index, int id) throws IOException {
+ return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT);
+ }
+
+ void assertThatIdsAreNotWritten(String index, int... ids) throws IOException {
+ for (final int id : ids) {
+ try {
+ final GetResponse response = getResponse(index, id);
+ assertThat(response.isExists())
+ .isFalse()
+ .as(String.format("Id %s is unexpectedly present.", id));
+ } catch (OpenSearchStatusException e) {
+ assertThat(e.status().getStatus()).isEqualTo(404);
+ }
+ }
+ }
+
+ void assertThatIdsAreWritten(String index, int... ids)
+ throws IOException, InterruptedException {
+ for (final int id : ids) {
+ GetResponse response;
+ do {
+ response = getResponse(index, id);
+ Thread.sleep(10);
+ } while (response.isSourceEmpty());
+ assertThat(response.getSource().get(DATA_FIELD_NAME)).isEqualTo(buildMessage(id));
+ }
+ }
+
+ String getDataFieldName() {
+ return DATA_FIELD_NAME;
+ }
+
+ static String buildMessage(int id) {
+ return "test-" + id;
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2WriterITCase.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2WriterITCase.java
new file mode 100644
index 0000000..92d0c09
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/Opensearch2WriterITCase.java
@@ -0,0 +1,404 @@
+/*
+ * 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.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.opensearch.OpensearchUtil;
+import org.apache.flink.connector.opensearch.test.DockerImageVersions;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.apache.http.HttpHost;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.testcontainers.OpensearchContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.flink.connector.opensearch.sink.Opensearch2TestClient.buildMessage;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link Opensearch2Writer}. */
+@Testcontainers
+@ExtendWith(TestLoggerExtension.class)
+class Opensearch2WriterITCase {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Opensearch2WriterITCase.class);
+
+ @Container
+ private static final OpensearchContainer OS_CONTAINER =
+ OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_2, LOG);
+
+ private RestHighLevelClient client;
+ private Opensearch2TestClient context;
+ private MetricListener metricListener;
+
+ @BeforeEach
+ void setUp() {
+ metricListener = new MetricListener();
+ client = OpensearchUtil.createClient(OS_CONTAINER);
+ context = new Opensearch2TestClient(client);
+ }
+
+ @AfterEach
+ void tearDown() throws IOException {
+ if (client != null) {
+ client.close();
+ }
+ }
+
+ @Test
+ void testWriteOnBulkFlush() throws Exception {
+ final String index = "test-bulk-flush-without-checkpoint";
+ final int flushAfterNActions = 5;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, false, bulkProcessorConfig)) {
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+ writer.write(Tuple2.of(3, buildMessage(3)), null);
+ writer.write(Tuple2.of(4, buildMessage(4)), null);
+
+ // Ignore flush on checkpoint
+ writer.flush(false);
+
+ context.assertThatIdsAreNotWritten(index, 1, 2, 3, 4);
+
+ // Trigger flush
+ writer.write(Tuple2.of(5, "test-5"), null);
+ context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5);
+
+ writer.write(Tuple2.of(6, "test-6"), null);
+ context.assertThatIdsAreNotWritten(index, 6);
+
+ // Force flush
+ writer.blockingFlushAllActions();
+ context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5, 6);
+ }
+ }
+
+ @Test
+ void testWriteOnBulkIntervalFlush() throws Exception {
+ final String index = "test-bulk-flush-with-interval";
+
+ // Configure bulk processor to flush every 1s;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0);
+
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, false, bulkProcessorConfig)) {
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+ writer.write(Tuple2.of(3, buildMessage(3)), null);
+ writer.write(Tuple2.of(4, buildMessage(4)), null);
+ writer.blockingFlushAllActions();
+ }
+
+ context.assertThatIdsAreWritten(index, 1, 2, 3, 4);
+ }
+
+ @Test
+ void testWriteOnCheckpoint() throws Exception {
+ final String index = "test-bulk-flush-with-checkpoint";
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ // Enable flush on checkpoint
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, true, bulkProcessorConfig)) {
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+ writer.write(Tuple2.of(3, buildMessage(3)), null);
+
+ context.assertThatIdsAreNotWritten(index, 1, 2, 3);
+
+ // Trigger flush
+ writer.flush(false);
+
+ context.assertThatIdsAreWritten(index, 1, 2, 3);
+ }
+ }
+
+ @Test
+ void testIncrementByteOutMetric() throws Exception {
+ final String index = "test-inc-byte-out";
+ final OperatorIOMetricGroup operatorIOMetricGroup =
+ UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup();
+ final int flushAfterNActions = 2;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(
+ index,
+ false,
+ bulkProcessorConfig,
+ TestingSinkWriterMetricGroup.getSinkWriterMetricGroup(
+ operatorIOMetricGroup, metricListener.getMetricGroup()))) {
+ final Counter numBytesOut = operatorIOMetricGroup.getNumBytesOutCounter();
+ assertThat(numBytesOut.getCount()).isEqualTo(0);
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+
+ writer.blockingFlushAllActions();
+ long first = numBytesOut.getCount();
+
+ assertThat(first).isGreaterThan(0);
+
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+
+ writer.blockingFlushAllActions();
+ assertThat(numBytesOut.getCount()).isGreaterThan(first);
+ }
+ }
+
+ @Test
+ void testIncrementRecordsSendMetric() throws Exception {
+ final String index = "test-inc-records-send";
+ final int flushAfterNActions = 2;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, false, bulkProcessorConfig)) {
+ final Optional<Counter> recordsSend =
+ metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ // Update existing index
+ writer.write(Tuple2.of(1, "u" + buildMessage(2)), null);
+ // Delete index
+ writer.write(Tuple2.of(1, "d" + buildMessage(3)), null);
+
+ writer.blockingFlushAllActions();
+
+ assertThat(recordsSend).isPresent();
+ assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+ }
+ }
+
+ @Test
+ void testCurrentSendTime() throws Exception {
+ final String index = "test-current-send-time";
+ final int flushAfterNActions = 2;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, false, bulkProcessorConfig)) {
+ final Optional<Gauge<Long>> currentSendTime =
+ metricListener.getGauge("currentSendTime");
+ writer.write(Tuple2.of(1, buildMessage(1)), null);
+ writer.write(Tuple2.of(2, buildMessage(2)), null);
+
+ writer.blockingFlushAllActions();
+
+ assertThat(currentSendTime).isPresent();
+ assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);
+ }
+ }
+
+ private static class TestHandler implements FailureHandler {
+ private boolean failed = false;
+
+ private synchronized void setFailed() {
+ failed = true;
+ }
+
+ public boolean isFailed() {
+ return failed;
+ }
+
+ @Override
+ public void onFailure(Throwable failure) {
+ setFailed();
+ }
+ }
+
+ @Test
+ void testWriteErrorOnUpdate() throws Exception {
+ final String index = "test-bulk-flush-with-error";
+ final int flushAfterNActions = 1;
+ final BulkProcessorConfig bulkProcessorConfig =
+ new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0);
+
+ final TestHandler testHandler = new TestHandler();
+ try (final Opensearch2Writer<Tuple2<Integer, String>> writer =
+ createWriter(index, true, bulkProcessorConfig, testHandler)) {
+ // Trigger an error by updating non-existing document
+ writer.write(Tuple2.of(1, "u" + buildMessage(1)), null);
+ context.assertThatIdsAreNotWritten(index, 1);
+ assertThat(testHandler.isFailed()).isEqualTo(true);
+ }
+ }
+
+ private Opensearch2Writer<Tuple2<Integer, String>> createWriter(
+ String index, boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig) {
+ return createWriter(
+ index,
+ flushOnCheckpoint,
+ bulkProcessorConfig,
+ TestingSinkWriterMetricGroup.getSinkWriterMetricGroup(
+ metricListener.getMetricGroup()),
+ new Opensearch2Writer.DefaultFailureHandler());
+ }
+
+ private Opensearch2Writer<Tuple2<Integer, String>> createWriter(
+ String index,
+ boolean flushOnCheckpoint,
+ BulkProcessorConfig bulkProcessorConfig,
+ FailureHandler failureHandler) {
+ return createWriter(
+ index,
+ flushOnCheckpoint,
+ bulkProcessorConfig,
+ TestingSinkWriterMetricGroup.getSinkWriterMetricGroup(
+ metricListener.getMetricGroup()),
+ failureHandler);
+ }
+
+ private Opensearch2Writer<Tuple2<Integer, String>> createWriter(
+ String index,
+ boolean flushOnCheckpoint,
+ BulkProcessorConfig bulkProcessorConfig,
+ SinkWriterMetricGroup metricGroup) {
+ return createWriter(
+ index,
+ flushOnCheckpoint,
+ bulkProcessorConfig,
+ metricGroup,
+ new DefaultFailureHandler());
+ }
+
+ private Opensearch2Writer<Tuple2<Integer, String>> createWriter(
+ String index,
+ boolean flushOnCheckpoint,
+ BulkProcessorConfig bulkProcessorConfig,
+ SinkWriterMetricGroup metricGroup,
+ FailureHandler failureHandler) {
+ return new Opensearch2Writer<Tuple2<Integer, String>>(
+ Collections.singletonList(HttpHost.create(OS_CONTAINER.getHttpHostAddress())),
+ new UpdatingEmitter(index, context.getDataFieldName()),
+ flushOnCheckpoint,
+ bulkProcessorConfig,
+ new NetworkClientConfig(
+ OS_CONTAINER.getUsername(),
+ OS_CONTAINER.getPassword(),
+ null,
+ null,
+ null,
+ null,
+ true),
+ metricGroup,
+ new TestMailbox(),
+ new DefaultRestClientFactory(),
+ new DefaultBulkResponseInspector(failureHandler));
+ }
+
+ private static class UpdatingEmitter implements OpensearchEmitter<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+
+ private final String dataFieldName;
+ private final String index;
+
+ UpdatingEmitter(String index, String dataFieldName) {
+ this.index = index;
+ this.dataFieldName = dataFieldName;
+ }
+
+ @Override
+ public void emit(
+ Tuple2<Integer, String> element,
+ SinkWriter.Context context,
+ RequestIndexer indexer) {
+
+ Map<String, Object> document = new HashMap<>();
+ document.put(dataFieldName, element.f1);
+
+ final char action = element.f1.charAt(0);
+ final String id = element.f0.toString();
+ switch (action) {
+ case 'd':
+ {
+ indexer.add(new DeleteRequest(index).id(id));
+ break;
+ }
+ case 'u':
+ {
+ indexer.add(new UpdateRequest().index(index).id(id).doc(document));
+ break;
+ }
+ default:
+ {
+ indexer.add(new IndexRequest(index).id(id).source(document));
+ }
+ }
+ }
+ }
+
+ private static class TestMailbox implements MailboxExecutor {
+
+ @Override
+ public void execute(
+ ThrowingRunnable<? extends Exception> command,
+ String descriptionFormat,
+ Object... descriptionArgs) {
+ try {
+ command.run();
+ } catch (Exception e) {
+ throw new RuntimeException("Unexpected error", e);
+ }
+ }
+
+ @Override
+ public void yield() throws InterruptedException, FlinkRuntimeException {
+ Thread.sleep(100);
+ }
+
+ @Override
+ public boolean tryYield() throws FlinkRuntimeException {
+ return false;
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java
new file mode 100644
index 0000000..f58682b
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/sink/TestEmitter.java
@@ -0,0 +1,75 @@
+/*
+ * 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.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.common.xcontent.XContentFactory;
+import org.opensearch.core.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+class TestEmitter implements OpensearchEmitter<Tuple2<Integer, String>> {
+
+ private final String index;
+ private final XContentBuilderProvider xContentBuilderProvider;
+ private final String dataFieldName;
+
+ public static TestEmitter jsonEmitter(String index, String dataFieldName) {
+ return new TestEmitter(index, dataFieldName, XContentFactory::jsonBuilder);
+ }
+
+ public static TestEmitter smileEmitter(String index, String dataFieldName) {
+ return new TestEmitter(index, dataFieldName, XContentFactory::smileBuilder);
+ }
+
+ private TestEmitter(
+ String index, String dataFieldName, XContentBuilderProvider xContentBuilderProvider) {
+ this.dataFieldName = dataFieldName;
+ this.index = index;
+ this.xContentBuilderProvider = xContentBuilderProvider;
+ }
+
+ @Override
+ public void emit(
+ Tuple2<Integer, String> element, SinkWriter.Context context, RequestIndexer indexer) {
+ indexer.add(createIndexRequest(element));
+ }
+
+ public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
+ Map<String, Object> document = new HashMap<>();
+ document.put(dataFieldName, element.f1);
+ try {
+ return new IndexRequest(index)
+ .id(element.f0.toString())
+ .source(xContentBuilderProvider.getBuilder().map(document));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @FunctionalInterface
+ private interface XContentBuilderProvider extends Serializable {
+ XContentBuilder getBuilder() throws IOException;
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java
new file mode 100644
index 0000000..7e88328
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/IndexGeneratorTest.java
@@ -0,0 +1,335 @@
+/*
+ * 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.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+
+import org.junit.jupiter.api.Test;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Suite tests for {@link IndexGenerator}. */
+class IndexGeneratorTest {
+
+ private static final List<String> fieldNames =
+ Arrays.asList(
+ "id",
+ "item",
+ "log_ts",
+ "log_date",
+ "order_timestamp",
+ "log_time",
+ "local_datetime",
+ "local_date",
+ "local_time",
+ "note",
+ "status");
+
+ private static final List<DataType> dataTypes =
+ Arrays.asList(
+ DataTypes.INT(),
+ DataTypes.STRING(),
+ DataTypes.BIGINT(),
+ DataTypes.DATE().bridgedTo(Date.class),
+ DataTypes.TIMESTAMP().bridgedTo(Timestamp.class),
+ DataTypes.TIME().bridgedTo(Time.class),
+ DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class),
+ DataTypes.DATE().bridgedTo(LocalDate.class),
+ DataTypes.TIME().bridgedTo(LocalTime.class),
+ DataTypes.STRING(),
+ DataTypes.BOOLEAN());
+
+ private static final List<RowData> rows =
+ Arrays.asList(
+ GenericRowData.of(
+ 1,
+ StringData.fromString("apple"),
+ Timestamp.valueOf("2020-03-18 12:12:14").getTime(),
+ (int) Date.valueOf("2020-03-18").toLocalDate().toEpochDay(),
+ TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-18 12:12:14")),
+ (int)
+ (Time.valueOf("12:12:14").toLocalTime().toNanoOfDay()
+ / 1_000_000L),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)),
+ (int) LocalDate.of(2020, 3, 18).toEpochDay(),
+ (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L),
+ "test1",
+ true),
+ GenericRowData.of(
+ 2,
+ StringData.fromString("peanut"),
+ Timestamp.valueOf("2020-03-19 12:22:14").getTime(),
+ (int) Date.valueOf("2020-03-19").toLocalDate().toEpochDay(),
+ TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-19 12:22:21")),
+ (int)
+ (Time.valueOf("12:22:21").toLocalTime().toNanoOfDay()
+ / 1_000_000L),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)),
+ (int) LocalDate.of(2020, 3, 19).toEpochDay(),
+ (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L),
+ "test2",
+ false));
+
+ @Test
+ public void testDynamicIndexFromTimestamp() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{order_timestamp|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("2020_03_18_12-14_index");
+ IndexGenerator indexGenerator1 =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{order_timestamp|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes);
+ indexGenerator1.open();
+ assertThat(indexGenerator1.generate(rows.get(1))).isEqualTo("2020_03_19_12_22_index");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalDateTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{local_datetime|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("2020_03_18_12-14_index");
+ IndexGenerator indexGenerator1 =
+ IndexGeneratorFactory.createIndexGenerator(
+ "{local_datetime|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes);
+ indexGenerator1.open();
+ assertThat(indexGenerator1.generate(rows.get(1))).isEqualTo("2020_03_19_12_22_index");
+ }
+
+ @Test
+ public void testDynamicIndexFromDate() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{log_date|yyyy/MM/dd}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-2020/03/18");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-2020/03/19");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalDate() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date|yyyy/MM/dd}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-2020/03/18");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-2020/03/19");
+ }
+
+ @Test
+ public void testDynamicIndexFromTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{log_time|HH-mm}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12-12");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12-22");
+ }
+
+ @Test
+ public void testDynamicIndexFromLocalTime() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_time|HH-mm}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12-13");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12-13");
+ }
+
+ @Test
+ public void testDynamicIndexDefaultFormat() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_time|}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index-12_13_14");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index-12_13_14");
+ }
+
+ @Test
+ public void testGeneralDynamicIndex() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator("index_{item}", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("index_apple");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("index_peanut");
+ }
+
+ @Test
+ public void testStaticIndex() {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator("my-index", fieldNames, dataTypes);
+ indexGenerator.open();
+ assertThat(indexGenerator.generate(rows.get(0))).isEqualTo("my-index");
+ assertThat(indexGenerator.generate(rows.get(1))).isEqualTo("my-index");
+ }
+
+ @Test
+ public void testUnknownField() {
+ String expectedExceptionMsg =
+ "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}',"
+ + " please check the field name.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{unknown_ts|yyyy-MM-dd}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedTimeType() {
+ String expectedExceptionMsg =
+ "Unsupported type 'INT' found in Opensearch dynamic index field, "
+ + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{id|yyyy-MM-dd}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedMultiParametersType() {
+ String expectedExceptionMsg =
+ "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported,"
+ + " only support single dynamic index pattern.";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date}-{local_time}", fieldNames, dataTypes);
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testDynamicIndexUnsupportedFormat() {
+ String expectedExceptionMsg = "Unsupported field: HourOfDay";
+ try {
+ IndexGeneratorFactory.createIndexGenerator(
+ "my-index-{local_date|yyyy/MM/dd HH:mm}", fieldNames, dataTypes);
+ } catch (UnsupportedTemporalTypeException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testUnsupportedIndexFieldType() {
+ String expectedExceptionMsg =
+ "Unsupported type BOOLEAN of index field, Supported types are:"
+ + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE,"
+ + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]";
+ try {
+ IndexGeneratorFactory.createIndexGenerator("index_{status}", fieldNames, dataTypes);
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ }
+
+ @Test
+ public void testDynamicIndexFromSystemTime() {
+ List<String> supportedUseCases =
+ Arrays.asList(
+ "now()",
+ "NOW()",
+ "now( )",
+ "NOW(\t)",
+ "\t NOW( ) \t",
+ "current_timestamp",
+ "CURRENT_TIMESTAMP",
+ "\tcurrent_timestamp\t",
+ " current_timestamp ");
+
+ supportedUseCases.stream()
+ .forEach(
+ f -> {
+ DateTimeFormatter dateTimeFormatter =
+ DateTimeFormatter.ofPattern("yyyy_MM_dd");
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ String.format("my-index-{%s|yyyy_MM_dd}", f),
+ fieldNames,
+ dataTypes);
+ indexGenerator.open();
+ // The date may change during the running of the unit test.
+ // Generate expected index-name based on the current time
+ // before and after calling the generate method.
+ String expectedIndex1 =
+ "my-index-" + LocalDateTime.now().format(dateTimeFormatter);
+ String actualIndex = indexGenerator.generate(rows.get(1));
+ String expectedIndex2 =
+ "my-index-" + LocalDateTime.now().format(dateTimeFormatter);
+ assertThat(
+ actualIndex.equals(expectedIndex1)
+ || actualIndex.equals(expectedIndex2))
+ .isTrue();
+ });
+
+ List<String> invalidUseCases =
+ Arrays.asList(
+ "now",
+ "now(",
+ "NOW",
+ "NOW)",
+ "current_timestamp()",
+ "CURRENT_TIMESTAMP()",
+ "CURRENT_timestamp");
+ invalidUseCases.stream()
+ .forEach(
+ f -> {
+ String expectedExceptionMsg =
+ String.format(
+ "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}',"
+ + " please check the field name.",
+ f, f);
+ try {
+ IndexGenerator indexGenerator =
+ IndexGeneratorFactory.createIndexGenerator(
+ String.format("my-index-{%s|yyyy_MM_dd}", f),
+ fieldNames,
+ dataTypes);
+ indexGenerator.open();
+ } catch (TableException e) {
+ assertThat(e.getMessage()).isEqualTo(expectedExceptionMsg);
+ }
+ });
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java
new file mode 100644
index 0000000..31ef457
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/KeyExtractorTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+
+import org.junit.jupiter.api.Test;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link KeyExtractor}. */
+class KeyExtractorTest {
+ @Test
+ public void testSimpleKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.BIGINT().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+ assertThat(key).isEqualTo("12");
+ }
+
+ @Test
+ public void testNoPrimaryKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex = Collections.emptyList();
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+ assertThat(key).isNull();
+ }
+
+ @Test
+ public void testTwoFieldsKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.BIGINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 2, DataTypes.TIMESTAMP().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key =
+ keyExtractor.apply(
+ GenericRowData.of(
+ 12L,
+ StringData.fromString("ABCD"),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2012-12-12T12:12:12"))));
+ assertThat(key).isEqualTo("12_2012-12-12T12:12:12");
+ }
+
+ @Test
+ public void testAllTypesKey() {
+ List<LogicalTypeWithIndex> logicalTypesWithIndex =
+ Stream.of(
+ new LogicalTypeWithIndex(
+ 0, DataTypes.TINYINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 1, DataTypes.SMALLINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 2, DataTypes.INT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 3, DataTypes.BIGINT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 4, DataTypes.BOOLEAN().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 5, DataTypes.FLOAT().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 6, DataTypes.DOUBLE().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 7, DataTypes.STRING().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 8, DataTypes.TIMESTAMP().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 9,
+ DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()
+ .notNull()
+ .getLogicalType()),
+ new LogicalTypeWithIndex(
+ 10, DataTypes.TIME().notNull().getLogicalType()),
+ new LogicalTypeWithIndex(
+ 11, DataTypes.DATE().notNull().getLogicalType()))
+ .collect(Collectors.toList());
+
+ Function<RowData, String> keyExtractor =
+ KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_");
+
+ String key =
+ keyExtractor.apply(
+ GenericRowData.of(
+ (byte) 1,
+ (short) 2,
+ 3,
+ (long) 4,
+ true,
+ 1.0f,
+ 2.0d,
+ StringData.fromString("ABCD"),
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2012-12-12T12:12:12")),
+ TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")),
+ (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000),
+ (int) LocalDate.parse("2015-05-15").toEpochDay()));
+ assertThat(key)
+ .isEqualTo(
+ "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15");
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactoryTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactoryTest.java
new file mode 100644
index 0000000..2a12452
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkFactoryTest.java
@@ -0,0 +1,292 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.typeutils.base.VoidSerializer;
+import org.apache.flink.connector.opensearch.OpensearchUtil;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.assertj.core.api.ThrowableAssert;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for validation in {@link Opensearch2DynamicSinkFactory}. */
+@ExtendWith(TestLoggerExtension.class)
+class Opensearch2DynamicSinkFactoryTest {
+ private TestContext createPrefilledTestContext() {
+ return TestContext.context()
+ .withOption(OpensearchConnectorOptions.INDEX_OPTION.key(), "MyIndex")
+ .withOption(
+ OpensearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345");
+ }
+
+ @Test
+ public void validateEmptyConfiguration() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "One or more required options are missing.\n"
+ + "\n"
+ + "Missing required options are:\n"
+ + "\n"
+ + "hosts\n"
+ + "index",
+ () -> sinkFactory.createDynamicTableSink(TestContext.context().build()));
+ }
+
+ void assertValidationException(
+ String expectedMessage, ThrowableAssert.ThrowingCallable executable) {
+ assertThatThrownBy(executable)
+ .isInstanceOf(ValidationException.class)
+ .hasMessage(expectedMessage);
+ }
+
+ @Test
+ public void validateWrongIndex() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+ assertValidationException(
+ "'index' must not be empty",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions.INDEX_OPTION.key(), "")
+ .build()));
+ }
+
+ @Test
+ public void validateWrongHosts() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+ assertValidationException(
+ "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'.",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions.HOSTS_OPTION.key(),
+ "wrong-host")
+ .build()));
+ }
+
+ @Test
+ public void validateWrongFlushSize() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+ assertValidationException(
+ "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions
+ .BULK_FLUSH_MAX_SIZE_OPTION
+ .key(),
+ "1kb")
+ .build()));
+ }
+
+ @Test
+ public void validateWrongRetries() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions
+ .BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION
+ .key(),
+ "0")
+ .build()));
+ }
+
+ @Test
+ public void validateWrongMaxActions() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "'sink.bulk-flush.max-actions' must be at least 1. Got: -2",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions
+ .BULK_FLUSH_MAX_ACTIONS_OPTION
+ .key(),
+ "-2")
+ .build()));
+ }
+
+ @Test
+ public void validateWrongBackoffDelay() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "Invalid value for option 'sink.bulk-flush.backoff.delay'.",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions
+ .BULK_FLUSH_BACKOFF_DELAY_OPTION
+ .key(),
+ "-1s")
+ .build()));
+ }
+
+ @Test
+ public void validatePrimaryKeyOnIllegalColumn() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "The table has a primary key on columns of illegal types: "
+ + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withSchema(
+ new ResolvedSchema(
+ Arrays.asList(
+ Column.physical(
+ "a",
+ DataTypes.BIGINT()
+ .notNull()),
+ Column.physical(
+ "b",
+ DataTypes.ARRAY(
+ DataTypes
+ .BIGINT()
+ .notNull())
+ .notNull()),
+ Column.physical(
+ "c",
+ DataTypes.MAP(
+ DataTypes
+ .BIGINT(),
+ DataTypes
+ .STRING())
+ .notNull()),
+ Column.physical(
+ "d",
+ DataTypes.MULTISET(
+ DataTypes
+ .BIGINT()
+ .notNull())
+ .notNull()),
+ Column.physical(
+ "e",
+ DataTypes.ROW(
+ DataTypes
+ .FIELD(
+ "a",
+ DataTypes
+ .BIGINT()))
+ .notNull()),
+ Column.physical(
+ "f",
+ DataTypes.RAW(
+ Void.class,
+ VoidSerializer
+ .INSTANCE)
+ .notNull()),
+ Column.physical(
+ "g",
+ DataTypes.BYTES()
+ .notNull())),
+ Collections.emptyList(),
+ UniqueConstraint.primaryKey(
+ "name",
+ Arrays.asList(
+ "a", "b", "c", "d", "e",
+ "f", "g"))))
+ .build()));
+ }
+
+ @Test
+ public void validateWrongCredential() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ assertValidationException(
+ "'username' and 'password' must be set at the same time. Got: username 'username' and password ''",
+ () ->
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions.USERNAME_OPTION.key(),
+ "username")
+ .withOption(
+ OpensearchConnectorOptions.PASSWORD_OPTION.key(),
+ "")
+ .build()));
+ }
+
+ @Test
+ public void testSinkParallelism() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+ DynamicTableSink sink =
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(SINK_PARALLELISM.key(), "2")
+ .build());
+ assertThat(sink).isInstanceOf(Opensearch2DynamicSink.class);
+ Opensearch2DynamicSink opensearchSink = (Opensearch2DynamicSink) sink;
+ SinkV2Provider provider =
+ (SinkV2Provider)
+ opensearchSink.getSinkRuntimeProvider(new OpensearchUtil.MockContext());
+ assertThat(2).isEqualTo(provider.getParallelism().get());
+ }
+
+ @Test
+ public void validateDynamicIndexOnChangelogStream() {
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+ DynamicTableSink sink =
+ sinkFactory.createDynamicTableSink(
+ createPrefilledTestContext()
+ .withOption(
+ OpensearchConnectorOptions.INDEX_OPTION.key(),
+ "dynamic-index-{now()|yyyy-MM-dd}_index")
+ .build());
+
+ ChangelogMode changelogMode =
+ ChangelogMode.newBuilder()
+ .addContainedKind(RowKind.DELETE)
+ .addContainedKind(RowKind.INSERT)
+ .build();
+ assertValidationException(
+ "Dynamic indexing based on system time only works on append only stream.",
+ () -> sink.getChangelogMode(changelogMode));
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkITCase.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkITCase.java
new file mode 100644
index 0000000..aeee80e
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/Opensearch2DynamicSinkITCase.java
@@ -0,0 +1,396 @@
+/*
+ * 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.table;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.opensearch.OpensearchUtil;
+import org.apache.flink.connector.opensearch.test.DockerImageVersions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.opensearch.OpenSearchStatusException;
+import org.opensearch.action.get.GetRequest;
+import org.opensearch.action.search.SearchRequest;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.search.SearchHits;
+import org.opensearch.testcontainers.OpensearchContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT tests for {@link OpensearchDynamicSink}. */
+@ExtendWith(TestLoggerExtension.class)
+@Testcontainers
+class Opensearch2DynamicSinkITCase {
+ private static final Logger LOG = LoggerFactory.getLogger(Opensearch2DynamicSinkITCase.class);
+
+ @Container
+ private static final OpensearchContainer OS_CONTAINER =
+ OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_2, LOG);
+
+ private TestContext getPrefilledTestContext(String index) {
+ return TestContext.context()
+ .withOption(OpensearchConnectorOptions.INDEX_OPTION.key(), index)
+ .withOption(
+ OpensearchConnectorOptions.HOSTS_OPTION.key(),
+ OS_CONTAINER.getHttpHostAddress())
+ .withOption(OpensearchConnectorOptions.ALLOW_INSECURE.key(), "true")
+ .withOption(
+ OpensearchConnectorOptions.USERNAME_OPTION.key(),
+ OS_CONTAINER.getUsername())
+ .withOption(
+ OpensearchConnectorOptions.PASSWORD_OPTION.key(),
+ OS_CONTAINER.getPassword());
+ }
+
+ private String getConnectorSql(String index) {
+ return String.format("'%s'='%s',\n", "connector", "opensearch-2")
+ + String.format(
+ "'%s'='%s',\n", OpensearchConnectorOptions.INDEX_OPTION.key(), index)
+ + String.format(
+ "'%s'='%s', \n",
+ OpensearchConnectorOptions.HOSTS_OPTION.key(),
+ OS_CONTAINER.getHttpHostAddress())
+ + String.format(
+ "'%s'='%s', \n",
+ OpensearchConnectorOptions.USERNAME_OPTION.key(),
+ OS_CONTAINER.getUsername())
+ + String.format(
+ "'%s'='%s', \n",
+ OpensearchConnectorOptions.PASSWORD_OPTION.key(),
+ OS_CONTAINER.getPassword())
+ + String.format(
+ "'%s'='%s'\n", OpensearchConnectorOptions.ALLOW_INSECURE.key(), true);
+ }
+
+ @Test
+ public void testWritingDocuments() throws Exception {
+ ResolvedSchema schema =
+ new ResolvedSchema(
+ Arrays.asList(
+ Column.physical("a", DataTypes.BIGINT().notNull()),
+ Column.physical("b", DataTypes.TIME()),
+ Column.physical("c", DataTypes.STRING().notNull()),
+ Column.physical("d", DataTypes.FLOAT()),
+ Column.physical("e", DataTypes.TINYINT().notNull()),
+ Column.physical("f", DataTypes.DATE()),
+ Column.physical("g", DataTypes.TIMESTAMP().notNull())),
+ Collections.emptyList(),
+ UniqueConstraint.primaryKey("name", Arrays.asList("a", "g")));
+ GenericRowData rowData =
+ GenericRowData.of(
+ 1L,
+ 12345,
+ StringData.fromString("ABCDE"),
+ 12.12f,
+ (byte) 2,
+ 12345,
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2012-12-12T12:12:12")));
+
+ String index = "writing-documents";
+ Opensearch2DynamicSinkFactory sinkFactory = new Opensearch2DynamicSinkFactory();
+
+ DynamicTableSink.SinkRuntimeProvider runtimeProvider =
+ sinkFactory
+ .createDynamicTableSink(
+ getPrefilledTestContext(index).withSchema(schema).build())
+ .getSinkRuntimeProvider(new OpensearchUtil.MockContext());
+
+ final SinkV2Provider sinkProvider = (SinkV2Provider) runtimeProvider;
+ final Sink<RowData> sink = sinkProvider.createSink();
+ StreamExecutionEnvironment environment =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ environment.setParallelism(4);
+
+ rowData.setRowKind(RowKind.UPDATE_AFTER);
+ environment.<RowData>fromElements(rowData).sinkTo(sink);
+ environment.execute();
+
+ RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+ Map<String, Object> response =
+ client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT)
+ .getSource();
+ Map<Object, Object> expectedMap = new HashMap<>();
+ expectedMap.put("a", 1);
+ expectedMap.put("b", "00:00:12");
+ expectedMap.put("c", "ABCDE");
+ expectedMap.put("d", 12.12d);
+ expectedMap.put("e", 2);
+ expectedMap.put("f", "2003-10-20");
+ expectedMap.put("g", "2012-12-12 12:12:12");
+ assertThat(response).isEqualTo(expectedMap);
+ }
+
+ @Test
+ public void testWritingDocumentsFromTableApi() throws Exception {
+ TableEnvironment tableEnvironment =
+ TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+ String index = "table-api";
+ tableEnvironment.executeSql(
+ "CREATE TABLE osTable ("
+ + "a BIGINT NOT NULL,\n"
+ + "b TIME,\n"
+ + "c STRING NOT NULL,\n"
+ + "d FLOAT,\n"
+ + "e TINYINT NOT NULL,\n"
+ + "f DATE,\n"
+ + "g TIMESTAMP NOT NULL,\n"
+ + "h as a + 2,\n"
+ + "PRIMARY KEY (a, g) NOT ENFORCED\n"
+ + ")\n"
+ + "WITH (\n"
+ + getConnectorSql(index)
+ + ")");
+
+ tableEnvironment
+ .fromValues(
+ row(
+ 1L,
+ LocalTime.ofNanoOfDay(12345L * 1_000_000L),
+ "ABCDE",
+ 12.12f,
+ (byte) 2,
+ LocalDate.ofEpochDay(12345),
+ LocalDateTime.parse("2012-12-12T12:12:12")))
+ .executeInsert("osTable")
+ .await();
+
+ RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+ Map<String, Object> response =
+ client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT)
+ .getSource();
+ Map<Object, Object> expectedMap = new HashMap<>();
+ expectedMap.put("a", 1);
+ expectedMap.put("b", "00:00:12");
+ expectedMap.put("c", "ABCDE");
+ expectedMap.put("d", 12.12d);
+ expectedMap.put("e", 2);
+ expectedMap.put("f", "2003-10-20");
+ expectedMap.put("g", "2012-12-12 12:12:12");
+ assertThat(response).isEqualTo(expectedMap);
+ }
+
+ @Test
+ public void testWritingDocumentsNoPrimaryKey() throws Exception {
+ TableEnvironment tableEnvironment =
+ TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+ String index = "no-primary-key";
+ tableEnvironment.executeSql(
+ "CREATE TABLE osTable ("
+ + "a BIGINT NOT NULL,\n"
+ + "b TIME,\n"
+ + "c STRING NOT NULL,\n"
+ + "d FLOAT,\n"
+ + "e TINYINT NOT NULL,\n"
+ + "f DATE,\n"
+ + "g TIMESTAMP NOT NULL\n"
+ + ")\n"
+ + "WITH (\n"
+ + getConnectorSql(index)
+ + ")");
+
+ tableEnvironment
+ .fromValues(
+ row(
+ 1L,
+ LocalTime.ofNanoOfDay(12345L * 1_000_000L),
+ "ABCDE",
+ 12.12f,
+ (byte) 2,
+ LocalDate.ofEpochDay(12345),
+ LocalDateTime.parse("2012-12-12T12:12:12")),
+ row(
+ 2L,
+ LocalTime.ofNanoOfDay(12345L * 1_000_000L),
+ "FGHIJK",
+ 13.13f,
+ (byte) 4,
+ LocalDate.ofEpochDay(12345),
+ LocalDateTime.parse("2013-12-12T13:13:13")))
+ .executeInsert("osTable")
+ .await();
+
+ RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+
+ // search API does not return documents that were not indexed, we might need to query
+ // the index a few times
+ Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30));
+ SearchHits hits;
+ do {
+ hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits();
+ if (hits.getTotalHits().value < 2) {
+ Thread.sleep(200);
+ }
+ } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft());
+
+ if (hits.getTotalHits().value < 2) {
+ throw new AssertionError("Could not retrieve results from Opensearch.");
+ }
+
+ HashSet<Map<String, Object>> resultSet = new HashSet<>();
+ resultSet.add(hits.getAt(0).getSourceAsMap());
+ resultSet.add(hits.getAt(1).getSourceAsMap());
+ Map<Object, Object> expectedMap1 = new HashMap<>();
+ expectedMap1.put("a", 1);
+ expectedMap1.put("b", "00:00:12");
+ expectedMap1.put("c", "ABCDE");
+ expectedMap1.put("d", 12.12d);
+ expectedMap1.put("e", 2);
+ expectedMap1.put("f", "2003-10-20");
+ expectedMap1.put("g", "2012-12-12 12:12:12");
+ Map<Object, Object> expectedMap2 = new HashMap<>();
+ expectedMap2.put("a", 2);
+ expectedMap2.put("b", "00:00:12");
+ expectedMap2.put("c", "FGHIJK");
+ expectedMap2.put("d", 13.13d);
+ expectedMap2.put("e", 4);
+ expectedMap2.put("f", "2003-10-20");
+ expectedMap2.put("g", "2013-12-12 13:13:13");
+ HashSet<Map<Object, Object>> expectedSet = new HashSet<>();
+ expectedSet.add(expectedMap1);
+ expectedSet.add(expectedMap2);
+ assertThat(resultSet).isEqualTo(expectedSet);
+ }
+
+ @Test
+ public void testWritingDocumentsWithDynamicIndex() throws Exception {
+ TableEnvironment tableEnvironment =
+ TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+ String index = "dynamic-index-{b|yyyy-MM-dd}";
+ tableEnvironment.executeSql(
+ "CREATE TABLE osTable ("
+ + "a BIGINT NOT NULL,\n"
+ + "b TIMESTAMP NOT NULL,\n"
+ + "PRIMARY KEY (a) NOT ENFORCED\n"
+ + ")\n"
+ + "WITH (\n"
+ + getConnectorSql(index)
+ + ")");
+
+ tableEnvironment
+ .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12")))
+ .executeInsert("osTable")
+ .await();
+
+ RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+ Map<String, Object> response =
+ client.get(new GetRequest("dynamic-index-2012-12-12", "1"), RequestOptions.DEFAULT)
+ .getSource();
+ Map<Object, Object> expectedMap = new HashMap<>();
+ expectedMap.put("a", 1);
+ expectedMap.put("b", "2012-12-12 12:12:12");
+ assertThat(response).isEqualTo(expectedMap);
+ }
+
+ @Test
+ public void testWritingDocumentsWithDynamicIndexFromSystemTime() throws Exception {
+ TableEnvironment tableEnvironment =
+ TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+ DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");
+ tableEnvironment
+ .getConfig()
+ .getConfiguration()
+ .setString("table.local-time-zone", "Asia/Shanghai");
+
+ String dynamicIndex1 =
+ "dynamic-index-"
+ + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai")))
+ + "_index";
+ String index = "dynamic-index-{now()|yyyy-MM-dd}_index";
+ tableEnvironment.executeSql(
+ "CREATE TABLE esTable ("
+ + "a BIGINT NOT NULL,\n"
+ + "b TIMESTAMP NOT NULL,\n"
+ + "PRIMARY KEY (a) NOT ENFORCED\n"
+ + ")\n"
+ + "WITH (\n"
+ + getConnectorSql(index)
+ + ")");
+ String dynamicIndex2 =
+ "dynamic-index-"
+ + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai")))
+ + "_index";
+
+ tableEnvironment
+ .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12")))
+ .executeInsert("esTable")
+ .await();
+
+ RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+
+ Map<String, Object> response;
+ try {
+ response =
+ client.get(new GetRequest(dynamicIndex1, "1"), RequestOptions.DEFAULT)
+ .getSource();
+ } catch (OpenSearchStatusException e) {
+ if (e.status() == RestStatus.NOT_FOUND) {
+ response =
+ client.get(new GetRequest(dynamicIndex2, "1"), RequestOptions.DEFAULT)
+ .getSource();
+ } else {
+ throw e;
+ }
+ }
+
+ Map<Object, Object> expectedMap = new HashMap<>();
+ expectedMap.put("a", 1);
+ expectedMap.put("b", "2012-12-12 12:12:12");
+ assertThat(response).isEqualTo(expectedMap);
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java
new file mode 100644
index 0000000..50da696
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch/table/TestContext.java
@@ -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.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** A utility class for mocking {@link DynamicTableFactory.Context}. */
+class TestContext {
+
+ private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME()));
+
+ private final Map<String, String> options = new HashMap<>();
+
+ public static TestContext context() {
+ return new TestContext();
+ }
+
+ public TestContext withSchema(ResolvedSchema schema) {
+ this.schema = schema;
+ return this;
+ }
+
+ DynamicTableFactory.Context build() {
+ return new FactoryUtil.DefaultDynamicTableContext(
+ ObjectIdentifier.of("default", "default", "t1"),
+ new ResolvedCatalogTable(
+ CatalogTable.of(
+ Schema.newBuilder().fromResolvedSchema(schema).build(),
+ "mock context",
+ Collections.emptyList(),
+ options),
+ schema),
+ Collections.emptyMap(),
+ new Configuration(),
+ TestContext.class.getClassLoader(),
+ false);
+ }
+
+ public TestContext withOption(String key, String value) {
+ options.put(key, value);
+ return this;
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkITCase.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkITCase.java
new file mode 100644
index 0000000..106c1c8
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkITCase.java
@@ -0,0 +1,157 @@
+/*
+ * 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.opensearch;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.opensearch.OpensearchUtil;
+import org.apache.flink.connector.opensearch.test.DockerImageVersions;
+import org.apache.flink.runtime.JobException;
+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.connectors.opensearch.testutils.SourceSinkDataTestKit;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.http.HttpHost;
+import org.junit.jupiter.api.Test;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.testcontainers.OpensearchContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** IT cases for the {@link Opensearch2Sink}. */
+@Testcontainers
+public class Opensearch2SinkITCase extends AbstractTestBase {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Opensearch2SinkITCase.class);
+
+ @Container
+ private static final OpensearchContainer OS_CONTAINER =
+ OpensearchUtil.createOpensearchContainer(DockerImageVersions.OPENSEARCH_2, LOG);
+
+ @Test
+ public void testOpensearchSink() throws Exception {
+ runOpensearchSinkTest(
+ "opensearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction);
+ }
+
+ @Test
+ public void testOpensearchSinkWithSmile() throws Exception {
+ runOpensearchSinkTest(
+ "opensearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction);
+ }
+
+ @Test
+ public void testNullAddresses() {
+ assertThatThrownBy(
+ () ->
+ createOpensearchSink(
+ 1, null, SourceSinkDataTestKit.getJsonSinkFunction("test")))
+ .isInstanceOfAny(IllegalArgumentException.class, NullPointerException.class);
+ }
+
+ @Test
+ public void testEmptyAddresses() {
+ assertThatThrownBy(
+ () ->
+ createOpensearchSink(
+ 1,
+ Collections.emptyList(),
+ SourceSinkDataTestKit.getJsonSinkFunction("test")))
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testInvalidOpensearchCluster() throws Exception {
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+ DataStreamSource<Tuple2<Integer, String>> source =
+ env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction());
+
+ source.addSink(
+ createOpensearchSinkForNode(
+ 1,
+ SourceSinkDataTestKit.getJsonSinkFunction("test"),
+ "123.123.123.123")); // incorrect ip address
+
+ assertThatThrownBy(() -> env.execute("Opensearch Sink Test"))
+ .isInstanceOf(JobExecutionException.class)
+ .hasCauseInstanceOf(JobException.class);
+ }
+
+ private Opensearch2Sink<Tuple2<Integer, String>> createOpensearchSink(
+ int bulkFlushMaxActions,
+ List<HttpHost> httpHosts,
+ OpensearchSinkFunction<Tuple2<Integer, String>> opensearchSinkFunction) {
+
+ Opensearch2Sink.Builder<Tuple2<Integer, String>> builder =
+ new Opensearch2Sink.Builder<>(httpHosts, opensearchSinkFunction);
+ builder.setBulkFlushMaxActions(bulkFlushMaxActions);
+
+ return builder.build();
+ }
+
+ private Opensearch2Sink<Tuple2<Integer, String>> createOpensearchSinkForNode(
+ int bulkFlushMaxActions,
+ OpensearchSinkFunction<Tuple2<Integer, String>> opensearchSinkFunction,
+ String hostAddress) {
+
+ ArrayList<HttpHost> httpHosts = new ArrayList<>();
+ httpHosts.add(HttpHost.create(hostAddress));
+
+ Opensearch2Sink.Builder<Tuple2<Integer, String>> builder =
+ new Opensearch2Sink.Builder<>(httpHosts, opensearchSinkFunction);
+ builder.setBulkFlushMaxActions(bulkFlushMaxActions);
+ builder.setRestClientFactory(OpensearchUtil.createClientFactory(OS_CONTAINER));
+
+ return builder.build();
+ }
+
+ private void runOpensearchSinkTest(
+ String index,
+ Function<String, OpensearchSinkFunction<Tuple2<Integer, String>>> functionFactory)
+ throws Exception {
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+ DataStreamSource<Tuple2<Integer, String>> source =
+ env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction());
+
+ source.addSink(
+ createOpensearchSinkForNode(
+ 1, functionFactory.apply(index), OS_CONTAINER.getHttpHostAddress()));
+
+ env.execute("Opensearch Sink Test");
+
+ // verify the results
+ final RestHighLevelClient client = OpensearchUtil.createClient(OS_CONTAINER);
+
+ SourceSinkDataTestKit.verifyProducedSinkData(client, index);
+
+ client.close();
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkTest.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkTest.java
new file mode 100644
index 0000000..cd02d19
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/Opensearch2SinkTest.java
@@ -0,0 +1,581 @@
+/*
+ * 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.opensearch;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.opensearch.util.NoOpFailureHandler;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+
+import org.apache.http.HttpHost;
+import org.apache.http.HttpResponse;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.entity.ContentType;
+import org.apache.http.impl.bootstrap.HttpServer;
+import org.apache.http.impl.bootstrap.ServerBootstrap;
+import org.apache.http.protocol.HttpRequestHandlerMapper;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.DocWriteRequest.OpType;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkItemResponse.Failure;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.action.index.IndexResponse;
+import org.opensearch.client.Requests;
+import org.opensearch.common.xcontent.json.JsonXContent;
+import org.opensearch.core.index.shard.ShardId;
+import org.opensearch.core.xcontent.ToXContent;
+import org.opensearch.core.xcontent.XContentBuilder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Suite of tests for {@link Opensearch2Sink}. */
+public class Opensearch2SinkTest {
+ private HttpServer server;
+ private final Deque<Consumer<HttpResponse>> responses = new ConcurrentLinkedDeque<>();
+ private final Lock lock = new ReentrantLock();
+ private final Condition flushed = lock.newCondition();
+
+ @BeforeEach
+ public void setUp() throws IOException {
+ final HttpRequestHandlerMapper handlers =
+ (request) -> {
+ final String method = request.getRequestLine().getMethod();
+ if (method.equalsIgnoreCase("HEAD")) {
+ // Connection request always OKed
+ return (req, resp, context) -> resp.setStatusCode(200);
+ } else if (method.equalsIgnoreCase("POST")) {
+ // Bulk responses are configured per test case
+ return (req, resp, context) -> {
+ lock.lock();
+ try {
+ responses.poll().accept(resp);
+ flushed.signalAll();
+ } finally {
+ lock.unlock();
+ }
+ };
+ } else {
+ return null;
+ }
+ };
+ server = ServerBootstrap.bootstrap().setHandlerMapper(handlers).create();
+ server.start();
+ }
+
+ @AfterEach
+ public void tearDown() {
+ server.stop();
+ server = null;
+ responses.clear();
+ }
+
+ /**
+ * Tests that any item failure in the listener callbacks is rethrown on an immediately following
+ * invoke call.
+ */
+ @Test
+ public void testItemFailureRethrownOnInvoke() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushMaxActions(1);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new Failure(
+ "test",
+ "1",
+ new Exception("artificial failure for record")))));
+ testHarness.open();
+
+ // setup the next bulk request, and its mock item failures
+ testHarness.processElement(new StreamRecord<>("msg"));
+
+ assertThatThrownBy(() -> testHarness.processElement(new StreamRecord<>("next msg")))
+ .getCause()
+ .hasMessageContaining("artificial failure for record");
+ }
+
+ /**
+ * Tests that any item failure in the listener callbacks is rethrown on an immediately following
+ * checkpoint.
+ */
+ @Test
+ public void testItemFailureRethrownOnCheckpoint() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushMaxActions(1);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new Failure(
+ "test",
+ "1",
+ new Exception("artificial failure for record")))));
+ testHarness.processElement(new StreamRecord<>("msg"));
+
+ assertThatThrownBy(() -> testHarness.snapshot(1L, 1000L))
+ .getCause()
+ .getCause()
+ .hasMessageContaining("artificial failure for record");
+ }
+
+ /**
+ * Tests that any item failure in the listener callbacks due to flushing on an immediately
+ * following checkpoint is rethrown; we set a timeout because the test will not finish if the
+ * logic is broken.
+ */
+ @Test
+ @Timeout(5)
+ public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushInterval(1000);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new IndexResponse(
+ new ShardId("test", "-", 0), "1", 0, 0, 1, true))));
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 2,
+ OpType.INDEX,
+ new Failure(
+ "test",
+ "2",
+ new Exception("artificial failure for record")))));
+
+ testHarness.processElement(new StreamRecord<>("msg-1"));
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ // setup the requests to be flushed in the snapshot
+ testHarness.processElement(new StreamRecord<>("msg-2"));
+ // let the snapshot-triggered flush continue (2 records in the bulk, so the 2nd one should
+ // fail)
+ testHarness.processElement(new StreamRecord<>("msg-3"));
+
+ CheckedThread snapshotThread =
+ new CheckedThread() {
+ @Override
+ public void go() throws Exception {
+ testHarness.snapshot(1L, 1000L);
+ }
+ };
+ snapshotThread.start();
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ assertThatThrownBy(snapshotThread::sync)
+ .getCause()
+ .getCause()
+ .hasMessageContaining("artificial failure for record");
+ }
+
+ /**
+ * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following
+ * invoke call.
+ */
+ @Test
+ public void testBulkFailureRethrownOnInvoke() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushMaxActions(1);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ // Let the whole bulk request fail
+ responses.add(response -> response.setStatusCode(500));
+
+ testHarness.processElement(new StreamRecord<>("msg"));
+
+ assertThatThrownBy(() -> testHarness.processElement(new StreamRecord<>("next msg")))
+ .getCause()
+ .hasMessageContaining("Unable to parse response body");
+ }
+
+ /**
+ * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following
+ * checkpoint.
+ */
+ @Test
+ public void testBulkFailureRethrownOnCheckpoint() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushMaxActions(1);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ // Let the whole bulk request fail
+ responses.add(response -> response.setStatusCode(500));
+
+ testHarness.processElement(new StreamRecord<>("msg"));
+
+ assertThatThrownBy(() -> testHarness.snapshot(1L, 1000L))
+ .getCause()
+ .getCause()
+ .hasMessageContaining("Unable to parse response body");
+ }
+
+ /**
+ * Tests that any bulk failure in the listener callbacks due to flushing on an immediately
+ * following checkpoint is rethrown; we set a timeout because the test will not finish if the
+ * logic is broken.
+ */
+ @Test
+ @Timeout(5)
+ public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushInterval(1000);
+ builder.setFailureHandler(new NoOpFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new IndexResponse(
+ new ShardId("test", "-", 0), "1", 0, 0, 1, true))));
+
+ // Let the whole bulk request fail
+ responses.add(response -> response.setStatusCode(500));
+
+ // setup the next bulk request, and let bulk request succeed
+ testHarness.processElement(new StreamRecord<>("msg-1"));
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ // setup the requests to be flushed in the snapshot
+ testHarness.processElement(new StreamRecord<>("msg-2"));
+ testHarness.processElement(new StreamRecord<>("msg-3"));
+
+ CheckedThread snapshotThread =
+ new CheckedThread() {
+ @Override
+ public void go() throws Exception {
+ testHarness.snapshot(1L, 1000L);
+ }
+ };
+ snapshotThread.start();
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ assertThatThrownBy(snapshotThread::sync)
+ .getCause()
+ .getCause()
+ .hasMessageContaining("Unable to parse response body");
+ }
+
+ /**
+ * Tests that the sink correctly waits for pending requests (including re-added requests) on
+ * checkpoints; we set a timeout because the test will not finish if the logic is broken.
+ */
+ @Test
+ @Timeout(5)
+ public void testAtLeastOnceSink() throws Throwable {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+ builder.setBulkFlushInterval(1000);
+ // use a failure handler that simply re-adds requests
+ builder.setFailureHandler(new DummyRetryFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ // setup the next bulk request, and its mock item failures;
+ // it contains 1 request, which will fail and re-added to the next bulk request
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new Failure(
+ "test",
+ "1",
+ new Exception("artificial failure for record")))));
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 2,
+ OpType.INDEX,
+ new IndexResponse(
+ new ShardId("test", "-", 0), "2", 0, 0, 1, true))));
+
+ testHarness.processElement(new StreamRecord<>("msg"));
+
+ // current number of pending request should be 1 due to the re-add
+ assertThat(sink.getNumPendingRequests()).isEqualTo(1);
+
+ CheckedThread snapshotThread =
+ new CheckedThread() {
+ @Override
+ public void go() throws Exception {
+ testHarness.snapshot(1L, 1000L);
+ }
+ };
+ snapshotThread.start();
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ // since the previous flush should have resulted in a request re-add from the failure
+ // handler,
+ // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd
+ // flush
+
+ // current number of pending request should be 1 due to the re-add, since the
+ // failureRequestIndexer will be called only on the next bulk flush interval, we may need
+ // to wait for numPendingRequests to be updated.
+ awaitForCondition(() -> sink.getNumPendingRequests() == 1);
+
+ // Await for flush to be complete
+ awaitForFlushToFinish();
+
+ // the snapshot should finish with no exceptions
+ snapshotThread.sync();
+
+ testHarness.close();
+ }
+
+ /**
+ * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is
+ * disabled, the snapshot method does indeed finishes without waiting for pending requests; we
+ * set a timeout because the test will not finish if the logic is broken.
+ */
+ @Test
+ @Timeout(5)
+ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception {
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ new SimpleSinkFunction<String>());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ sink.disableFlushOnCheckpoint(); // disable flushing
+
+ final OneInputStreamOperatorTestHarness<String, Object> testHarness =
+ new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+ testHarness.open();
+
+ responses.add(
+ createResponse(
+ new BulkItemResponse(
+ 1,
+ OpType.INDEX,
+ new Failure(
+ "test",
+ "1",
+ new Exception("artificial failure for record")))));
+
+ testHarness.processElement(new StreamRecord<>("msg-1"));
+
+ // the snapshot should not block even though we haven't flushed the bulk request
+ testHarness.snapshot(1L, 1000L);
+
+ assertThatThrownBy(() -> testHarness.close())
+ .getCause()
+ .hasMessageContaining("artificial failure for record");
+ }
+
+ @Test
+ public void testOpenAndCloseInSinkFunction() throws Exception {
+ final SimpleClosableSinkFunction<String> sinkFunction = new SimpleClosableSinkFunction<>();
+ final Opensearch2Sink.Builder<String> builder =
+ new Opensearch2Sink.Builder<>(
+ Arrays.asList(new HttpHost("localhost", server.getLocalPort())),
+ sinkFunction);
+ builder.setFailureHandler(new DummyRetryFailureHandler());
+
+ final Opensearch2Sink<String> sink = builder.build();
+ sink.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 0));
+ sink.open(new Configuration());
+ sink.close();
+
+ assertThat(sinkFunction.openCalled).isTrue();
+ assertThat(sinkFunction.closeCalled).isTrue();
+ }
+
+ private static class SimpleSinkFunction<String> implements OpensearchSinkFunction<String> {
+ private static final long serialVersionUID = -176739293659135148L;
+
+ @Override
+ public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
+ Map<java.lang.String, Object> json = new HashMap<>();
+ json.put("data", element);
+
+ indexer.add(Requests.indexRequest().index("index").id("id").source(json));
+ }
+ }
+
+ private static class SimpleClosableSinkFunction<String>
+ implements OpensearchSinkFunction<String> {
+
+ private static final long serialVersionUID = 1872065917794006848L;
+
+ private boolean openCalled;
+ private boolean closeCalled;
+
+ @Override
+ public void open() {
+ openCalled = true;
+ }
+
+ @Override
+ public void close() {
+ closeCalled = true;
+ }
+
+ @Override
+ public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {}
+ }
+
+ private static class DummyRetryFailureHandler implements ActionRequestFailureHandler {
+ private static final long serialVersionUID = 5400023700099200745L;
+
+ @Override
+ public void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer)
+ throws Throwable {
+ indexer.add(action);
+ }
+ }
+
+ private static Consumer<HttpResponse> createResponse(BulkItemResponse item) {
+ return response -> {
+ try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+ response.setStatusCode(200);
+ try (XContentBuilder builder =
+ new XContentBuilder(JsonXContent.jsonXContent, baos)) {
+ final BulkResponse bulkResponse =
+ new BulkResponse(new BulkItemResponse[] {item}, 200);
+ bulkResponse.toXContent(builder, ToXContent.EMPTY_PARAMS);
+ }
+ response.setEntity(
+ new ByteArrayEntity(baos.toByteArray(), ContentType.APPLICATION_JSON));
+ } catch (final IOException ex) {
+ response.setStatusCode(500);
+ }
+ };
+ }
+
+ private static void awaitForCondition(Supplier<Boolean> condition) throws InterruptedException {
+ while (!condition.get()) {
+ Thread.sleep(1);
+ }
+ }
+
+ private void awaitForFlushToFinish() throws InterruptedException {
+ lock.lock();
+ try {
+ flushed.await();
+ } finally {
+ lock.unlock();
+ }
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java
new file mode 100644
index 0000000..ebd1198
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/java/org/apache/flink/streaming/connectors/opensearch/testutils/SourceSinkDataTestKit.java
@@ -0,0 +1,144 @@
+/*
+ * 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.opensearch.testutils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.opensearch.OpensearchSinkFunction;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+
+import org.opensearch.action.get.GetRequest;
+import org.opensearch.action.get.GetResponse;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.common.xcontent.XContentFactory;
+import org.opensearch.core.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * This class contains utilities and a pre-defined source function and Opensearch Sink function used
+ * to simulate and verify data used in tests.
+ */
+public class SourceSinkDataTestKit {
+
+ private static final int NUM_ELEMENTS = 20;
+
+ private static final String DATA_PREFIX = "message #";
+ private static final String DATA_FIELD_NAME = "data";
+
+ /**
+ * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 -
+ * 20.
+ */
+ public static class TestDataSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+
+ private volatile boolean running = true;
+
+ @Override
+ public void run(SourceFunction.SourceContext<Tuple2<Integer, String>> ctx)
+ throws Exception {
+ for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+ ctx.collect(Tuple2.of(i, DATA_PREFIX + i));
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ }
+
+ public static OpensearchSinkFunction<Tuple2<Integer, String>> getJsonSinkFunction(
+ String index) {
+ return new TestOpensearchSinkFunction(index, XContentFactory::jsonBuilder);
+ }
+
+ public static OpensearchSinkFunction<Tuple2<Integer, String>> getSmileSinkFunction(
+ String index) {
+ return new TestOpensearchSinkFunction(index, XContentFactory::smileBuilder);
+ }
+
+ private static class TestOpensearchSinkFunction
+ implements OpensearchSinkFunction<Tuple2<Integer, String>> {
+ private static final long serialVersionUID = 1L;
+
+ private final String index;
+ private final XContentBuilderProvider contentBuilderProvider;
+
+ /**
+ * Create the sink function, specifying a target Opensearch index.
+ *
+ * @param index Name of the target Opensearch index.
+ */
+ public TestOpensearchSinkFunction(
+ String index, XContentBuilderProvider contentBuilderProvider) {
+ this.index = index;
+ this.contentBuilderProvider = contentBuilderProvider;
+ }
+
+ public IndexRequest createIndexRequest(Tuple2<Integer, String> element) {
+ Map<String, Object> document = new HashMap<>();
+ document.put(DATA_FIELD_NAME, element.f1);
+
+ try {
+ return new IndexRequest(index)
+ .id(element.f0.toString())
+ .source(contentBuilderProvider.getBuilder().map(document));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void process(
+ Tuple2<Integer, String> element, RuntimeContext ctx, RequestIndexer indexer) {
+ indexer.add(createIndexRequest(element));
+ }
+ }
+
+ /**
+ * Verify the results in an Opensearch index. The results must first be produced into the index
+ * using a {@link TestOpensearchSinkFunction};
+ *
+ * @param client The client to use to connect to Opensearch
+ * @param index The index to check
+ * @throws IOException IOException
+ */
+ public static void verifyProducedSinkData(RestHighLevelClient client, String index)
+ throws IOException {
+ for (int i = 0; i < NUM_ELEMENTS; i++) {
+ GetResponse response =
+ client.get(new GetRequest(index, Integer.toString(i)), RequestOptions.DEFAULT);
+ assertThat(response.getSource().get(DATA_FIELD_NAME)).isEqualTo(DATA_PREFIX + i);
+ }
+ }
+
+ @FunctionalInterface
+ private interface XContentBuilderProvider extends Serializable {
+ XContentBuilder getBuilder() throws IOException;
+ }
+}
diff --git a/flink-connector-opensearch2/src/test/resources/archunit.properties b/flink-connector-opensearch2/src/test/resources/archunit.properties
new file mode 100644
index 0000000..48011f9
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/resources/archunit.properties
@@ -0,0 +1,33 @@
+#
+# 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.
+#
+
+# By default we allow removing existing violations, but fail when new violations are added.
+freeze.store.default.allowStoreUpdate=true
+
+# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
+#freeze.store.default.allowStoreCreation=true
+
+# Enable this to add allow new violations to be recorded.
+# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
+# violation, please try to avoid creating the violation. If the violation was created due to a
+# shortcoming of the rule, file a JIRA issue so the rule can be improved.
+#freeze.refreeze=true
+
+freeze.store.default.path=archunit-violations
+
+archRule.failOnEmptyShould = false
diff --git a/flink-connector-opensearch2/src/test/resources/log4j2-test.properties b/flink-connector-opensearch2/src/test/resources/log4j2-test.properties
new file mode 100644
index 0000000..835c2ec
--- /dev/null
+++ b/flink-connector-opensearch2/src/test/resources/log4j2-test.properties
@@ -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.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+rootLogger.level = OFF
+rootLogger.appenderRef.test.ref = TestLogger
+
+appender.testlogger.name = TestLogger
+appender.testlogger.type = CONSOLE
+appender.testlogger.target = SYSTEM_ERR
+appender.testlogger.layout.type = PatternLayout
+appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-sql-connector-opensearch/pom.xml b/flink-sql-connector-opensearch/pom.xml
index 8f3cdc1..cdcf927 100644
--- a/flink-sql-connector-opensearch/pom.xml
+++ b/flink-sql-connector-opensearch/pom.xml
@@ -121,8 +121,12 @@
</filters>
<relocations>
<relocation>
- <pattern>org.apache.commons</pattern>
- <shadedPattern>org.apache.flink.opensearch.shaded.org.apache.commons</shadedPattern>
+ <pattern>org.apache.commons.codec</pattern>
+ <shadedPattern>org.apache.flink.opensearch.shaded.org.apache.commons.codec</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.commons.logging</pattern>
+ <shadedPattern>org.apache.flink.opensearch.shaded.org.apache.commons.logging</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.http</pattern>
diff --git a/flink-sql-connector-opensearch2/pom.xml b/flink-sql-connector-opensearch2/pom.xml
new file mode 100644
index 0000000..6015d0b
--- /dev/null
+++ b/flink-sql-connector-opensearch2/pom.xml
@@ -0,0 +1,185 @@
+<?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-connector-opensearch-parent</artifactId>
+ <version>2.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>flink-sql-connector-opensearch2</artifactId>
+ <name>Flink : Connectors : SQL : Opensearch : 2</name>
+
+ <packaging>jar</packaging>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-opensearch2</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>shade-flink</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <artifactSet>
+ <includes>
+ <include>*:*</include>
+ </includes>
+ <excludes>
+ <!-- These dependencies are not required. -->
+ <exclude>com.tdunning:t-digest</exclude>
+ <exclude>joda-time:joda-time</exclude>
+ <exclude>net.sf.jopt-simple:jopt-simple</exclude>
+ <exclude>net.java.dev.jna:jna</exclude>
+ <exclude>org.hdrhistogram:HdrHistogram</exclude>
+ <exclude>org.yaml:snakeyaml</exclude>
+ </excludes>
+ </artifactSet>
+ <filters>
+ <!-- Unless otherwise noticed these filters only serve to reduce the size of the resulting
+ jar by removing unnecessary files -->
+ <filter>
+ <artifact>org.opensearch:opensearch</artifact>
+ <excludes>
+ <exclude>config/**</exclude>
+ <exclude>modules.txt</exclude>
+ <exclude>plugins.txt</exclude>
+ <exclude>org/joda/**</exclude>
+ <!-- Exclude bootstrap due to license issues -->
+ <exclude>org/opensearch/bootstrap/**</exclude>
+ <exclude>extensions/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>org.opensearch.client:opensearch-rest-high-level-client</artifact>
+ <excludes>
+ <exclude>forbidden/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>org.apache.httpcomponents:httpclient</artifact>
+ <excludes>
+ <exclude>mozilla/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>org.apache.lucene:lucene-analysis-common</artifact>
+ <excludes>
+ <exclude>org/tartarus/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>com.github.luben:zstd-jni</artifact>
+ <excludes>
+ <exclude>darwin/**</exclude>
+ <exclude>linux/**</exclude>
+ <exclude>win/**</exclude>
+ <exclude>freebsd/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <!-- exclude Java 9 specific classes as otherwise the shade-plugin crashes -->
+ <exclude>META-INF/versions/**</exclude>
+ <exclude>META-INF/services/com.fasterxml.**</exclude>
+ <exclude>META-INF/services/org.apache.lucene.**</exclude>
+ <exclude>META-INF/services/org.opensearch.**</exclude>
+ <exclude>META-INF/LICENSE.txt</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ <relocations>
+ <relocation>
+ <pattern>org.apache.commons.codec</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.apache.commons.codec</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.commons.logging</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.apache.commons.logging</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.http</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.apache.http</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.lucene</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.apache.lucene</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.opensearch</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.opensearch</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.fasterxml.jackson</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.com.fasterxml.jackson</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.carrotsearch.hppc</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.com.carrotsearch.hppc</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.github.mustachejava</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.com.github.mustachejava</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>net.jpountz</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.net.jpountz</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.github.luben.zstd</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.com.github.luben.zstd-jni</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.logging.log4j.jul</pattern>
+ <shadedPattern>org.apache.flink.opensearch2.shaded.org.apache.logging.log4j.jul</shadedPattern>
+ </relocation>
+ </relocations>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+</project>
diff --git a/flink-sql-connector-opensearch2/src/main/resources/META-INF/NOTICE b/flink-sql-connector-opensearch2/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..60c7232
--- /dev/null
+++ b/flink-sql-connector-opensearch2/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,55 @@
+flink-sql-connector-opensearch
+Copyright 2014-2023 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+- com.fasterxml.jackson.core:jackson-core:2.15.3
+- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.15.3
+- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.15.3
+- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.15.3
+- com.github.spullara.mustache.java:compiler:0.9.10
+- commons-codec:commons-codec:1.15
+- commons-logging:commons-logging:1.2
+- org.apache.httpcomponents:httpasyncclient:4.1.5
+- org.apache.httpcomponents:httpclient:4.5.13
+- org.apache.httpcomponents:httpcore:4.4.14
+- org.apache.httpcomponents:httpcore-nio:4.4.12
+- org.apache.logging.log4j:log4j-jul:2.20.0
+- org.apache.lucene:lucene-backward-codecs:9.7.0
+- org.apache.lucene:lucene-analysis-common:9.7.0
+- org.apache.lucene:lucene-core:9.7.0
+- org.apache.lucene:lucene-grouping:9.7.0
+- org.apache.lucene:lucene-highlighter:9.7.0
+- org.apache.lucene:lucene-join:9.7.0
+- org.apache.lucene:lucene-memory:9.7.0
+- org.apache.lucene:lucene-misc:9.7.0
+- org.apache.lucene:lucene-queries:9.7.0
+- org.apache.lucene:lucene-queryparser:9.7.0
+- org.apache.lucene:lucene-sandbox:9.7.0
+- org.apache.lucene:lucene-spatial3d:9.7.0
+- org.apache.lucene:lucene-spatial-extras:9.7.0
+- org.apache.lucene:lucene-suggest:9.7.0
+- org.opensearch.client:opensearch-rest-client:2.11.1
+- org.opensearch.client:opensearch-rest-high-level-client:2.11.1
+- org.opensearch:opensearch-cli:2.11.1
+- org.opensearch:opensearch-common:2.11.1
+- org.opensearch:opensearch-compress:2.11.1
+- org.opensearch:opensearch-core:2.11.1
+- org.opensearch:opensearch-geo:2.11.1
+- org.opensearch:opensearch:2.11.1
+- org.opensearch:opensearch-secure-sm:2.11.1
+- org.opensearch:opensearch-telemetry:2.11.1
+- org.opensearch:opensearch-x-content:2.11.1
+- org.opensearch.plugin:aggs-matrix-stats-client:2.11.1
+- org.opensearch.plugin:lang-mustache-client:2.11.1
+- org.opensearch.plugin:mapper-extras-client:2.11.1
+- org.opensearch.plugin:parent-join-client:2.11.1
+- org.opensearch.plugin:rank-eval-client:2.11.1
+
+This project bundles the following dependencies under the BSD license.
+See bundled license files for details.
+
+- com.github.luben:zstd-jni:1.5.5-5
diff --git a/flink-sql-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch2/PackagingITCase.java b/flink-sql-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch2/PackagingITCase.java
new file mode 100644
index 0000000..503652a
--- /dev/null
+++ b/flink-sql-connector-opensearch2/src/test/java/org/apache/flink/connector/opensearch2/PackagingITCase.java
@@ -0,0 +1,48 @@
+/*
+ * 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.opensearch2;
+
+import org.apache.flink.packaging.PackagingTestUtils;
+import org.apache.flink.table.factories.Factory;
+import org.apache.flink.test.resources.ResourceTestUtils;
+
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.Path;
+import java.util.Arrays;
+
+class PackagingITCase {
+
+ @Test
+ void testPackaging() throws Exception {
+ final Path jar =
+ ResourceTestUtils.getResource(".*/flink-sql-connector-opensearch2-[^/]*\\.jar");
+
+ PackagingTestUtils.assertJarContainsOnlyFilesMatching(
+ jar,
+ Arrays.asList(
+ "META-INF/",
+ "org/apache/flink/connector/base/",
+ "org/apache/flink/connector/opensearch/",
+ "org/apache/flink/opensearch/",
+ "org/apache/flink/opensearch2/shaded/",
+ "org/apache/flink/streaming/connectors/opensearch/"));
+ PackagingTestUtils.assertJarContainsServiceEntry(jar, Factory.class);
+ }
+}
diff --git a/pom.xml b/pom.xml
index 631b6b3..18d7aa4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -51,14 +51,14 @@
</scm>
<modules>
+ <module>flink-connector-opensearch-base</module>
<module>flink-connector-opensearch</module>
<module>flink-connector-opensearch-e2e-tests</module>
<module>flink-sql-connector-opensearch</module>
</modules>
<properties>
- <flink.version>1.17.1</flink.version>
-
+ <flink.version>1.18.0</flink.version>
<commons-compress.version>1.26.1</commons-compress.version>
<jackson-bom.version>2.15.3</jackson-bom.version>
<junit5.version>5.10.2</junit5.version>
@@ -372,9 +372,37 @@
<version>1.0.0</version>
<scope>test</scope>
</dependency>
+
+ <dependency>
+ <!-- mockito/powermock mismatch -->
+ <groupId>net.bytebuddy</groupId>
+ <artifactId>byte-buddy</artifactId>
+ <version>1.14.10</version>
+ </dependency>
+
+ <dependency>
+ <!-- mockito/powermock mismatch -->
+ <groupId>net.bytebuddy</groupId>
+ <artifactId>byte-buddy-agent</artifactId>
+ <version>1.14.10</version>
+ </dependency>
</dependencies>
</dependencyManagement>
+ <profiles>
+ <profile>
+ <id>java11</id>
+ <activation>
+ <jdk>[11,)</jdk>
+ </activation>
+ <modules>
+ <module>flink-connector-opensearch2</module>
+ <module>flink-connector-opensearch2-e2e-tests</module>
+ <module>flink-sql-connector-opensearch2</module>
+ </modules>
+ </profile>
+ </profiles>
+
<build>
<plugins>