[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-13 Thread via GitHub


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


##
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java:
##
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import 
org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.conn.ssl.TrustAllStrategy;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.ssl.SSLContexts;
+import org.opensearch.OpenSearchException;
+import org.opensearch.action.ActionListener;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestClientBuilder;
+import org.opensearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Flink's Async Sink Writer to insert or update data in an Opensearch 
index (see please
+ * {@link OpensearchAsyncSink}).
+ *
+ * @param  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter extends AsyncSinkWriter> {
+private static final Logger LOG = 
LoggerFactory.getLogger(OpensearchAsyncWriter.class);
+
+private final RestHighLevelClient client;
+private final Counter numRecordsOutErrorsCounter;
+private volatile boolean closed = false;
+
+private static final FatalExceptionClassifier 
OPENSEARCH_FATAL_EXCEPTION_CLASSIFIER =
+FatalExceptionClassifier.createChain(
+new FatalExceptionClassifier(
+err ->
+err instanceof NoRouteToHostException
+|| err instanceof ConnectException,
+err ->
+new OpenSearchException(
+"Could not connect to Opensearch 
cluster using provided hosts",
+err)));
+
+/**
+ * Constructor creating an Opensearch async writer.
+ *
+ * @param context the initialization context
+ * @param elementConverter converting incoming records to Opensearch write 
document requests
+ * @param maxBatchSize the maximum size of a batch of entries that may be 
sent
+ * @param maxInFlightRequests he maximum number of in flight requests that 
may exist, if any
+ * more in flight requests need to be initiated once the maximum has 
been reached, then it
+ * will be blocked until some have completed
+ * @param maxBufferedRequests the maximum number of elements held in the 
buffer, requests to add
+ * elements will be bloc

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-10 Thread via GitHub


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


##
.github/workflows/push_pr.yml:
##
@@ -25,6 +25,6 @@ jobs:
   compile_and_test:
 uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
 with:
-  flink_version: 1.16.0
-  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz
+  flink_version: 1.16.1
+  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.1/flink-1.16.1-bin-scala_2.12.tgz

Review Comment:
   (addressed in https://github.com/apache/flink-connector-opensearch/pull/7)



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-10 Thread via GitHub


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


##
flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncWriter.java:
##
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.opensearch.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import 
org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.conn.ssl.TrustAllStrategy;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.ssl.SSLContexts;
+import org.opensearch.OpenSearchException;
+import org.opensearch.action.ActionListener;
+import org.opensearch.action.bulk.BulkItemResponse;
+import org.opensearch.action.bulk.BulkRequest;
+import org.opensearch.action.bulk.BulkResponse;
+import org.opensearch.client.RequestOptions;
+import org.opensearch.client.RestClient;
+import org.opensearch.client.RestClientBuilder;
+import org.opensearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Apache Flink's Async Sink Writer to insert or update data in an Opensearch 
index (see please
+ * {@link OpensearchAsyncSink}).
+ *
+ * @param  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter extends AsyncSinkWriter> {
+private static final Logger LOG = 
LoggerFactory.getLogger(OpensearchAsyncWriter.class);
+
+private final RestHighLevelClient client;
+private final Counter numRecordsOutErrorsCounter;
+private volatile boolean closed = false;
+
+private static final FatalExceptionClassifier 
OPENSEARCH_FATAL_EXCEPTION_CLASSIFIER =
+FatalExceptionClassifier.createChain(
+new FatalExceptionClassifier(
+err ->
+err instanceof NoRouteToHostException
+|| err instanceof ConnectException,
+err ->
+new OpenSearchException(
+"Could not connect to Opensearch 
cluster using provided hosts",
+err)));
+
+/**
+ * Constructor creating an Opensearch async writer.
+ *
+ * @param context the initialization context
+ * @param elementConverter converting incoming records to Opensearch write 
document requests
+ * @param maxBatchSize the maximum size of a batch of entries that may be 
sent
+ * @param maxInFlightRequests he maximum number of in flight requests that 
may exist, if any
+ * more in flight requests need to be initiated once the maximum has 
been reached, then it
+ * will be blocked until some have completed
+ * @param maxBufferedRequests the maximum number of elements held in the 
buffer, requests to add
+ * elements will be bloc

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-10 Thread via GitHub


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


##
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  type of the write request
+ */
+@PublicEvolving
+public class DocSerdeRequest implements Serializable {

Review Comment:
   ~This is my bad, the `T` must be constrained, I will fix it~ Removed `T`, 
not necessary indeed



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-10 Thread via GitHub


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


##
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  type of the write request
+ */
+@PublicEvolving
+public class DocSerdeRequest implements Serializable {

Review Comment:
   This is my bad, the `T` must be constrained, I will fix it



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-10 Thread via GitHub


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


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

Review Comment:
   @dannycranmer https://github.com/apache/flink-connector-opensearch/pull/7



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
.github/workflows/push_pr.yml:
##
@@ -25,6 +25,6 @@ jobs:
   compile_and_test:
 uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
 with:
-  flink_version: 1.16.0
-  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz
+  flink_version: 1.16.1
+  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.1/flink-1.16.1-bin-scala_2.12.tgz

Review Comment:
   Gotcha, thank you! (yes, I think we should support 1.16.1)



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
.github/workflows/push_pr.yml:
##
@@ -25,6 +25,6 @@ jobs:
   compile_and_test:
 uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
 with:
-  flink_version: 1.16.0
-  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz
+  flink_version: 1.16.1
+  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.1/flink-1.16.1-bin-scala_2.12.tgz

Review Comment:
   Thanks @MartijnVisser , still needs version change, right? 1.16.0 -> 1.16.1?



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
.github/workflows/push_pr.yml:
##
@@ -25,6 +25,6 @@ jobs:
   compile_and_test:
 uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
 with:
-  flink_version: 1.16.0
-  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz
+  flink_version: 1.16.1
+  flink_url: 
https://dist.apache.org/repos/dist/release/flink/flink-1.16.1/flink-1.16.1-bin-scala_2.12.tgz

Review Comment:
   
`https://dist.apache.org/repos/dist/release/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz`
 is not available anymore :-|



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter extends AsyncSinkWriter> {
+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
+ */
+

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter extends AsyncSinkWriter> {
+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
+ */
+

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the write request
+ */
+@PublicEvolving
+public class DocSerdeRequest implements Serializable {
+private static final long serialVersionUID = 1L;
+private final DocWriteRequest request;
+
+private DocSerdeRequest(DocWriteRequest request) {
+this.request = request;
+}
+
+public DocWriteRequest getRequest() {
+return request;
+}
+
+static  DocSerdeRequest from(DocWriteRequest 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:
   They are tested in scope if integration test, OpensearchAsyncSinkITCase, 
both reading and writing side.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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 
 --index ");
+return;
+}
+
+final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+env.enableCheckpointing(5000);
+
+DataStream> source =
+env.fromSequence(0, parameterTool.getInt("numRecords") - 1)
+.flatMap(
+new FlatMapFunction>() {
+@Override
+public void flatMap(
+Long value, 
Collector> 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 httpHosts = new ArrayList<>();
+httpHosts.add(new HttpHost("127.0.0.1", 9200, "http"));
+
+OpensearchAsyncSinkBuilder> osSinkBuilder =
+OpensearchAsyncSink.>builder()
+.setHosts(new HttpHost("localhost:9200"))
+.setElementConverter(
+(element, context) ->
+new IndexRequest("my-index")
+.id(element.f0.toString())
+.source(element.f1));

Review Comment:
   The `DocSerdeRequest` is sadly a necessary leaking abstraction (AsyncSink 
requires `Serializable`), we should export in the places when it is inevitable 
but in general we should only operate over OpenSearch APIs.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ */
+@Internal
+class OpensearchAsyncWriter extends AsyncSinkWriter> {
+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
+ */
+

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink
+ */
+@PublicEvolving
+public class OpensearchAsyncSink extends AsyncSinkBase> {
+private static final long serialVersionUID = 1L;
+
+private final List 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:
   This is specific to AWS OpenSearch managed service, this is not applicable 
to OpenSearch in general.
   



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 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 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 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 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
+ * @r

[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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}.
+ *
+ * The following example shows the minimal setup to create a 
OpensearchAsyncSink that submits
+ * actions with the default number of actions to buffer (1000).
+ *
+ * {@code
+ * OpensearchAsyncSink> sink = OpensearchAsyncSink
+ * .>builder()
+ * .setHosts(new HttpHost("localhost:9200")
+ * .setElementConverter((element, context) ->
+ * new 
IndexRequest("my-index").id(element.f0.toString()).source(element.f1));
+ * .build();
+ * }
+ *
+ * @param  type of the records converted to Opensearch actions
+ */
+@PublicEvolving
+public class OpensearchAsyncSinkBuilder
+extends AsyncSinkBaseBuilder<
+InputT, DocSerdeRequest, 
OpensearchAsyncSinkBuilder> {
+private List hosts;
+private String username;
+private String password;
+private String connectionPathPrefix;
+private Integer connectionTimeout;
+private Integer connectionRequestTimeout;
+private Integer socketTimeout;
+private Boolean allowInsecure;
+private ElementConverter> elementConverter;
+
+/**
+ * Sets the element converter.
+ *
+ * @param elementConverter element converter
+ */
+public OpensearchAsyncSinkBuilder setElementConverter(
+ElementConverter> 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 setHosts(HttpHost... hosts) {
+checkNotNull(hosts);
+checkState(hosts.length > 0, "Hosts cannot be empty.");

Review Comment:
   I thing checking twice is acceptable here: we should fail as early as 
possible, allowing to construct a builder with possibly illegal arguments and 
carrying it over could potentially raise an exception down the stack, when 
`build` method is called. By validating early, we are preventing that.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink
+ */
+@PublicEvolving
+public class OpensearchAsyncSink extends AsyncSinkBase> {
+private static final long serialVersionUID = 1L;
+
+private final List 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:
   :+1: 



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-08 Thread via GitHub


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


##
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  type of the records converted to Opensearch actions 
(instances of {@link
+ * DocSerdeRequest})
+ * @see OpensearchAsyncSinkBuilder on how to construct a OpensearchAsyncSink
+ */
+@PublicEvolving
+public class OpensearchAsyncSink extends AsyncSinkBase> {
+private static final long serialVersionUID = 1L;
+
+private final List 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:
   :+1: 



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-07 Thread via GitHub


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


##
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 
 --index ");
+return;
+}
+
+final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+env.enableCheckpointing(5000);
+
+DataStream> source =
+env.fromSequence(0, parameterTool.getInt("numRecords") - 1)
+.flatMap(
+new FlatMapFunction>() {
+@Override
+public void flatMap(
+Long value, 
Collector> 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 httpHosts = new ArrayList<>();
+httpHosts.add(new HttpHost("127.0.0.1", 9200, "http"));
+
+OpensearchAsyncSinkBuilder> osSinkBuilder =
+OpensearchAsyncSink.>builder()
+.setHosts(new HttpHost("localhost:9200"))

Review Comment:
   Oh I see, there are 2 places where same host is used, should be 1, I will 
fix that, thank you



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-02-07 Thread via GitHub


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


##
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 
 --index ");
+return;
+}
+
+final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+env.enableCheckpointing(5000);
+
+DataStream> source =
+env.fromSequence(0, parameterTool.getInt("numRecords") - 1)
+.flatMap(
+new FlatMapFunction>() {
+@Override
+public void flatMap(
+Long value, 
Collector> 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 httpHosts = new ArrayList<>();
+httpHosts.add(new HttpHost("127.0.0.1", 9200, "http"));

Review Comment:
   That's just an example of using AsyncSync, we used to have them 
`https://github.com/apache/flink-connector-opensearch/tree/main/flink-connector-opensearch-e2e-tests/src/main/java/org/apache/flink/streaming/tests`
   



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[GitHub] [flink-connector-opensearch] reta commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink

2023-01-03 Thread GitBox


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


##
flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkITCase.java:
##
@@ -146,29 +146,30 @@ private void runTest(
 .setAllowInsecure(true)
 .build();
 
-final StreamExecutionEnvironment env = new LocalStreamEnvironment();
-env.enableCheckpointing(100L);
-if (!allowRestarts) {
-env.setRestartStrategy(RestartStrategies.noRestart());
-}
-DataStream stream = env.fromSequence(1, 5);
+try (final StreamExecutionEnvironment env = new 
LocalStreamEnvironment()) {

Review Comment:
   Not related by `StreamExecutionEnvironment` is `AutoCloseable`, slightly 
changing the test case



##
flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/OpensearchSinkITCase.java:
##
@@ -146,29 +146,30 @@ private void runTest(
 .setAllowInsecure(true)
 .build();
 
-final StreamExecutionEnvironment env = new LocalStreamEnvironment();
-env.enableCheckpointing(100L);
-if (!allowRestarts) {
-env.setRestartStrategy(RestartStrategies.noRestart());
-}
-DataStream stream = env.fromSequence(1, 5);
+try (final StreamExecutionEnvironment env = new 
LocalStreamEnvironment()) {

Review Comment:
   Not related but `StreamExecutionEnvironment` is `AutoCloseable`, slightly 
changing the test case



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org