hlteoh37 commented on code in PR #5: URL: https://github.com/apache/flink-connector-opensearch/pull/5#discussion_r1099182385
########## 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")) Review Comment: Hm, should we instead define a constant something like `OPENSEARCH_DOMAIN` so users can use the example more easily? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { + checkState(timeout >= 0, "Connection request timeout must be larger than or equal to 0."); Review Comment: would `checkArgument` be better here? (same for the other arguments) ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { Review Comment: Do we want to add the unit in the name? `setConnectionRequestTimeoutMillis`? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java: ########## @@ -0,0 +1,274 @@ +/* + * 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.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.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.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; + + /** + * 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 void handleFullyFailedBulkRequest( + Throwable err, Review Comment: Should we consider logging this error? Otherwise the sink can get stuck in a retry loop without any logs. ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSink.java: ########## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBase; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Apache Flink's Async Sink to insert or update data in an Opensearch index (see please {@link + * OpensearchAsyncWriter}). + * + * @param <InputT> type of the records converted to Opensearch actions (instances of {@link + * DocSerdeRequest}) + * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink + */ +@PublicEvolving +public class OpensearchAsyncSink<InputT> extends AsyncSinkBase<InputT, DocSerdeRequest<?>> { + private static final long serialVersionUID = 1L; + + private final List<HttpHost> hosts; + private final NetworkClientConfig networkClientConfig; + + /** + * Constructor creating an Opensearch async sink. + * + * @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 Review Comment: ```suggestion * @param maxInFlightRequests the maximum number of in flight requests that may exist, if any ``` ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSink.java: ########## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBase; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Apache Flink's Async Sink to insert or update data in an Opensearch index (see please {@link + * OpensearchAsyncWriter}). + * + * @param <InputT> type of the records converted to Opensearch actions (instances of {@link + * DocSerdeRequest}) + * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink + */ +@PublicEvolving +public class OpensearchAsyncSink<InputT> extends AsyncSinkBase<InputT, DocSerdeRequest<?>> { + private static final long serialVersionUID = 1L; + + private final List<HttpHost> hosts; + private final NetworkClientConfig networkClientConfig; + + /** + * Constructor creating an Opensearch async sink. + * + * @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 Review Comment: ~~KDS~~ OpenSearch ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSink.java: ########## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBase; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Apache Flink's Async Sink to insert or update data in an Opensearch index (see please {@link + * OpensearchAsyncWriter}). + * + * @param <InputT> type of the records converted to Opensearch actions (instances of {@link + * DocSerdeRequest}) + * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink + */ +@PublicEvolving +public class OpensearchAsyncSink<InputT> extends AsyncSinkBase<InputT, DocSerdeRequest<?>> { + private static final long serialVersionUID = 1L; + + private final List<HttpHost> hosts; + private final NetworkClientConfig networkClientConfig; + + /** + * Constructor creating an Opensearch async sink. + * + * @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 elementConverter converting incoming records to Opensearch write document requests + * @param hosts the reachable Opensearch cluster nodes + * @param networkClientConfig describing properties of the network connection used to connect to + * the Opensearch cluster + */ + OpensearchAsyncSink( + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long maxBatchSizeInBytes, + long maxTimeInBufferMS, + long maxRecordSizeInBytes, Review Comment: Should we consider adding validation for this value to be < 100MB, which is the maximum configurable payload size on OpenSearch? https://docs.aws.amazon.com/opensearch-service/latest/developerguide/limits.html ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(hosts.length > 0, "Hosts cannot be empty."); Review Comment: would `checkArgument` be a better method to call here? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(hosts.length > 0, "Hosts cannot be empty."); Review Comment: Hm, also we check this twice, once in builder, and once in constructor. Would it be better to just validate this in the constructor ? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); Review Comment: can we add another check validating this isn't empty? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java: ########## @@ -0,0 +1,274 @@ +/* + * 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.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.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.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; + + /** + * 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 void handleFullyFailedBulkRequest( + Throwable err, Review Comment: Are there any exceptions we want to classify as non-retryable and fail the Flink job? For example "domain doesn't exist" or "insufficient permissions"? See example here https://github.com/apache/flink-connector-aws/blob/9e09d57210/flink-connector-dynamodb/src/main/java/org/apache/flink/connector/dynamodb/sink/DynamoDbSinkWriter.java#L104 ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { + checkState(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 for the connection + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return this; + } + + /** + * 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 OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout) { Review Comment: should we add the unit to the name? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + this.password = password; Review Comment: same here, can we check not empty? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { + checkState(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 for the connection + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return this; + } + + /** + * 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 OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return this; + } + + /** + * 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() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); Review Comment: Hm, spreading out the checks makes it quite hard to identify all the relevant checks. Could we transfer all of them into the constructor of `OpensearchAsyncSink`? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { + checkState(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 for the connection + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return this; + } + + /** + * 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 OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return this; + } + + /** + * 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() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + + return new OpensearchAsyncSink<InputT>( + nonNullOrDefault( + getMaxBatchSize(), + 1000), /* OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION */ Review Comment: Could we use Optional instead? `Optional.ofNullable(getMaxBatchSize()).orElse(DEFAULT_MAX_BATCH_SIZE)` Also should we define constant names instead of using `1000` directly? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ########## @@ -0,0 +1,220 @@ +/* + * 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 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 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); + checkState(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); + 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); + 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 for the connection request + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionRequestTimeout(int timeout) { + checkState(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 for the connection + * @return this builder + */ + public OpensearchAsyncSinkBuilder<InputT> setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return this; + } + + /** + * 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 OpensearchAsyncSinkBuilder<InputT> setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return this; + } + + /** + * 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() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + + return new OpensearchAsyncSink<InputT>( + nonNullOrDefault( + getMaxBatchSize(), + 1000), /* OpensearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION */ + nonNullOrDefault( + getMaxInFlightRequests(), 1), /* BulkProcessor::concurrentRequests */ + nonNullOrDefault(getMaxBufferedRequests(), 10000), + nonNullOrDefault( + getMaxBatchSizeInBytes(), + 2 * 1024 + * 1024), /* OpensearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION */ + nonNullOrDefault( Review Comment: should we make the default something along the lines of OpenSearch service limits like 10MiB or 100MiB? https://docs.aws.amazon.com/opensearch-service/latest/developerguide/limits.html ########## 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: These methods are untested. Should we add unit tests for them? ########## 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: Hmm.. Since we have to implement a `DocSerdeRequest`, should we consider exposing this in the interface instead of `OpenSearch` classes? This might be helpful in the event OpenSearch's interface changes. ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java: ########## @@ -0,0 +1,274 @@ +/* + * 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.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.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.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; + + /** + * 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 void handleFullyFailedBulkRequest( + Throwable err, + List<DocSerdeRequest<?>> requestEntries, + Consumer<List<DocSerdeRequest<?>>> requestResult) { + numRecordsOutErrorsCounter.inc(requestEntries.size()); + 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) { Review Comment: Should we consider logging this error? ########## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java: ########## @@ -0,0 +1,274 @@ +/* + * 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.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.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.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; + + /** + * 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 void handleFullyFailedBulkRequest( + Throwable err, + List<DocSerdeRequest<?>> requestEntries, + Consumer<List<DocSerdeRequest<?>>> requestResult) { + numRecordsOutErrorsCounter.inc(requestEntries.size()); + 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; + }); + 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; + }); + } Review Comment: nit: Seems unnecessary to do 2 null checks. Should we instead just remove the outer `if`? -- 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]
