[GitHub] [flink-connector-opensearch] dannycranmer commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink
dannycranmer commented on code in PR #5: URL: https://github.com/apache/flink-connector-opensearch/pull/5#discussion_r1115860671 ## flink-connector-opensearch/src/test/java/org/apache/flink/connector/opensearch/sink/DocSerdeRequestTest.java: ## @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.assertj.core.api.recursive.comparison.RecursiveComparisonConfiguration; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; + +class DocSerdeRequestTest { +@ParameterizedTest +@MethodSource("requests") +void serde(DocWriteRequest request) throws IOException { +final DocSerdeRequest serialized = DocSerdeRequest.from(request); + +try (final ByteArrayOutputStream bytes = new ByteArrayOutputStream()) { +try (final DataOutputStream out = new DataOutputStream(bytes)) { +serialized.writeTo(out); +} + +try (final DataInputStream in = +new DataInputStream(new ByteArrayInputStream(bytes.toByteArray( { +final DocSerdeRequest deserialized = DocSerdeRequest.readFrom(Byte.MAX_VALUE, in); +assertThat(deserialized.getRequest()) +.usingRecursiveComparison( +RecursiveComparisonConfiguration.builder() +/* ignoring 'type', it is deprecated but backfilled for 1.x compatibility */ +.withIgnoredFields("type", "doc.type") +.build()) +.isEqualTo(serialized.getRequest()); +} +} +} + +@Test +@SuppressWarnings("unchecked") +void unsupportedRequestType() throws IOException { +final DocSerdeRequest serialized = DocSerdeRequest.from(mock(DocWriteRequest.class)); Review Comment: Mockito is [banned](https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#avoid-mockito---use-reusable-test-implementations). Since you only have one usage here can we remove 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] dannycranmer commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink
dannycranmer commented on code in PR #5: URL: https://github.com/apache/flink-connector-opensearch/pull/5#discussion_r1104151987 ## 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
[GitHub] [flink-connector-opensearch] dannycranmer commented on a diff in pull request #5: [FLINK-30488] OpenSearch implementation of Async Sink
dannycranmer commented on code in PR #5: URL: https://github.com/apache/flink-connector-opensearch/pull/5#discussion_r1102989699 ## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/DocSerdeRequest.java: ## @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.io.stream.InputStreamStreamInput; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.Serializable; + +/** + * Wrapper class around {@link DocWriteRequest} since it does not implement {@link Serializable}, + * required by AsyncSink scaffolding. + * + * @param type of the write request + */ +@PublicEvolving +public class DocSerdeRequest implements Serializable { Review Comment: Ii think the class level generics are redundant here. We are using `` throughout. Consider changing `private final DocWriteRequest request;` to `private final DocWriteRequest request;` and removing class generics. This makes the Sink interface a bit messy `extends AsyncSinkBase>` ## flink-connector-opensearch/src/main/java/org/apache/flink/connector/opensearch/sink/OpensearchAsyncSinkBuilder.java: ## @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.opensearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; +import org.apache.flink.connector.base.sink.writer.ElementConverter; + +import org.apache.http.HttpHost; +import org.opensearch.action.DocWriteRequest; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Builder to construct an Opensearch compatible {@link OpensearchAsyncSink}. + * + * 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 all