dannycranmer commented on code in PR #5:
URL: 
https://github.com/apache/flink-connector-opensearch/pull/5#discussion_r1102989699


##########
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DocSerdeRequest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.common.bytes.BytesReference;
+import org.opensearch.common.io.stream.BytesStreamOutput;
+import org.opensearch.common.io.stream.InputStreamStreamInput;
+import org.opensearch.common.io.stream.StreamInput;
+import org.opensearch.common.io.stream.StreamOutput;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Wrapper class around {@link DocWriteRequest} since it does not implement 
{@link Serializable},
+ * required by AsyncSink scaffolding.
+ *
+ * @param <T> type of the write request
+ */
+@PublicEvolving
+public class DocSerdeRequest<T> implements Serializable {

Review Comment:
   Ii think the class level generics are redundant here. We are using `<?>` 
throughout. Consider changing `private final DocWriteRequest<T> request;` to 
`private final DocWriteRequest<?> request;` and removing class generics. This 
makes the Sink interface a bit messy `extends AsyncSinkBase<InputT, 
DocSerdeRequest<?>>`



##########
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.DocWriteRequest;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Builder to construct an Opensearch compatible {@link OpensearchAsyncSink}.
+ *
+ * <p>The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * <pre>{@code
+ * OpensearchAsyncSink<Tuple2<String, String>> sink = OpensearchAsyncSink
+ *     .<Tuple2<String, String>>builder()
+ *     .setHosts(new HttpHost("localhost:9200")
+ *     .setElementConverter((element, context) ->
+ *         new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ *     .build();
+ * }</pre>
+ *
+ * @param <InputT> type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder<InputT>
+        extends AsyncSinkBaseBuilder<
+                InputT, DocSerdeRequest<?>, 
OpensearchAsyncSinkBuilder<InputT>> {
+    private List<HttpHost> hosts;
+    private String username;
+    private String password;
+    private String connectionPathPrefix;
+    private Integer connectionTimeout;
+    private Integer connectionRequestTimeout;
+    private Integer socketTimeout;
+    private Boolean allowInsecure;
+    private ElementConverter<InputT, DocSerdeRequest<?>> elementConverter;
+
+    /**
+     * Sets the element converter.
+     *
+     * @param elementConverter element converter
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setElementConverter(
+            ElementConverter<InputT, DocWriteRequest<?>> elementConverter) {
+        this.elementConverter =
+                (element, context) ->
+                        DocSerdeRequest.from(elementConverter.apply(element, 
context));
+        return this;
+    }
+
+    /**
+     * Sets the hosts where the Opensearch cluster nodes are reachable.
+     *
+     * @param hosts http addresses describing the node locations
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setHosts(HttpHost... hosts) {
+        checkNotNull(hosts);
+        checkArgument(hosts.length > 0, "Hosts cannot be empty.");
+        this.hosts = Arrays.asList(hosts);
+        return this;
+    }
+
+    /**
+     * Sets the username used to authenticate the connection with the 
Opensearch cluster.
+     *
+     * @param username of the Opensearch cluster user
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionUsername(String 
username) {
+        checkNotNull(username);
+        checkArgument(!username.trim().isEmpty(), "Username cannot be empty");
+        this.username = username;
+        return this;
+    }
+
+    /**
+     * Sets the password used to authenticate the conection with the 
Opensearch cluster.
+     *
+     * @param password of the Opensearch cluster user
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionPassword(String 
password) {
+        checkNotNull(password);
+        checkArgument(!password.trim().isEmpty(), "Password cannot be empty");
+        this.password = password;
+        return this;
+    }
+
+    /**
+     * Sets a prefix which used for every REST communication to the Opensearch 
cluster.
+     *
+     * @param prefix for the communication
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionPathPrefix(String 
prefix) {
+        checkNotNull(prefix);
+        this.connectionPathPrefix = prefix;
+        return this;
+    }
+
+    /**
+     * Sets the timeout for requesting the connection of the Opensearch 
cluster from the connection
+     * manager.
+     *
+     * @param timeout timeout for the connection request (in milliseconds)
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int 
timeout) {
+        checkArgument(
+                timeout >= 0, "Connection request timeout must be larger than 
or equal to 0.");
+        this.connectionRequestTimeout = timeout;
+        return this;
+    }
+
+    /**
+     * Sets the timeout for establishing a connection of the Opensearch 
cluster.
+     *
+     * @param timeout timeout for the connection (in milliseconds)
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int 
timeout) {
+        checkArgument(timeout >= 0, "Connection timeout must be larger than or 
equal to 0.");
+        this.connectionTimeout = timeout;
+        return this;
+    }
+
+    /**
+     * Sets the timeout for establishing a connection of the Opensearch 
cluster.
+     *
+     * @param timeout timeout for the connection (in milliseconds)
+     * @param timeUnit timeout time unit
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int 
timeout, TimeUnit timeUnit) {
+        checkNotNull(timeUnit, "TimeUnit cannot be null.");
+        return setConnectionTimeout((int) timeUnit.toMillis(timeout));
+    }
+
+    /**
+     * Sets the timeout for waiting for data or, put differently, a maximum 
period inactivity
+     * between two consecutive data packets.
+     *
+     * @param timeout timeout for the socket (in milliseconds)
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout) {
+        checkArgument(timeout >= 0, "Socket timeout must be larger than or 
equal to 0.");
+        this.socketTimeout = timeout;
+        return this;
+    }
+
+    /**
+     * Sets the timeout for waiting for data or, put differently, a maximum 
period inactivity
+     * between two consecutive data packets.
+     *
+     * @param timeout timeout for the socket
+     * @param timeUnit timeout time unit
+     * @return this builder
+     */
+    public OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout, 
TimeUnit timeUnit) {
+        checkNotNull(timeUnit, "TimeUnit cannot be null.");
+        return setSocketTimeout((int) timeUnit.toMillis(timeout));
+    }
+
+    /**
+     * 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 OpensearchAsyncSinkBuilder<InputT> setAllowInsecure(boolean 
allowInsecure) {
+        this.allowInsecure = allowInsecure;
+        return this;
+    }
+
+    @Override
+    public OpensearchAsyncSink<InputT> build() {
+        return new OpensearchAsyncSink<InputT>(
+                nonNullOrDefault(
+                        getMaxBatchSize(),
+                        1000), /* 
OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION */
+                nonNullOrDefault(
+                        getMaxInFlightRequests(), 1), /* 
BulkProcessor::concurrentRequests */
+                nonNullOrDefault(getMaxBufferedRequests(), 10000),

Review Comment:
   Can we also promote the other magic numbers to constants? `10000` and `2 * 
1024 * 1024`



##########
.github/workflows/push_pr.yml:
##########
@@ -25,4 +25,4 @@ jobs:
   compile_and_test:
     uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
     with:
-      flink_version: 1.16.0
+      flink_version: 1.16.1

Review Comment:
   We should split this out to a separate PR. We can merge this independently 



##########
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DocSerdeRequest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.opensearch.action.DocWriteRequest;
+import org.opensearch.action.delete.DeleteRequest;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.update.UpdateRequest;
+import org.opensearch.common.bytes.BytesReference;
+import org.opensearch.common.io.stream.BytesStreamOutput;
+import org.opensearch.common.io.stream.InputStreamStreamInput;
+import org.opensearch.common.io.stream.StreamInput;
+import org.opensearch.common.io.stream.StreamOutput;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Wrapper class around {@link DocWriteRequest} since it does not implement 
{@link Serializable},
+ * required by AsyncSink scaffolding.
+ *
+ * @param <T> type of the write request
+ */
+@PublicEvolving
+public class DocSerdeRequest<T> implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final DocWriteRequest<T> request;
+
+    private DocSerdeRequest(DocWriteRequest<T> request) {
+        this.request = request;
+    }
+
+    public DocWriteRequest<T> getRequest() {
+        return request;
+    }
+
+    static <T> DocSerdeRequest<T> from(DocWriteRequest<T> request) {
+        return new DocSerdeRequest<>(request);
+    }
+
+    static DocSerdeRequest<?> readFrom(long requestSize, DataInputStream in) 
throws IOException {
+        try (final StreamInput stream = new InputStreamStreamInput(in, 
requestSize)) {
+            return new DocSerdeRequest<>(readDocumentRequest(stream));
+        }
+    }
+
+    void writeTo(DataOutputStream out) throws IOException {
+        try (BytesStreamOutput stream = new BytesStreamOutput()) {
+            writeDocumentRequest(stream, request);
+            out.write(BytesReference.toBytes(stream.bytes()));
+        }
+    }
+
+    /** Read a document write (index/delete/update) request. */
+    private static DocWriteRequest<?> readDocumentRequest(StreamInput in) 
throws IOException {
+        byte type = in.readByte();
+        DocWriteRequest<?> docWriteRequest;
+        if (type == 0) {
+            docWriteRequest = new IndexRequest(in);
+        } else if (type == 1) {
+            docWriteRequest = new DeleteRequest(in);
+        } else if (type == 2) {
+            docWriteRequest = new UpdateRequest(in);
+        } else {
+            throw new IllegalStateException("Invalid request type [" + type + 
" ]");
+        }
+        return docWriteRequest;

Review Comment:
   +1 for a unit test. Unless there is a good reason not to, unit tests give 
quicker feedback.



##########
flink-connector-opensearch-e2e-tests/src/main/java/org/apache/flink/streaming/tests/OpensearchAsyncSinkExample.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.tests;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.connector.opensearch.sink.OpensearchAsyncSink;
+import org.apache.flink.connector.opensearch.sink.OpensearchAsyncSinkBuilder;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+
+import org.apache.http.HttpHost;
+import org.opensearch.action.index.IndexRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** End to end test for OpensearchAsyncSink. */
+public class OpensearchAsyncSinkExample {
+
+    public static void main(String[] args) throws Exception {
+
+        final ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+        if (parameterTool.getNumberOfParameters() < 2) {
+            System.out.println(
+                    "Missing parameters!\n" + "Usage: --numRecords 
<numRecords> --index <index>");
+            return;
+        }
+
+        final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(5000);
+
+        DataStream<Tuple2<String, String>> source =
+                env.fromSequence(0, parameterTool.getInt("numRecords") - 1)
+                        .flatMap(
+                                new FlatMapFunction<Long, Tuple2<String, 
String>>() {
+                                    @Override
+                                    public void flatMap(
+                                            Long value, 
Collector<Tuple2<String, String>> out) {
+                                        final String key = 
String.valueOf(value);
+                                        final String message = "message #" + 
value;
+                                        out.collect(Tuple2.of(key, message + 
"update #1"));
+                                        out.collect(Tuple2.of(key, message + 
"update #2"));
+                                    }
+                                });
+
+        List<HttpHost> httpHosts = new ArrayList<>();
+        httpHosts.add(new HttpHost("127.0.0.1", 9200, "http"));
+
+        OpensearchAsyncSinkBuilder<Tuple2<String, String>> osSinkBuilder =
+                OpensearchAsyncSink.<Tuple2<String, String>>builder()
+                        .setHosts(new HttpHost("localhost:9200"))
+                        .setElementConverter(
+                                (element, context) ->
+                                        new IndexRequest("my-index")
+                                                .id(element.f0.toString())
+                                                .source(element.f1));

Review Comment:
   This is a shame indeed, because Async Sink does not actually need 
Serializable. https://issues.apache.org/jira/browse/FLINK-27537



##########
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.api.connector.sink2.Sink;
+import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import 
org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+
+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.OpenSearchException;
+import org.opensearch.action.ActionListener;
+import org.opensearch.action.bulk.BulkItemResponse;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Flink's Async Sink Writer to insert or update data in an Opensearch 
index (see please
+ * {@link OpensearchAsyncSink}).
+ *
+ * @param <InputT> type of the records converted to Opensearch actions 
(instances of {@link
+ *     DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter<InputT> extends AsyncSinkWriter<InputT, 
DocSerdeRequest<?>> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(OpensearchAsyncWriter.class);
+
+    private final RestHighLevelClient client;
+    private final Counter numRecordsOutErrorsCounter;
+    private volatile boolean closed = false;
+
+    private static final FatalExceptionClassifier 
OPENSEARCH_FATAL_EXCEPTION_CLASSIFIER =
+            FatalExceptionClassifier.createChain(
+                    new FatalExceptionClassifier(
+                            err ->
+                                    err instanceof NoRouteToHostException
+                                            || err instanceof ConnectException,
+                            err ->
+                                    new OpenSearchException(
+                                            "Could not connect to Opensearch 
cluster using provided hosts",
+                                            err)));
+
+    /**
+     * Constructor creating an Opensearch async writer.
+     *
+     * @param context the initialization context
+     * @param elementConverter converting incoming records to Opensearch write 
document requests
+     * @param maxBatchSize the maximum size of a batch of entries that may be 
sent
+     * @param maxInFlightRequests he maximum number of in flight requests that 
may exist, if any
+     *     more in flight requests need to be initiated once the maximum has 
been reached, then it
+     *     will be blocked until some have completed
+     * @param maxBufferedRequests the maximum number of elements held in the 
buffer, requests to add
+     *     elements will be blocked while the number of elements in the buffer 
is at the maximum
+     * @param maxBatchSizeInBytes the maximum size of a batch of entries that 
may be sent to KDS
+     *     measured in bytes
+     * @param maxTimeInBufferMS the maximum amount of time an entry is allowed 
to live in the
+     *     buffer, if any element reaches this age, the entire buffer will be 
flushed immediately
+     * @param maxRecordSizeInBytes the maximum size of a record the sink will 
accept into the
+     *     buffer, a record of size larger than this will be rejected when 
passed to the sink
+     * @param hosts the reachable Opensearch cluster nodes
+     * @param networkClientConfig describing properties of the network 
connection used to connect to
+     *     the Opensearch cluster
+     * @param initialStates the initial state of the sink
+     */
+    OpensearchAsyncWriter(
+            Sink.InitContext context,
+            ElementConverter<InputT, DocSerdeRequest<?>> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            List<HttpHost> hosts,
+            NetworkClientConfig networkClientConfig,
+            Collection<BufferedRequestState<DocSerdeRequest<?>>> 
initialStates) {
+        super(
+                elementConverter,
+                context,
+                AsyncSinkWriterConfiguration.builder()
+                        .setMaxBatchSize(maxBatchSize)
+                        .setMaxBatchSizeInBytes(maxBatchSizeInBytes)
+                        .setMaxInFlightRequests(maxInFlightRequests)
+                        .setMaxBufferedRequests(maxBufferedRequests)
+                        .setMaxTimeInBufferMS(maxTimeInBufferMS)
+                        .setMaxRecordSizeInBytes(maxRecordSizeInBytes)
+                        .build(),
+                initialStates);
+
+        this.client =
+                new RestHighLevelClient(
+                        configureRestClientBuilder(
+                                RestClient.builder(hosts.toArray(new 
HttpHost[0])),
+                                networkClientConfig));
+
+        final SinkWriterMetricGroup metricGroup = context.metricGroup();
+        checkNotNull(metricGroup);
+
+        this.numRecordsOutErrorsCounter = 
metricGroup.getNumRecordsOutErrorsCounter();
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DocSerdeRequest<?>> requestEntries,
+            Consumer<List<DocSerdeRequest<?>>> requestResult) {
+
+        BulkRequest bulkRequest = new BulkRequest();
+        requestEntries.forEach(r -> bulkRequest.add(r.getRequest()));
+
+        final CompletableFuture<BulkResponse> future = new 
CompletableFuture<>();
+        client.bulkAsync(
+                bulkRequest,
+                RequestOptions.DEFAULT,
+                new ActionListener<BulkResponse>() {
+                    @Override
+                    public void onResponse(BulkResponse response) {
+                        future.complete(response);
+                    }
+
+                    @Override
+                    public void onFailure(Exception e) {
+                        future.completeExceptionally(e);
+                    }
+                });
+
+        future.whenComplete(
+                (response, err) -> {
+                    if (err != null) {
+                        handleFullyFailedBulkRequest(err, requestEntries, 
requestResult);
+                    } else if (response.hasFailures()) {
+                        handlePartiallyFailedBulkRequests(response, 
requestEntries, requestResult);
+                    } else {
+                        requestResult.accept(Collections.emptyList());
+                    }
+                });
+    }
+
+    @Override
+    protected long getSizeInBytes(DocSerdeRequest<?> requestEntry) {
+        return requestEntry.getRequest().ramBytesUsed();
+    }
+
+    @Override
+    public void close() {
+        if (!closed) {
+            closed = true;
+
+            try {
+                client.close();
+            } catch (final IOException ex) {
+                LOG.warn("Error while closing RestHighLevelClient instance", 
ex);
+            }
+        }
+    }
+
+    private boolean isRetryable(Throwable err) {
+        // isFatal() is really isNotFatal()
+        if (!OPENSEARCH_FATAL_EXCEPTION_CLASSIFIER.isFatal(err, 
getFatalExceptionCons())) {
+            return false;
+        }
+        return true;
+    }
+
+    private void handleFullyFailedBulkRequest(
+            Throwable err,
+            List<DocSerdeRequest<?>> requestEntries,
+            Consumer<List<DocSerdeRequest<?>>> requestResult) {
+        final boolean retryable = isRetryable(err.getCause());
+
+        LOG.warn(
+                "Opensearch AsyncWwriter failed to persist {} entries 
(retryable = {})",
+                requestEntries.size(),
+                retryable,
+                err);
+
+        numRecordsOutErrorsCounter.inc(requestEntries.size());
+
+        if (retryable) {
+            requestResult.accept(requestEntries);
+        }
+    }
+
+    private void handlePartiallyFailedBulkRequests(
+            BulkResponse response,
+            List<DocSerdeRequest<?>> requestEntries,
+            Consumer<List<DocSerdeRequest<?>>> requestResult) {
+
+        final List<DocSerdeRequest<?>> failedRequestEntries = new 
ArrayList<>();
+        final BulkItemResponse[] items = response.getItems();
+
+        for (int i = 0; i < items.length; i++) {
+            if (items[i].getFailure() != null) {
+                
failedRequestEntries.add(DocSerdeRequest.from(requestEntries.get(i).getRequest()));
+            }
+        }
+
+        numRecordsOutErrorsCounter.inc(failedRequestEntries.size());
+        requestResult.accept(failedRequestEntries);
+    }
+
+    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().orElse(false)) {
+                        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;

Review Comment:
   nit: Should we move this out to a separate class?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to