diff --git a/flink-connectors/flink-connector-opensearch/pom.xml b/flink-connectors/flink-connector-opensearch/pom.xml
new file mode 100644
index 00000000000000..f889db69412ffe
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/pom.xml
@@ -0,0 +1,186 @@
+
+
+
+
+ 4.0.0
+
+
+ org.apache.flink
+ flink-connectors
+ 1.16-SNAPSHOT
+ ..
+
+
+ flink-connector-opensearch
+ Flink : Connectors : Opensearch
+
+ jar
+
+
+
+ 1.3.0
+
+
+
+
+
+
+
+ org.apache.flink
+ flink-connector-base
+ ${project.version}
+
+
+ org.apache.flink
+ flink-streaming-java
+ ${project.version}
+ provided
+
+
+
+
+
+
+ org.apache.flink
+ flink-table-api-java-bridge
+ ${project.version}
+ provided
+ true
+
+
+
+
+
+ org.opensearch
+ opensearch
+ ${opensearch.version}
+
+
+
+ org.opensearch.client
+ opensearch-rest-high-level-client
+ ${opensearch.version}
+
+
+ org.apache.httpcomponents
+ httpcore-nio
+
+
+
+
+
+
+ org.apache.httpcomponents
+ httpcore-nio
+ 4.4.12
+
+
+
+
+ org.apache.flink
+ flink-test-utils
+ ${project.version}
+ test
+
+
+
+ org.apache.flink
+ flink-test-utils
+ ${project.version}
+ test
+
+
+
+ org.apache.flink
+ flink-runtime
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.apache.flink
+ flink-streaming-java
+ ${project.version}
+ test
+ test-jar
+
+
+
+
+ org.apache.flink
+ flink-table-common
+ ${project.version}
+ test-jar
+ test
+
+
+
+
+ org.apache.flink
+ flink-json
+ ${project.version}
+ test
+
+
+
+
+ org.apache.flink
+ flink-table-planner_${scala.binary.version}
+ ${project.version}
+ test
+
+
+
+
+
+ org.apache.logging.log4j
+ log4j-api
+ provided
+
+
+
+ org.apache.logging.log4j
+ log4j-core
+ test
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+
+ 1
+
+
+
+
+
+
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java
new file mode 100644
index 00000000000000..ff284a64e3ded2
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorBuilderFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.TriFunction;
+
+import org.opensearch.action.bulk.BulkProcessor;
+import org.opensearch.client.RestHighLevelClient;
+
+import java.io.Serializable;
+
+@Internal
+interface BulkProcessorBuilderFactory
+ extends Serializable,
+ TriFunction<
+ RestHighLevelClient,
+ BulkProcessorConfig,
+ BulkProcessor.Listener,
+ BulkProcessor.Builder> {}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.java
new file mode 100644
index 00000000000000..07ed9a4537d7fe
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkProcessorConfig.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.sink;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class BulkProcessorConfig implements Serializable {
+
+ private final int bulkFlushMaxActions;
+ private final int bulkFlushMaxMb;
+ private final long bulkFlushInterval;
+ private final FlushBackoffType flushBackoffType;
+ private final int bulkFlushBackoffRetries;
+ private final long bulkFlushBackOffDelay;
+
+ BulkProcessorConfig(
+ int bulkFlushMaxActions,
+ int bulkFlushMaxMb,
+ long bulkFlushInterval,
+ FlushBackoffType flushBackoffType,
+ int bulkFlushBackoffRetries,
+ long bulkFlushBackOffDelay) {
+ this.bulkFlushMaxActions = bulkFlushMaxActions;
+ this.bulkFlushMaxMb = bulkFlushMaxMb;
+ this.bulkFlushInterval = bulkFlushInterval;
+ this.flushBackoffType = checkNotNull(flushBackoffType);
+ this.bulkFlushBackoffRetries = bulkFlushBackoffRetries;
+ this.bulkFlushBackOffDelay = bulkFlushBackOffDelay;
+ }
+
+ public int getBulkFlushMaxActions() {
+ return bulkFlushMaxActions;
+ }
+
+ public int getBulkFlushMaxMb() {
+ return bulkFlushMaxMb;
+ }
+
+ public long getBulkFlushInterval() {
+ return bulkFlushInterval;
+ }
+
+ public FlushBackoffType getFlushBackoffType() {
+ return flushBackoffType;
+ }
+
+ public int getBulkFlushBackoffRetries() {
+ return bulkFlushBackoffRetries;
+ }
+
+ public long getBulkFlushBackOffDelay() {
+ return bulkFlushBackOffDelay;
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java
new file mode 100644
index 00000000000000..0eb3f6872e2151
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/BulkRequestConsumerFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.Internal;
+
+import org.opensearch.action.ActionListener;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+
+import java.util.function.BiConsumer;
+
+/**
+ * {@link BulkRequestConsumerFactory} is used to bridge incompatible Opensearch Java API calls
+ * across different Opensearch versions.
+ */
+@Internal
+interface BulkRequestConsumerFactory
+ extends BiConsumer> {}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java
new file mode 100644
index 00000000000000..5b015ec5eaabf6
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/FlushBackoffType.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Used to control whether the sink should retry failed requests at all or with which kind back off
+ * strategy.
+ */
+@PublicEvolving
+public enum FlushBackoffType {
+ /** After every failure, it waits a configured time until the retries are exhausted. */
+ CONSTANT,
+ /**
+ * After every failure, it waits initially the configured time and increases the waiting time
+ * exponentially until the retries are exhausted.
+ */
+ EXPONENTIAL,
+ /** The failure is not retried. */
+ NONE,
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java
new file mode 100644
index 00000000000000..cb9993d54a887a
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/NetworkClientConfig.java
@@ -0,0 +1,86 @@
+/*
+ * 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 javax.annotation.Nullable;
+
+import java.io.Serializable;
+
+class NetworkClientConfig implements Serializable {
+
+ @Nullable private final String username;
+ @Nullable private final String password;
+ @Nullable private final String connectionPathPrefix;
+ @Nullable private final Integer connectionRequestTimeout;
+ @Nullable private final Integer connectionTimeout;
+ @Nullable private final Integer socketTimeout;
+ @Nullable private final Boolean allowInsecure;
+
+ NetworkClientConfig(
+ @Nullable String username,
+ @Nullable String password,
+ @Nullable String connectionPathPrefix,
+ @Nullable Integer connectionRequestTimeout,
+ @Nullable Integer connectionTimeout,
+ @Nullable Integer socketTimeout,
+ @Nullable Boolean allowInsecure) {
+ this.username = username;
+ this.password = password;
+ this.connectionPathPrefix = connectionPathPrefix;
+ this.connectionRequestTimeout = connectionRequestTimeout;
+ this.connectionTimeout = connectionTimeout;
+ this.socketTimeout = socketTimeout;
+ this.allowInsecure = allowInsecure;
+ }
+
+ @Nullable
+ public String getUsername() {
+ return username;
+ }
+
+ @Nullable
+ public String getPassword() {
+ return password;
+ }
+
+ @Nullable
+ public Integer getConnectionRequestTimeout() {
+ return connectionRequestTimeout;
+ }
+
+ @Nullable
+ public Integer getConnectionTimeout() {
+ return connectionTimeout;
+ }
+
+ @Nullable
+ public Integer getSocketTimeout() {
+ return socketTimeout;
+ }
+
+ @Nullable
+ public String getConnectionPathPrefix() {
+ return connectionPathPrefix;
+ }
+
+ @Nullable
+ public Boolean isAllowInsecure() {
+ return allowInsecure;
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java
new file mode 100644
index 00000000000000..cf2e0c461d45e4
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchEmitter.java
@@ -0,0 +1,77 @@
+/*
+ * 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.api.common.functions.Function;
+import org.apache.flink.api.connector.sink.SinkWriter;
+
+import org.opensearch.action.ActionRequest;
+
+/**
+ * Creates none or multiple {@link ActionRequest ActionRequests} from the incoming elements.
+ *
+ *
This is used by sinks to prepare elements for sending them to Opensearch.
+ *
+ *
+ *
+ * @param The type of the element handled by this {@link OpensearchEmitter}
+ */
+@PublicEvolving
+public interface OpensearchEmitter extends Function {
+
+ /**
+ * Initialization method for the function. It is called once before the actual working process
+ * methods.
+ */
+ default void open() throws Exception {}
+
+ /** Tear-down method for the function. It is called when the sink closes. */
+ default void close() throws Exception {}
+
+ /**
+ * Process the incoming element to produce multiple {@link ActionRequest ActionRequests}. The
+ * produced requests should be added to the provided {@link RequestIndexer}.
+ *
+ * @param element incoming element to process
+ * @param context to access additional information about the record
+ * @param indexer request indexer that {@code ActionRequest} should be added to
+ */
+ void emit(T element, SinkWriter.Context context, RequestIndexer indexer);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java
new file mode 100644
index 00000000000000..6f4d28296c9c8f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSink.java
@@ -0,0 +1,121 @@
+/*
+ * 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.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import org.apache.http.HttpHost;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+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.
+ *
+ *
+ *
{@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.
+ *
{@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.
+ *
+ *
+ * @param type of the records converted to Opensearch actions
+ * @see OpensearchSinkBuilder on how to construct a OpensearchSink
+ */
+@PublicEvolving
+public class OpensearchSink implements Sink {
+
+ private final List hosts;
+ private final OpensearchEmitter super IN> emitter;
+ private final BulkProcessorConfig buildBulkProcessorConfig;
+ private final BulkProcessorBuilderFactory bulkProcessorBuilderFactory;
+ private final NetworkClientConfig networkClientConfig;
+ private final DeliveryGuarantee deliveryGuarantee;
+
+ OpensearchSink(
+ List hosts,
+ OpensearchEmitter super IN> emitter,
+ DeliveryGuarantee deliveryGuarantee,
+ BulkProcessorBuilderFactory bulkProcessorBuilderFactory,
+ BulkProcessorConfig buildBulkProcessorConfig,
+ NetworkClientConfig networkClientConfig) {
+ this.hosts = checkNotNull(hosts);
+ this.bulkProcessorBuilderFactory = checkNotNull(bulkProcessorBuilderFactory);
+ checkArgument(!hosts.isEmpty(), "Hosts cannot be empty.");
+ this.emitter = checkNotNull(emitter);
+ this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+ this.buildBulkProcessorConfig = checkNotNull(buildBulkProcessorConfig);
+ this.networkClientConfig = checkNotNull(networkClientConfig);
+ }
+
+ @Override
+ public SinkWriter createWriter(InitContext context, List states)
+ throws IOException {
+ return new OpensearchWriter<>(
+ hosts,
+ emitter,
+ deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE,
+ buildBulkProcessorConfig,
+ bulkProcessorBuilderFactory,
+ networkClientConfig,
+ context.metricGroup(),
+ context.getMailboxExecutor());
+ }
+
+ @Override
+ public Optional> getWriterStateSerializer() {
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional> createCommitter() throws IOException {
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional> createGlobalCommitter() throws IOException {
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional> getCommittableSerializer() {
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional> getGlobalCommittableSerializer() {
+ return Optional.empty();
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java
new file mode 100644
index 00000000000000..04ffbf7fa96669
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilder.java
@@ -0,0 +1,449 @@
+/*
+ * 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.api.common.ExecutionConfig;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.ActionListener;
+import org.opensearch.action.bulk.BackoffPolicy;
+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.RestHighLevelClient;
+import org.opensearch.common.unit.ByteSizeUnit;
+import org.opensearch.common.unit.ByteSizeValue;
+import org.opensearch.common.unit.TimeValue;
+
+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 Opensearch compatible {@link OpensearchSink}.
+ *
+ *
The following example shows the minimal setup to create a OpensearchSink that submits actions
+ * on checkpoint or the default number of actions was buffered (1000).
+ *
+ *
+ *
+ * @param type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchSinkBuilder {
+
+ 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.NONE;
+ private List 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;
+
+ public OpensearchSinkBuilder() {}
+
+ @SuppressWarnings("unchecked")
+ protected > 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 OpensearchSinkBuilder setEmitter(
+ OpensearchEmitter super T> emitter) {
+ checkNotNull(emitter);
+ checkState(
+ InstantiationUtil.isSerializable(emitter),
+ "The Opensearch emitter must be serializable.");
+
+ final OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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}.
+ *
+ *
Sets the amount of delay between each backoff attempt when flushing bulk requests, in
+ * milliseconds.
+ *
+ *
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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder 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 OpensearchSinkBuilder setAllowInsecure(boolean allowInsecure) {
+ this.allowInsecure = allowInsecure;
+ return self();
+ }
+
+ protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() {
+ return new BulkProcessorBuilderFactory() {
+ @Override
+ public BulkProcessor.Builder apply(
+ RestHighLevelClient client,
+ BulkProcessorConfig bulkProcessorConfig,
+ BulkProcessor.Listener listener) {
+ 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
+ bulkResponseActionListener) {
+ client.bulkAsync(
+ bulkRequest,
+ RequestOptions.DEFAULT,
+ bulkResponseActionListener);
+ }
+ },
+ listener);
+
+ 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);
+ return builder;
+ }
+ };
+ }
+
+ /**
+ * Constructs the {@link OpensearchSink} with the properties configured this builder.
+ *
+ * @return {@link OpensearchSink}
+ */
+ public OpensearchSink build() {
+ checkNotNull(emitter);
+ checkNotNull(hosts);
+
+ NetworkClientConfig networkClientConfig = buildNetworkClientConfig();
+ BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig();
+
+ BulkProcessorBuilderFactory bulkProcessorBuilderFactory = getBulkProcessorBuilderFactory();
+ ClosureCleaner.clean(
+ bulkProcessorBuilderFactory, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+
+ return new OpensearchSink<>(
+ hosts,
+ emitter,
+ deliveryGuarantee,
+ bulkProcessorBuilderFactory,
+ bulkProcessorConfig,
+ networkClientConfig);
+ }
+
+ 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
+ + '\''
+ + '}';
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java
new file mode 100644
index 00000000000000..0935634163ee08
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchWriter.java
@@ -0,0 +1,350 @@
+/*
+ * 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.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.connector.sink.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.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.conn.ssl.TrustAllStrategy;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.ssl.SSLContexts;
+import org.opensearch.action.DocWriteRequest;
+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.RestClient;
+import org.opensearch.client.RestClientBuilder;
+import org.opensearch.client.RestHighLevelClient;
+import org.opensearch.rest.RestStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class OpensearchWriter implements SinkWriter {
+
+ private static final Logger LOG = LoggerFactory.getLogger(OpensearchWriter.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 bulkProcessorBuilderFactory configuring the {@link BulkProcessor}'s builder
+ * @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
+ */
+ OpensearchWriter(
+ List hosts,
+ OpensearchEmitter super IN> emitter,
+ boolean flushOnCheckpoint,
+ BulkProcessorConfig bulkProcessorConfig,
+ BulkProcessorBuilderFactory bulkProcessorBuilderFactory,
+ NetworkClientConfig networkClientConfig,
+ SinkWriterMetricGroup metricGroup,
+ MailboxExecutor mailboxExecutor) {
+ this.emitter = checkNotNull(emitter);
+ this.flushOnCheckpoint = flushOnCheckpoint;
+ this.mailboxExecutor = checkNotNull(mailboxExecutor);
+ this.client =
+ new RestHighLevelClient(
+ configureRestClientBuilder(
+ RestClient.builder(hosts.toArray(new HttpHost[0])),
+ networkClientConfig));
+ this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig);
+ this.requestIndexer = new DefaultRequestIndexer();
+ 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 List prepareCommit(boolean flush) throws IOException, InterruptedException {
+ checkpointInProgress = true;
+ while (pendingActions != 0 && (flushOnCheckpoint || flush)) {
+ bulkProcessor.flush();
+ LOG.info("Waiting for the response of {} pending actions.", pendingActions);
+ mailboxExecutor.yield();
+ }
+ checkpointInProgress = false;
+ return Collections.emptyList();
+ }
+
+ @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 static RestClientBuilder configureRestClientBuilder(
+ RestClientBuilder builder, NetworkClientConfig networkClientConfig) {
+ if (networkClientConfig.getConnectionPathPrefix() != null) {
+ builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix());
+ }
+
+ builder.setHttpClientConfigCallback(
+ httpClientBuilder -> {
+ if (networkClientConfig.getPassword() != null
+ && networkClientConfig.getUsername() != null) {
+ final CredentialsProvider credentialsProvider =
+ new BasicCredentialsProvider();
+ credentialsProvider.setCredentials(
+ AuthScope.ANY,
+ new UsernamePasswordCredentials(
+ networkClientConfig.getUsername(),
+ networkClientConfig.getPassword()));
+
+ httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider);
+ }
+
+ if (networkClientConfig.isAllowInsecure() != null
+ && networkClientConfig.isAllowInsecure()) {
+ try {
+ httpClientBuilder.setSSLContext(
+ SSLContexts.custom()
+ .loadTrustMaterial(new TrustAllStrategy())
+ .build());
+ } catch (final NoSuchAlgorithmException
+ | KeyStoreException
+ | KeyManagementException ex) {
+ throw new IllegalStateException(
+ "Unable to create custom SSL context", ex);
+ }
+ }
+
+ return httpClientBuilder;
+ });
+ if (networkClientConfig.getConnectionRequestTimeout() != null
+ || networkClientConfig.getConnectionTimeout() != null
+ || networkClientConfig.getSocketTimeout() != null) {
+ builder.setRequestConfigCallback(
+ requestConfigBuilder -> {
+ if (networkClientConfig.getConnectionRequestTimeout() != null) {
+ requestConfigBuilder.setConnectionRequestTimeout(
+ networkClientConfig.getConnectionRequestTimeout());
+ }
+ if (networkClientConfig.getConnectionTimeout() != null) {
+ requestConfigBuilder.setConnectTimeout(
+ networkClientConfig.getConnectionTimeout());
+ }
+ if (networkClientConfig.getSocketTimeout() != null) {
+ requestConfigBuilder.setSocketTimeout(
+ networkClientConfig.getSocketTimeout());
+ }
+ return requestConfigBuilder;
+ });
+ }
+ return builder;
+ }
+
+ private BulkProcessor createBulkProcessor(
+ BulkProcessorBuilderFactory bulkProcessorBuilderFactory,
+ BulkProcessorConfig bulkProcessorConfig) {
+
+ BulkProcessor.Builder builder =
+ bulkProcessorBuilderFactory.apply(client, bulkProcessorConfig, new BulkListener());
+
+ // This makes flush() blocking
+ builder.setConcurrentRequests(0);
+
+ return builder.build();
+ }
+
+ private class BulkListener implements BulkProcessor.Listener {
+
+ @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 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 void extractFailures(BulkRequest request, BulkResponse response) {
+ if (!response.hasFailures()) {
+ pendingActions -= request.numberOfActions();
+ 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;
+ }
+ throw new FlinkRuntimeException(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);
+ }
+ }
+
+ private boolean isClosed() {
+ if (closed) {
+ LOG.warn("Writer was closed before all records were acknowledged by Opensearch.");
+ }
+ return closed;
+ }
+
+ private class DefaultRequestIndexer implements RequestIndexer {
+
+ @Override
+ public void add(DeleteRequest... deleteRequests) {
+ for (final DeleteRequest deleteRequest : deleteRequests) {
+ pendingActions++;
+ bulkProcessor.add(deleteRequest);
+ }
+ }
+
+ @Override
+ public void add(IndexRequest... indexRequests) {
+ for (final IndexRequest indexRequest : indexRequests) {
+ pendingActions++;
+ bulkProcessor.add(indexRequest);
+ }
+ }
+
+ @Override
+ public void add(UpdateRequest... updateRequests) {
+ for (final UpdateRequest updateRequest : updateRequests) {
+ pendingActions++;
+ bulkProcessor.add(updateRequest);
+ }
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java
new file mode 100644
index 00000000000000..227a526e81b1fe
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/RequestIndexer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+
+/**
+ * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them
+ * for sending to an Opensearch cluster.
+ */
+@Internal
+public interface RequestIndexer {
+ /**
+ * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param deleteRequests The multiple {@link DeleteRequest} to add.
+ */
+ void add(DeleteRequest... deleteRequests);
+
+ /**
+ * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param indexRequests The multiple {@link IndexRequest} to add.
+ */
+ void add(IndexRequest... indexRequests);
+
+ /**
+ * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param updateRequests The multiple {@link UpdateRequest} to add.
+ */
+ void add(UpdateRequest... updateRequests);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.java
new file mode 100644
index 00000000000000..80a7a8c2a6537e
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/AbstractTimeIndexGenerator.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.connector.opensearch.table;
+
+import org.apache.flink.annotation.Internal;
+
+import java.time.format.DateTimeFormatter;
+
+/** Abstract class for time related {@link IndexGenerator}. */
+@Internal
+abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase {
+
+ private final String dateTimeFormat;
+ protected transient DateTimeFormatter dateTimeFormatter;
+
+ public AbstractTimeIndexGenerator(String index, String dateTimeFormat) {
+ super(index);
+ this.dateTimeFormat = dateTimeFormat;
+ }
+
+ @Override
+ public void open() {
+ this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat);
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java
new file mode 100644
index 00000000000000..cd821fc59a8953
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGenerator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.table.data.RowData;
+import org.apache.flink.types.Row;
+
+import java.io.Serializable;
+
+/** This interface is responsible to generate index name from given {@link Row} record. */
+@Internal
+interface IndexGenerator extends Serializable {
+
+ /**
+ * Initialize the index generator, this will be called only once before {@link
+ * #generate(RowData)} is called.
+ */
+ default void open() {}
+
+ /** Generate index name according to the given row. */
+ String generate(RowData row);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java
new file mode 100644
index 00000000000000..e5298c1a31324e
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorBase.java
@@ -0,0 +1,52 @@
+/*
+ * 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 java.util.Objects;
+
+/** Base class for {@link IndexGenerator}. */
+@Internal
+public abstract class IndexGeneratorBase implements IndexGenerator {
+
+ private static final long serialVersionUID = 1L;
+ protected final String index;
+
+ public IndexGeneratorBase(String index) {
+ this.index = index;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof IndexGeneratorBase)) {
+ return false;
+ }
+ IndexGeneratorBase that = (IndexGeneratorBase) o;
+ return index.equals(that.index);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(index);
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java
new file mode 100644
index 00000000000000..e87c41f77fdaea
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/IndexGeneratorFactory.java
@@ -0,0 +1,278 @@
+/*
+ * 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.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import javax.annotation.Nonnull;
+
+import java.io.Serializable;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Factory of {@link IndexGenerator}.
+ *
+ *
Flink supports both static index and dynamic index.
+ *
+ *
If you want to have a static index, this option value should be a plain string, e.g.
+ * 'myusers', all the records will be consistently written into "myusers" index.
+ *
+ *
If you want to have a dynamic index, you can use '{field_name}' to reference a field value in
+ * the record to dynamically generate a target index. You can also use
+ * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the
+ * format specified by date_format_string. The date_format_string is compatible with {@link
+ * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}',
+ * then a record with log_ts field value 2020-03-27 12:25:55 will be written into
+ * "myusers_2020-03-27" index.
+ */
+@Internal
+final class IndexGeneratorFactory {
+
+ private IndexGeneratorFactory() {}
+
+ public static IndexGenerator createIndexGenerator(
+ String index, List fieldNames, List dataTypes) {
+ final IndexHelper indexHelper = new IndexHelper();
+ if (indexHelper.checkIsDynamicIndex(index)) {
+ return createRuntimeIndexGenerator(
+ index,
+ fieldNames.toArray(new String[0]),
+ dataTypes.toArray(new DataType[0]),
+ indexHelper);
+ } else {
+ return new StaticIndexGenerator(index);
+ }
+ }
+
+ interface DynamicFormatter extends Serializable {
+ String format(@Nonnull Object fieldValue, DateTimeFormatter formatter);
+ }
+
+ private static IndexGenerator createRuntimeIndexGenerator(
+ String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) {
+ final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index);
+ final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr));
+ final String indexSuffix =
+ index.substring(indexPrefix.length() + dynamicIndexPatternStr.length());
+
+ final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index);
+ final int indexFieldPos =
+ indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat);
+ final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType();
+ final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot();
+
+ // validate index field type
+ indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot);
+
+ // time extract dynamic index pattern
+ final RowData.FieldGetter fieldGetter =
+ RowData.createFieldGetter(indexFieldType, indexFieldPos);
+
+ if (isDynamicIndexWithFormat) {
+ final String dateTimeFormat =
+ indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot);
+ DynamicFormatter formatFunction =
+ createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot);
+
+ return new AbstractTimeIndexGenerator(index, dateTimeFormat) {
+ @Override
+ public String generate(RowData row) {
+ Object fieldOrNull = fieldGetter.getFieldOrNull(row);
+ final String formattedField;
+ // TODO we can possibly optimize it to use the nullability of the field
+ if (fieldOrNull != null) {
+ formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter);
+ } else {
+ formattedField = "null";
+ }
+ return indexPrefix.concat(formattedField).concat(indexSuffix);
+ }
+ };
+ }
+ // general dynamic index pattern
+ return new IndexGeneratorBase(index) {
+ @Override
+ public String generate(RowData row) {
+ Object indexField = fieldGetter.getFieldOrNull(row);
+ return indexPrefix
+ .concat(indexField == null ? "null" : indexField.toString())
+ .concat(indexSuffix);
+ }
+ };
+ }
+
+ private static DynamicFormatter createFormatFunction(
+ LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) {
+ switch (indexFieldLogicalTypeRoot) {
+ case DATE:
+ return (value, dateTimeFormatter) -> {
+ Integer indexField = (Integer) value;
+ return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter);
+ };
+ case TIME_WITHOUT_TIME_ZONE:
+ return (value, dateTimeFormatter) -> {
+ Integer indexField = (Integer) value;
+ return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter);
+ };
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return (value, dateTimeFormatter) -> {
+ TimestampData indexField = (TimestampData) value;
+ return indexField.toLocalDateTime().format(dateTimeFormatter);
+ };
+ case TIMESTAMP_WITH_TIME_ZONE:
+ throw new UnsupportedOperationException(
+ "TIMESTAMP_WITH_TIME_ZONE is not supported yet");
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ return (value, dateTimeFormatter) -> {
+ TimestampData indexField = (TimestampData) value;
+ return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter);
+ };
+ default:
+ throw new TableException(
+ String.format(
+ "Unsupported type '%s' found in Opensearch dynamic index field, "
+ + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.",
+ indexFieldType));
+ }
+ }
+
+ /**
+ * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field
+ * type ans parse index format from pattern.
+ */
+ private static class IndexHelper {
+ private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?");
+ private static final Pattern dynamicIndexTimeExtractPattern =
+ Pattern.compile(".*\\{.+\\|.*\\}.*");
+ private static final List supportedTypes = new ArrayList<>();
+ private static final Map defaultFormats = new HashMap<>();
+
+ static {
+ // time related types
+ supportedTypes.add(LogicalTypeRoot.DATE);
+ supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE);
+ supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+ supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE);
+ supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+ // general types
+ supportedTypes.add(LogicalTypeRoot.VARCHAR);
+ supportedTypes.add(LogicalTypeRoot.CHAR);
+ supportedTypes.add(LogicalTypeRoot.TINYINT);
+ supportedTypes.add(LogicalTypeRoot.INTEGER);
+ supportedTypes.add(LogicalTypeRoot.BIGINT);
+ }
+
+ static {
+ defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd");
+ defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss");
+ defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss");
+ defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss");
+ defaultFormats.put(
+ LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX");
+ }
+
+ /** Validate the index field Type. */
+ void validateIndexFieldType(LogicalTypeRoot logicalType) {
+ if (!supportedTypes.contains(logicalType)) {
+ throw new IllegalArgumentException(
+ String.format(
+ "Unsupported type %s of index field, " + "Supported types are: %s",
+ logicalType, supportedTypes));
+ }
+ }
+
+ /** Get the default date format. */
+ String getDefaultFormat(LogicalTypeRoot logicalType) {
+ return defaultFormats.get(logicalType);
+ }
+
+ /** Check general dynamic index is enabled or not by index pattern. */
+ boolean checkIsDynamicIndex(String index) {
+ final Matcher matcher = dynamicIndexPattern.matcher(index);
+ int count = 0;
+ while (matcher.find()) {
+ count++;
+ }
+ if (count > 1) {
+ throw new TableException(
+ String.format(
+ "Chaining dynamic index pattern %s is not supported,"
+ + " only support single dynamic index pattern.",
+ index));
+ }
+ return count == 1;
+ }
+
+ /** Check time extract dynamic index is enabled or not by index pattern. */
+ boolean checkIsDynamicIndexWithFormat(String index) {
+ return dynamicIndexTimeExtractPattern.matcher(index).matches();
+ }
+
+ /** Extract dynamic index pattern string from index pattern string. */
+ String extractDynamicIndexPatternStr(String index) {
+ int start = index.indexOf("{");
+ int end = index.lastIndexOf("}");
+ return index.substring(start, end + 1);
+ }
+
+ /** Extract index field position in a fieldNames, return the field position. */
+ int extractIndexFieldPos(
+ String index, String[] fieldNames, boolean isDynamicIndexWithFormat) {
+ List fieldList = Arrays.asList(fieldNames);
+ String indexFieldName;
+ if (isDynamicIndexWithFormat) {
+ indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|"));
+ } else {
+ indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}"));
+ }
+ if (!fieldList.contains(indexFieldName)) {
+ throw new TableException(
+ String.format(
+ "Unknown field '%s' in index pattern '%s', please check the field name.",
+ indexFieldName, index));
+ }
+ return fieldList.indexOf(indexFieldName);
+ }
+
+ /** Extract dateTime format by the date format that extracted from index pattern string. */
+ private String extractDateFormat(String index, LogicalTypeRoot logicalType) {
+ String format = index.substring(index.indexOf("|") + 1, index.indexOf("}"));
+ if ("".equals(format)) {
+ format = getDefaultFormat(logicalType);
+ }
+ return format;
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java
new file mode 100644
index 00000000000000..20e1013f53ff52
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/KeyExtractor.java
@@ -0,0 +1,97 @@
+/*
+ * 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.table.data.RowData;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.Period;
+import java.util.List;
+
+/** An extractor for a Opensearch key from a {@link RowData}. */
+@Internal
+class KeyExtractor implements SerializableFunction {
+ private final FieldFormatter[] fieldFormatters;
+ private final String keyDelimiter;
+
+ private interface FieldFormatter extends Serializable {
+ String format(RowData rowData);
+ }
+
+ private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) {
+ this.fieldFormatters = fieldFormatters;
+ this.keyDelimiter = keyDelimiter;
+ }
+
+ @Override
+ public String apply(RowData rowData) {
+ final StringBuilder builder = new StringBuilder();
+ for (int i = 0; i < fieldFormatters.length; i++) {
+ if (i > 0) {
+ builder.append(keyDelimiter);
+ }
+ final String value = fieldFormatters[i].format(rowData);
+ builder.append(value);
+ }
+ return builder.toString();
+ }
+
+ public static SerializableFunction createKeyExtractor(
+ List primaryKeyTypesWithIndex, String keyDelimiter) {
+ if (!primaryKeyTypesWithIndex.isEmpty()) {
+ FieldFormatter[] formatters =
+ primaryKeyTypesWithIndex.stream()
+ .map(
+ logicalTypeWithIndex ->
+ toFormatter(
+ logicalTypeWithIndex.index,
+ logicalTypeWithIndex.logicalType))
+ .toArray(FieldFormatter[]::new);
+ return new KeyExtractor(formatters, keyDelimiter);
+ } else {
+ return (row) -> null;
+ }
+ }
+
+ private static FieldFormatter toFormatter(int index, LogicalType type) {
+ switch (type.getTypeRoot()) {
+ case DATE:
+ return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString();
+ case TIME_WITHOUT_TIME_ZONE:
+ return (row) ->
+ LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString();
+ case INTERVAL_YEAR_MONTH:
+ return (row) -> Period.ofDays(row.getInt(index)).toString();
+ case INTERVAL_DAY_TIME:
+ return (row) -> Duration.ofMillis(row.getLong(index)).toString();
+ case DISTINCT_TYPE:
+ return toFormatter(index, ((DistinctType) type).getSourceType());
+ default:
+ RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index);
+ return (row) -> fieldGetter.getFieldOrNull(row).toString();
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java
new file mode 100644
index 00000000000000..e5fe9a6ebbc4d2
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/LogicalTypeWithIndex.java
@@ -0,0 +1,31 @@
+/*
+ * 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.types.logical.LogicalType;
+
+class LogicalTypeWithIndex {
+ public final int index;
+ public final LogicalType logicalType;
+
+ LogicalTypeWithIndex(int index, LogicalType logicalType) {
+ this.index = index;
+ this.logicalType = logicalType;
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java
new file mode 100644
index 00000000000000..a6f6dc60d8051e
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConfiguration.java
@@ -0,0 +1,163 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.opensearch.sink.FlushBackoffType;
+import org.apache.flink.table.api.ValidationException;
+
+import org.apache.http.HttpHost;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+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.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;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Opensearch base configuration. */
+@Internal
+class OpensearchConfiguration {
+ protected final ReadableConfig config;
+
+ OpensearchConfiguration(ReadableConfig config) {
+ this.config = checkNotNull(config);
+ }
+
+ public int getBulkFlushMaxActions() {
+ return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION);
+ }
+
+ public MemorySize getBulkFlushMaxByteSize() {
+ return config.get(BULK_FLUSH_MAX_SIZE_OPTION);
+ }
+
+ public long getBulkFlushInterval() {
+ return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis();
+ }
+
+ public DeliveryGuarantee getDeliveryGuarantee() {
+ return config.get(DELIVERY_GUARANTEE_OPTION);
+ }
+
+ public Optional getUsername() {
+ return config.getOptional(USERNAME_OPTION);
+ }
+
+ public Optional getPassword() {
+ return config.getOptional(PASSWORD_OPTION);
+ }
+
+ public Optional getBulkFlushBackoffType() {
+ return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION);
+ }
+
+ public Optional getBulkFlushBackoffRetries() {
+ return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION);
+ }
+
+ public Optional getBulkFlushBackoffDelay() {
+ return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis);
+ }
+
+ public String getIndex() {
+ return config.get(INDEX_OPTION);
+ }
+
+ public String getKeyDelimiter() {
+ return config.get(KEY_DELIMITER_OPTION);
+ }
+
+ public Optional getPathPrefix() {
+ return config.getOptional(CONNECTION_PATH_PREFIX_OPTION);
+ }
+
+ public Optional getConnectionRequestTimeout() {
+ return config.getOptional(CONNECTION_REQUEST_TIMEOUT);
+ }
+
+ public Optional getConnectionTimeout() {
+ return config.getOptional(CONNECTION_TIMEOUT);
+ }
+
+ public Optional getSocketTimeout() {
+ return config.getOptional(SOCKET_TIMEOUT);
+ }
+
+ public List getHosts() {
+ return config.get(HOSTS_OPTION).stream()
+ .map(OpensearchConfiguration::validateAndParseHostsString)
+ .collect(Collectors.toList());
+ }
+
+ public Optional getParallelism() {
+ return config.getOptional(SINK_PARALLELISM);
+ }
+
+ public Optional isAllowInsecure() {
+ return config.getOptional(ALLOW_INSECURE);
+ }
+
+ private static HttpHost validateAndParseHostsString(String host) {
+ try {
+ HttpHost httpHost = HttpHost.create(host);
+ if (httpHost.getPort() < 0) {
+ throw new ValidationException(
+ String.format(
+ "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.",
+ host, HOSTS_OPTION.key()));
+ }
+
+ if (httpHost.getSchemeName() == null) {
+ throw new ValidationException(
+ String.format(
+ "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.",
+ host, HOSTS_OPTION.key()));
+ }
+ return httpHost;
+ } catch (Exception e) {
+ throw new ValidationException(
+ String.format(
+ "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.",
+ host, HOSTS_OPTION.key()),
+ e);
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.java
new file mode 100644
index 00000000000000..dcd87cb7d232a6
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchConnectorOptions.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.connector.opensearch.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.opensearch.sink.FlushBackoffType;
+
+import java.time.Duration;
+import java.util.List;
+
+/**
+ * Base options for the Opensearch connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class OpensearchConnectorOptions {
+
+ OpensearchConnectorOptions() {}
+
+ public static final ConfigOption> HOSTS_OPTION =
+ ConfigOptions.key("hosts")
+ .stringType()
+ .asList()
+ .noDefaultValue()
+ .withDescription("Opensearch hosts to connect to.");
+
+ public static final ConfigOption INDEX_OPTION =
+ ConfigOptions.key("index")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Opensearch index for every record.");
+
+ public static final ConfigOption PASSWORD_OPTION =
+ ConfigOptions.key("password")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Password used to connect to Opensearch instance.");
+
+ public static final ConfigOption USERNAME_OPTION =
+ ConfigOptions.key("username")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Username used to connect to Opensearch instance.");
+
+ public static final ConfigOption KEY_DELIMITER_OPTION =
+ ConfigOptions.key("document-id.key-delimiter")
+ .stringType()
+ .defaultValue("_")
+ .withDescription(
+ "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\".");
+
+ public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION =
+ ConfigOptions.key("sink.bulk-flush.max-actions")
+ .intType()
+ .defaultValue(1000)
+ .withDescription("Maximum number of actions to buffer for each bulk request.");
+
+ public static final ConfigOption BULK_FLUSH_MAX_SIZE_OPTION =
+ ConfigOptions.key("sink.bulk-flush.max-size")
+ .memoryType()
+ .defaultValue(MemorySize.parse("2mb"))
+ .withDescription("Maximum size of buffered actions per bulk request");
+
+ public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION =
+ ConfigOptions.key("sink.bulk-flush.interval")
+ .durationType()
+ .defaultValue(Duration.ofSeconds(1))
+ .withDescription("Bulk flush interval");
+
+ public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION =
+ ConfigOptions.key("sink.bulk-flush.backoff.strategy")
+ .enumType(FlushBackoffType.class)
+ .noDefaultValue()
+ .withDescription("Backoff strategy");
+
+ public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION =
+ ConfigOptions.key("sink.bulk-flush.backoff.max-retries")
+ .intType()
+ .noDefaultValue()
+ .withDescription("Maximum number of retries.");
+
+ public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION =
+ ConfigOptions.key("sink.bulk-flush.backoff.delay")
+ .durationType()
+ .noDefaultValue()
+ .withDescription("Delay between each backoff attempt.");
+
+ public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION =
+ ConfigOptions.key("connection.path-prefix")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Prefix string to be added to every REST communication.");
+
+ public static final ConfigOption CONNECTION_REQUEST_TIMEOUT =
+ ConfigOptions.key("connection.request-timeout")
+ .durationType()
+ .noDefaultValue()
+ .withDescription(
+ "The timeout for requesting a connection from the connection manager.");
+
+ public static final ConfigOption CONNECTION_TIMEOUT =
+ ConfigOptions.key("connection.timeout")
+ .durationType()
+ .noDefaultValue()
+ .withDescription("The timeout for establishing a connection.");
+
+ public static final ConfigOption SOCKET_TIMEOUT =
+ ConfigOptions.key("socket.timeout")
+ .durationType()
+ .noDefaultValue()
+ .withDescription(
+ "The socket timeout (SO_TIMEOUT) for waiting for data or, put differently,"
+ + "a maximum period inactivity between two consecutive data packets.");
+
+ public static final ConfigOption FORMAT_OPTION =
+ ConfigOptions.key("format")
+ .stringType()
+ .defaultValue("json")
+ .withDescription(
+ "The format must produce a valid JSON document. "
+ + "Please refer to the documentation on formats for more details.");
+
+ public static final ConfigOption DELIVERY_GUARANTEE_OPTION =
+ ConfigOptions.key("sink.delivery-guarantee")
+ .enumType(DeliveryGuarantee.class)
+ .defaultValue(DeliveryGuarantee.NONE)
+ .withDescription("Optional delivery guarantee when committing.");
+
+ public static final ConfigOption ALLOW_INSECURE =
+ ConfigOptions.key("allow-insecure")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Allow insecure connections to HTTPS endpoints (disable certificates validation)");
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java
new file mode 100644
index 00000000000000..cc60b167521f6f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSink.java
@@ -0,0 +1,201 @@
+/*
+ * 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.OpensearchSink;
+import org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder;
+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.SinkProvider;
+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.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 OpensearchSink} from a logical
+ * description.
+ */
+@Internal
+class OpensearchDynamicSink implements DynamicTableSink {
+
+ final EncodingFormat> format;
+ final DataType physicalRowDataType;
+ final List primaryKeyLogicalTypesWithIndex;
+ final OpensearchConfiguration config;
+
+ final String summaryString;
+ final OpensearchSinkBuilderSupplier builderSupplier;
+
+ OpensearchDynamicSink(
+ EncodingFormat> format,
+ OpensearchConfiguration config,
+ List primaryKeyLogicalTypesWithIndex,
+ DataType physicalRowDataType,
+ String summaryString,
+ OpensearchSinkBuilderSupplier builderSupplier) {
+ this.format = checkNotNull(format);
+ this.physicalRowDataType = checkNotNull(physicalRowDataType);
+ this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex);
+ this.config = checkNotNull(config);
+ this.summaryString = checkNotNull(summaryString);
+ this.builderSupplier = checkNotNull(builderSupplier);
+ }
+
+ Function createKeyExtractor() {
+ return KeyExtractor.createKeyExtractor(
+ primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter());
+ }
+
+ IndexGenerator createIndexGenerator() {
+ return IndexGeneratorFactory.createIndexGenerator(
+ config.getIndex(),
+ DataType.getFieldNames(physicalRowDataType),
+ DataType.getFieldDataTypes(physicalRowDataType));
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+ ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+ for (RowKind kind : requestedMode.getContainedKinds()) {
+ if (kind != RowKind.UPDATE_BEFORE) {
+ builder.addContainedKind(kind);
+ }
+ }
+ return builder.build();
+ }
+
+ @Override
+ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+ SerializationSchema format =
+ this.format.createRuntimeEncoder(context, physicalRowDataType);
+
+ final RowOpensearchEmitter rowOpensearchEmitter =
+ new RowOpensearchEmitter(
+ createIndexGenerator(), format, XContentType.JSON, createKeyExtractor());
+
+ OpensearchSinkBuilder builder = builderSupplier.get();
+ 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 SinkProvider.of(builder.build(), config.getParallelism().orElse(null));
+ }
+
+ @Override
+ public DynamicTableSink copy() {
+ return new OpensearchDynamicSink(
+ format,
+ config,
+ primaryKeyLogicalTypesWithIndex,
+ physicalRowDataType,
+ summaryString,
+ builderSupplier);
+ }
+
+ @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;
+ }
+ OpensearchDynamicSink that = (OpensearchDynamicSink) 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)
+ && Objects.equals(builderSupplier, that.builderSupplier);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ format,
+ physicalRowDataType,
+ primaryKeyLogicalTypesWithIndex,
+ config,
+ summaryString,
+ builderSupplier);
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java
new file mode 100644
index 00000000000000..d02dce9dfbf77c
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchDynamicSinkFactory.java
@@ -0,0 +1,222 @@
+/*
+ * 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.Configuration;
+import org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder;
+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.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.DynamicTableFactory;
+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.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.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;
+import static org.opensearch.common.Strings.capitalize;
+
+/** A {@link DynamicTableSinkFactory} for discovering OpensearchDynamicSink. */
+@Internal
+public class OpensearchDynamicSinkFactory implements DynamicTableSinkFactory {
+ private static final String FACTORY_IDENTIFIER = "opensearch";
+
+ private final OpensearchSinkBuilderSupplier sinkBuilderSupplier;
+
+ public OpensearchDynamicSinkFactory() {
+ this.sinkBuilderSupplier = OpensearchSinkBuilder::new;
+ }
+
+ @Override
+ public DynamicTableSink createDynamicTableSink(Context context) {
+ List primaryKeyLogicalTypesWithIndex =
+ getPrimaryKeyLogicalTypesWithIndex(context);
+ EncodingFormat> format =
+ getValidatedEncodingFormat(this, context);
+
+ OpensearchConfiguration config = getConfiguration(context);
+ validateConfiguration(config);
+
+ return new OpensearchDynamicSink(
+ format,
+ config,
+ primaryKeyLogicalTypesWithIndex,
+ context.getPhysicalRowDataType(),
+ capitalize(FACTORY_IDENTIFIER),
+ sinkBuilderSupplier);
+ }
+
+ OpensearchConfiguration getConfiguration(Context context) {
+ return new OpensearchConfiguration(
+ Configuration.fromMap(context.getCatalogTable().getOptions()));
+ }
+
+ 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("")));
+ }
+ }
+
+ static void validate(boolean condition, Supplier message) {
+ if (!condition) {
+ throw new ValidationException(message.get());
+ }
+ }
+
+ EncodingFormat> getValidatedEncodingFormat(
+ DynamicTableFactory factory, DynamicTableFactory.Context context) {
+ final FactoryUtil.TableFactoryHelper helper =
+ FactoryUtil.createTableFactoryHelper(factory, context);
+ final EncodingFormat> format =
+ helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION);
+ helper.validate();
+ return format;
+ }
+
+ List 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 = 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> requiredOptions() {
+ return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set> 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-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.java
new file mode 100644
index 00000000000000..80fc30ff28206f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchSinkBuilderSupplier.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.OpensearchSinkBuilder;
+
+import java.util.function.Supplier;
+
+interface OpensearchSinkBuilderSupplier extends Supplier> {}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java
new file mode 100644
index 00000000000000..e9ea7e286f2804
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/OpensearchValidationUtils.java
@@ -0,0 +1,92 @@
+/*
+ * 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.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating Opensearch properties. */
+@Internal
+class OpensearchValidationUtils {
+ private static final Set ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+ static {
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DATE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+ ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+ }
+
+ /**
+ * Checks that the table does not have a primary key defined on illegal types. In Opensearch the
+ * primary key is used to calculate the Opensearch document id, which is a string of up to 512
+ * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.
+ * Certain types do not have a good string representation to be used in this scenario. The
+ * illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link
+ * LogicalTypeRoot#RAW} type.
+ */
+ public static void validatePrimaryKey(DataType primaryKeyDataType) {
+ List fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType);
+ List illegalTypes =
+ fieldDataTypes.stream()
+ .map(DataType::getLogicalType)
+ .map(
+ logicalType -> {
+ if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) {
+ return ((DistinctType) logicalType)
+ .getSourceType()
+ .getTypeRoot();
+ } else {
+ return logicalType.getTypeRoot();
+ }
+ })
+ .filter(t -> !ALLOWED_PRIMARY_KEY_TYPES.contains(t))
+ .collect(Collectors.toList());
+ if (!illegalTypes.isEmpty()) {
+ throw new ValidationException(
+ String.format(
+ "The table has a primary key on columns of illegal types: %s.",
+ illegalTypes));
+ }
+ }
+
+ private OpensearchValidationUtils() {}
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java
new file mode 100644
index 00000000000000..f6d14286ee4beb
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/RowOpensearchEmitter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.sink.SinkWriter;
+import org.apache.flink.connector.opensearch.sink.OpensearchEmitter;
+import org.apache.flink.connector.opensearch.sink.RequestIndexer;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+
+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 RowOpensearchEmitter implements OpensearchEmitter {
+
+ private final IndexGenerator indexGenerator;
+ private final SerializationSchema serializationSchema;
+ private final XContentType contentType;
+ private final Function createKey;
+
+ public RowOpensearchEmitter(
+ IndexGenerator indexGenerator,
+ SerializationSchema serializationSchema,
+ XContentType contentType,
+ Function createKey) {
+ this.indexGenerator = checkNotNull(indexGenerator);
+ this.serializationSchema = checkNotNull(serializationSchema);
+ this.contentType = checkNotNull(contentType);
+ this.createKey = checkNotNull(createKey);
+ }
+
+ @Override
+ public void open() {
+ 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-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java
new file mode 100644
index 00000000000000..768dd84cd9368f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/table/StaticIndexGenerator.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.RowData;
+
+/** A static {@link IndexGenerator} which generate fixed index name. */
+@Internal
+final class StaticIndexGenerator extends IndexGeneratorBase {
+
+ public StaticIndexGenerator(String index) {
+ super(index);
+ }
+
+ public String generate(RowData row) {
+ return index;
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java
new file mode 100644
index 00000000000000..2473bf94f2bfa0
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/ActionRequestFailureHandler.java
@@ -0,0 +1,77 @@
+/*
+ * 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.opensearch.action.ActionRequest;
+
+import java.io.Serializable;
+
+/**
+ * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how
+ * failed {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing
+ * malformed documents, or simply requesting them to be sent to Opensearch again if the failure is
+ * only temporary.
+ *
+ *
Example:
+ *
+ *
{@code
+ * private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
+ *
+ * @Override
+ * void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
+ * if (ExceptionUtils.findThrowable(failure, OpenSearchRejectedExecutionException.class).isPresent()) {
+ * // full queue; re-add document for indexing
+ * indexer.add(action);
+ * } else if (ExceptionUtils.findThrowable(failure, OpensearchParseException.class).isPresent()) {
+ * // malformed document; simply drop request without failing sink
+ * } else {
+ * // for all other failures, fail the sink;
+ * // here the failure is simply rethrown, but users can also choose to throw custom exceptions
+ * throw failure;
+ * }
+ * }
+ * }
+ *
+ * }
+ *
+ *
The above example will let the sink re-add requests that failed due to queue capacity
+ * saturation and drop requests with malformed documents, without failing the sink. For all other
+ * failures, the sink will fail.
+ *
+ * @deprecated This has been deprecated and will be removed in the future.
+ */
+@Deprecated
+@PublicEvolving
+public interface ActionRequestFailureHandler extends Serializable {
+
+ /**
+ * Handle a failed {@link ActionRequest}.
+ *
+ * @param action the {@link ActionRequest} that failed due to the failure
+ * @param failure the cause of failure
+ * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved)
+ * @param indexer request indexer to re-add the failed action, if intended to do so
+ * @throws Throwable if the sink should fail on this failure, the implementation should rethrow
+ * the exception or a custom one
+ */
+ void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer)
+ throws Throwable;
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.java
new file mode 100644
index 00000000000000..c0d93c1bd7d4c2
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/BufferingNoOpRequestIndexer.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.streaming.connectors.opensearch;
+
+import org.apache.flink.annotation.Internal;
+
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.Collections;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests}
+ * before re-sending them to the Opensearch cluster upon request.
+ */
+@Internal
+@NotThreadSafe
+class BufferingNoOpRequestIndexer implements RequestIndexer {
+
+ private ConcurrentLinkedQueue bufferedRequests;
+
+ BufferingNoOpRequestIndexer() {
+ this.bufferedRequests = new ConcurrentLinkedQueue();
+ }
+
+ @Override
+ public void add(DeleteRequest... deleteRequests) {
+ Collections.addAll(bufferedRequests, deleteRequests);
+ }
+
+ @Override
+ public void add(IndexRequest... indexRequests) {
+ Collections.addAll(bufferedRequests, indexRequests);
+ }
+
+ @Override
+ public void add(UpdateRequest... updateRequests) {
+ Collections.addAll(bufferedRequests, updateRequests);
+ }
+
+ void processBufferedRequests(RequestIndexer actualIndexer) {
+ for (ActionRequest request : bufferedRequests) {
+ if (request instanceof IndexRequest) {
+ actualIndexer.add((IndexRequest) request);
+ } else if (request instanceof DeleteRequest) {
+ actualIndexer.add((DeleteRequest) request);
+ } else if (request instanceof UpdateRequest) {
+ actualIndexer.add((UpdateRequest) request);
+ }
+ }
+
+ bufferedRequests.clear();
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java
new file mode 100644
index 00000000000000..f60eb5967d4506
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchBulkProcessorIndexer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.opensearch;
+
+import org.apache.flink.annotation.Internal;
+
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.bulk.BulkProcessor;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest
+ * ActionRequests} will be buffered before sending a bulk request to the Opensearch cluster.
+ *
+ *
Note: This class is binary compatible to Opensearch 1.x.
+ */
+@Internal
+class OpensearchBulkProcessorIndexer implements RequestIndexer {
+
+ private final BulkProcessor bulkProcessor;
+ private final boolean flushOnCheckpoint;
+ private final AtomicLong numPendingRequestsRef;
+
+ OpensearchBulkProcessorIndexer(
+ BulkProcessor bulkProcessor,
+ boolean flushOnCheckpoint,
+ AtomicLong numPendingRequestsRef) {
+ this.bulkProcessor = checkNotNull(bulkProcessor);
+ this.flushOnCheckpoint = flushOnCheckpoint;
+ this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef);
+ }
+
+ @Override
+ public void add(DeleteRequest... deleteRequests) {
+ for (DeleteRequest deleteRequest : deleteRequests) {
+ if (flushOnCheckpoint) {
+ numPendingRequestsRef.getAndIncrement();
+ }
+ this.bulkProcessor.add(deleteRequest);
+ }
+ }
+
+ @Override
+ public void add(IndexRequest... indexRequests) {
+ for (IndexRequest indexRequest : indexRequests) {
+ if (flushOnCheckpoint) {
+ numPendingRequestsRef.getAndIncrement();
+ }
+ this.bulkProcessor.add(indexRequest);
+ }
+ }
+
+ @Override
+ public void add(UpdateRequest... updateRequests) {
+ for (UpdateRequest updateRequest : updateRequests) {
+ if (flushOnCheckpoint) {
+ numPendingRequestsRef.getAndIncrement();
+ }
+ this.bulkProcessor.add(updateRequest);
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java
new file mode 100644
index 00000000000000..aae3623b6cae6f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSink.java
@@ -0,0 +1,807 @@
+/*
+ * 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.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+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.ByteSizeUnit;
+import org.opensearch.common.unit.ByteSizeValue;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.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.
+ *
+ *
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 Type of the elements handled by this sink
+ */
+@Internal
+public class OpensearchSink extends RichSinkFunction implements CheckpointedFunction {
+ private static final long serialVersionUID = -1007596293618451942L;
+ private static final Logger LOG = LoggerFactory.getLogger(OpensearchSink.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.
+ *
+ *
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.
+ *
+ *
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 userConfig;
+
+ /**
+ * The function that is used to construct multiple {@link ActionRequest ActionRequests} from
+ * each incoming element.
+ */
+ private final OpensearchSinkFunction 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 OpensearchSinkBase#flushOnCheckpoint} is {@code true}.
+ *
+ *
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 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.
+ *
+ *
Errors will be checked and rethrown before processing each input element, and when the
+ * sink is closed.
+ */
+ private final AtomicReference failureThrowable = new AtomicReference<>();
+
+ private OpensearchSink(
+ Map userConfig,
+ List httpHosts,
+ OpensearchSinkFunction 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
+ // "OpensearchSinkBase 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.
+ *
+ *
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(userConfig);
+ 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}. */
+ protected 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(Map clientConfig) 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.
+ *
+ *
Called by {@link OpensearchSinkBase#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 OpensearchSink.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 OpensearchSink}.
+ *
+ * @param Type of the elements handled by the sink this builder creates.
+ * @deprecated This has been deprecated, please use {@link
+ * org.apache.flink.connector.opensearch.sink.OpensearchSinkBuilder}.
+ */
+ @Deprecated
+ @PublicEvolving
+ public static class Builder {
+
+ private final List httpHosts;
+ private final OpensearchSinkFunction opensearchSinkFunction;
+
+ private Map 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 httpHosts, OpensearchSinkFunction 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 OpensearchSink build() {
+ return new OpensearchSink<>(
+ 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-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java
new file mode 100644
index 00000000000000..b3c3e941756107
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/OpensearchSinkFunction.java
@@ -0,0 +1,82 @@
+/*
+ * 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.common.functions.Function;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import org.opensearch.action.ActionRequest;
+
+import java.io.Serializable;
+
+/**
+ * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream.
+ *
+ *
This is used by sinks to prepare elements for sending them to Opensearch.
+ *
+ *
+ *
+ * @param The type of the element handled by this {@code OpensearchSinkFunction}
+ * @deprecated This has been deprecated and will be removed in the future.
+ */
+@Deprecated
+@PublicEvolving
+public interface OpensearchSinkFunction extends Serializable, Function {
+
+ /**
+ * Initialization method for the function. It is called once before the actual working process
+ * methods.
+ */
+ default void open() throws Exception {}
+
+ /** Tear-down method for the function. It is called when the sink closes. */
+ default void close() throws Exception {}
+
+ /**
+ * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. The
+ * produced requests should be added to the provided {@link RequestIndexer}.
+ *
+ * @param element incoming element to process
+ * @param ctx runtime context containing information about the sink instance
+ * @param indexer request indexer that {@code ActionRequest} should be added to
+ */
+ void process(T element, RuntimeContext ctx, RequestIndexer indexer);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java
new file mode 100644
index 00000000000000..78fefed14f29d6
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RequestIndexer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.opensearch;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.opensearch.action.ActionRequest;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+
+/**
+ * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them
+ * for sending to an Opensearch cluster.
+ *
+ * @deprecated This has been deprecated and will be removed in the future.
+ */
+@Deprecated
+@PublicEvolving
+public interface RequestIndexer {
+
+ /**
+ * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param actionRequests The multiple {@link ActionRequest} to add.
+ * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest}
+ */
+ @Deprecated
+ default void add(ActionRequest... actionRequests) {
+ for (ActionRequest actionRequest : actionRequests) {
+ if (actionRequest instanceof IndexRequest) {
+ add((IndexRequest) actionRequest);
+ } else if (actionRequest instanceof DeleteRequest) {
+ add((DeleteRequest) actionRequest);
+ } else if (actionRequest instanceof UpdateRequest) {
+ add((UpdateRequest) actionRequest);
+ } else {
+ throw new IllegalArgumentException(
+ "RequestIndexer only supports Index, Delete and Update requests");
+ }
+ }
+ }
+
+ /**
+ * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param deleteRequests The multiple {@link DeleteRequest} to add.
+ */
+ void add(DeleteRequest... deleteRequests);
+
+ /**
+ * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param indexRequests The multiple {@link IndexRequest} to add.
+ */
+ void add(IndexRequest... indexRequests);
+
+ /**
+ * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to
+ * Opensearch.
+ *
+ * @param updateRequests The multiple {@link UpdateRequest} to add.
+ */
+ void add(UpdateRequest... updateRequests);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java
new file mode 100644
index 00000000000000..c90e94ce454d3d
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/RestClientFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.opensearch.client.RestClientBuilder;
+
+import java.io.Serializable;
+
+/**
+ * A factory that is used to configure the {@link org.opensearch.client.RestHighLevelClient}
+ * internally used in the {@link OpensearchSink}.
+ *
+ * @deprecated This has been deprecated and will be removed in the future.
+ */
+@Deprecated
+@PublicEvolving
+public interface RestClientFactory extends Serializable {
+
+ /**
+ * Configures the rest client builder.
+ *
+ * @param restClientBuilder the configured rest client builder.
+ */
+ void configureRestClientBuilder(RestClientBuilder restClientBuilder);
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java
new file mode 100644
index 00000000000000..a04cd559b1aa3d
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/IgnoringFailureHandler.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.opensearch.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+
+import org.opensearch.action.ActionRequest;
+
+/** Ignores all kinds of failures and drops the affected {@link ActionRequest}. */
+@Internal
+public class IgnoringFailureHandler implements ActionRequestFailureHandler {
+
+ private static final long serialVersionUID = 1662846593501L;
+
+ @Override
+ public void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) {
+ // ignore failure
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java
new file mode 100644
index 00000000000000..a2c60733bd5a1f
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/NoOpFailureHandler.java
@@ -0,0 +1,54 @@
+/*
+ * 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.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+
+import org.opensearch.action.ActionRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */
+@Internal
+public class NoOpFailureHandler implements ActionRequestFailureHandler {
+
+ private static final long serialVersionUID = 737941343410827885L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class);
+
+ @Override
+ public void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer)
+ throws Throwable {
+ LOG.error("Failed Opensearch item request: {}", failure.getMessage(), failure);
+ // simply fail the sink
+ throw failure;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ return o instanceof NoOpFailureHandler;
+ }
+
+ @Override
+ public int hashCode() {
+ return NoOpFailureHandler.class.hashCode();
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.java
new file mode 100644
index 00000000000000..922004ebe24dea
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/main/java/org/apache/flink/streaming/connectors/opensearch/util/RetryRejectedExecutionFailureHandler.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.connectors.opensearch.util;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.streaming.connectors.opensearch.ActionRequestFailureHandler;
+import org.apache.flink.streaming.connectors.opensearch.RequestIndexer;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.opensearch.action.ActionRequest;
+import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary {@link
+ * OpenSearchRejectedExecutionException}s (which means that Opensearch node queues are currently
+ * full), and fails for all other failures.
+ *
+ * @deprecated This hase been deprecated and will be removed in the future.
+ */
+@Deprecated
+@PublicEvolving
+public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler {
+
+ private static final long serialVersionUID = -7423562912824511906L;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class);
+
+ @Override
+ public void onFailure(
+ ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer)
+ throws Throwable {
+ LOG.error("Failed Opensearch item request: {}", failure.getMessage(), failure);
+ if (ExceptionUtils.findThrowable(failure, OpenSearchRejectedExecutionException.class)
+ .isPresent()) {
+ indexer.add(action);
+ } else {
+ // rethrow all other failures
+ throw failure;
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-opensearch/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 00000000000000..2cbd625e15e6d7
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/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.OpensearchDynamicSinkFactory
diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java
new file mode 100644
index 00000000000000..d550d847f1d458
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/OpensearchUtil.java
@@ -0,0 +1,166 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.opensearch.RestClientFactory;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.testcontainers.opensearch.OpensearchContainer;
+
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.conn.ssl.TrustAllStrategy;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.ssl.SSLContexts;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestClientBuilder.HttpClientConfigCallback;
+import org.opensearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+
+/** Collection of utility methods for Opensearch tests. */
+@Internal
+public class OpensearchUtil {
+
+ private OpensearchUtil() {}
+
+ /**
+ * Creates a preconfigured {@link OpensearchContainer} with limited memory allocation and aligns
+ * the internal Opensearch log levels with the ones used by the capturing logger.
+ *
+ * @param dockerImageVersion describing the Opensearch image
+ * @param log to derive the log level from
+ * @return configured Opensearch container
+ */
+ public static OpensearchContainer createOpensearchContainer(
+ String dockerImageVersion, Logger log) {
+ String logLevel;
+ if (log.isTraceEnabled()) {
+ logLevel = "TRACE";
+ } else if (log.isDebugEnabled()) {
+ logLevel = "DEBUG";
+ } else if (log.isInfoEnabled()) {
+ logLevel = "INFO";
+ } else if (log.isWarnEnabled()) {
+ logLevel = "WARN";
+ } else if (log.isErrorEnabled()) {
+ logLevel = "ERROR";
+ } else {
+ logLevel = "OFF";
+ }
+
+ return new OpensearchContainer(DockerImageName.parse(dockerImageVersion))
+ .withEnv("OPENSEARCH_JAVA_OPTS", "-Xms2g -Xmx2g")
+ .withEnv("logger.org.opensearch", logLevel)
+ .withLogConsumer(new Slf4jLogConsumer(log));
+ }
+
+ /**
+ * Creates a preconfigured {@link RestHighLevelClient} instance for specific {@link
+ * OpensearchContainer} instance.
+ *
+ * @return preconfigured {@link RestHighLevelClient} instance
+ */
+ public static RestHighLevelClient createClient(OpensearchContainer container) {
+ final String username = container.getUsername();
+ final String password = container.getPassword();
+
+ return new RestHighLevelClient(
+ RestClient.builder(HttpHost.create(container.getHttpHostAddress()))
+ .setHttpClientConfigCallback(
+ createClientConfigCallback(username, password)));
+ }
+
+ /**
+ * Creates a preconfigured {@link RestClientFactory} instance for specific {@link
+ * OpensearchContainer} instance.
+ *
+ * @return preconfigured {@link RestClientFactory} instance
+ */
+ public static RestClientFactory createClientFactory(OpensearchContainer container) {
+ final String username = container.getUsername();
+ final String password = container.getPassword();
+
+ return factory ->
+ factory.setHttpClientConfigCallback(createClientConfigCallback(username, password));
+ }
+
+ /**
+ * Creates a dedicated {@link HttpClientConfigCallback} instance for specific {@link
+ * OpensearchContainer} instance.
+ *
+ * @return dedicated {@link HttpClientConfigCallback} instance
+ */
+ private static HttpClientConfigCallback createClientConfigCallback(
+ final String username, final String password) {
+
+ return (httpClientBuilder) -> {
+ try {
+ final CredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+ credentialsProvider.setCredentials(
+ AuthScope.ANY, new UsernamePasswordCredentials(username, password));
+
+ return httpClientBuilder
+ .setDefaultCredentialsProvider(credentialsProvider)
+ .setSSLContext(
+ SSLContexts.custom()
+ .loadTrustMaterial(new TrustAllStrategy())
+ .build());
+ } catch (final NoSuchAlgorithmException
+ | KeyStoreException
+ | KeyManagementException ex) {
+ throw new RuntimeException(ex);
+ }
+ };
+ }
+
+ /** A mock {@link DynamicTableSink.Context} for Opensearch tests. */
+ public static class MockContext implements DynamicTableSink.Context {
+ @Override
+ public boolean isBounded() {
+ return false;
+ }
+
+ @Override
+ public TypeInformation> createTypeInformation(DataType consumedDataType) {
+ return null;
+ }
+
+ @Override
+ public TypeInformation> createTypeInformation(LogicalType consumedLogicalType) {
+ return null;
+ }
+
+ @Override
+ public DynamicTableSink.DataStructureConverter createDataStructureConverter(
+ DataType consumedDataType) {
+ return null;
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java
new file mode 100644
index 00000000000000..d7fb231bd9277c
--- /dev/null
+++ b/flink-connectors/flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkBuilderTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for {@link OpensearchSinkBuilder}. */
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+class OpensearchSinkBuilderTest {
+
+ @TestFactory
+ Stream testValidBuilders() {
+ Stream> 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,
+ OpensearchSinkBuilder::toString,
+ builder -> assertDoesNotThrow(builder::build));
+ }
+
+ @Test
+ void testThrowIfExactlyOnceConfigured() {
+ assertThrows(
+ IllegalStateException.class,
+ () -> createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE));
+ }
+
+ @Test
+ void testThrowIfHostsNotSet() {
+ assertThrows(
+ NullPointerException.class,
+ () -> createEmptyBuilder().setEmitter((element, indexer, context) -> {}).build());
+ }
+
+ @Test
+ void testThrowIfEmitterNotSet() {
+ assertThrows(
+ NullPointerException.class,
+ () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build());
+ }
+
+ @Test
+ void testThrowIfSetInvalidTimeouts() {
+ assertThrows(
+ IllegalStateException.class,
+ () -> createEmptyBuilder().setConnectionRequestTimeout(-1).build());
+ assertThrows(
+ IllegalStateException.class,
+ () -> createEmptyBuilder().setConnectionTimeout(-1).build());
+ assertThrows(
+ IllegalStateException.class,
+ () -> createEmptyBuilder().setSocketTimeout(-1).build());
+ }
+
+ private OpensearchSinkBuilder