Skip to content

[FLINK-30488] OpenSearch implementation of Async Sink #5

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Draft
wants to merge 8 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
79 changes: 79 additions & 0 deletions docs/content/docs/connectors/datastream/opensearch.md
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,85 @@ This will buffer elements before sending them in bulk to the cluster. The `BulkP
executes bulk requests one at a time, i.e. there will be no two concurrent
flushes of the buffered actions in progress.

## Opensearch AsyncSink

The example below shows how to configure and create a AsyncSink (see please [FLIP-171](https://cwiki.apache.org/confluence/display/FLINK/FLIP-171%3A+Async+Sink)):

{{< tabs "b1732edd-4218-470e-adad-b1ebb4021a1b" >}}
{{< tab "Java" >}}

```java
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.connector.opensearch.sink.OpensearchAsyncSink;
import org.apache.flink.streaming.api.datastream.DataStream;

import org.apache.http.HttpHost;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.client.Requests;

import java.util.HashMap;
import java.util.Map;

DataStream<String> input = ...;

input.sinkTo(
OpensearchAsyncSink.<String>builder()
.setHosts(new HttpHost("localhost", 9200, "http"))
.setElementConverter((element: String, context: SinkWriter.Context) -> createIndexRequest(element))
.build());


private static IndexRequest createIndexRequest(String element) {
Map<String, Object> json = new HashMap<>();
json.put("data", element);

return Requests.indexRequest()
.index("my-index")
.id(element)
.source(json);
}
```
{{< /tab >}}
{{< tab "Scala" >}}
```scala
import org.apache.flink.api.connector.sink.SinkWriter
import org.apache.flink.connector.opensearch.sink.{OpensearchAsyncSink, RequestIndexer}
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.http.HttpHost
import org.opensearch.action.index.IndexRequest
import org.opensearch.client.Requests

val input: DataStream[String] = ...

input.sinkTo(
OpensearchAsyncSink[String]
.builder()
.setMaxBatchSize(1) // Instructs the AsyncSink to emit after every element, otherwise they would be buffered
.setHosts(new HttpHost("127.0.0.1", 9200, "http"))
.setElementConverter((element: String, context: SinkWriter.Context) => createIndexRequest(element))
.build())

def createIndexRequest(element: (String)): IndexRequest = {

val json = Map(
"data" -> element.asInstanceOf[AnyRef]
)

Requests.indexRequest.index("my-index").source(mapAsJavaMap(json))
}
```

{{< /tab >}}
{{< /tabs >}}

Note that the example only demonstrates performing a single index
request for each incoming element. Generally, the `ElementConverter`
can be used to produce the requests of different types (ex.,
`DeleteRequest`, `UpdateRequest`, etc.).

Internally, each parallel instance of the Flink Opensearch AsyncSink uses
a `RestHighLevelClient::bulkAsync` to send action requests to the cluster.

### Opensearch Sinks and Fault Tolerance

With Flink’s checkpointing enabled, the Flink Opensearch Sink guarantees
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.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;

/** End to end test for OpensearchAsyncSink. */
public class OpensearchAsyncSinkExample {

public static void main(String[] args) throws Exception {

final ParameterTool parameterTool = ParameterTool.fromArgs(args);

if (parameterTool.getNumberOfParameters() < 2) {
System.out.println(
"Missing parameters!\n" + "Usage: --numRecords <numRecords> --index <index>");
return;
}

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.enableCheckpointing(5000);

DataStream<Tuple2<String, String>> source =
env.fromSequence(0, parameterTool.getInt("numRecords") - 1)
.flatMap(
new FlatMapFunction<Long, Tuple2<String, String>>() {
@Override
public void flatMap(
Long value, Collector<Tuple2<String, String>> out) {
final String key = String.valueOf(value);
final String message = "message #" + value;
out.collect(Tuple2.of(key, message + "update #1"));
out.collect(Tuple2.of(key, message + "update #2"));
}
});

OpensearchAsyncSinkBuilder<Tuple2<String, String>> osSinkBuilder =
OpensearchAsyncSink.<Tuple2<String, String>>builder()
.setHosts(new HttpHost("localhost", 9200, "http"))
.setElementConverter(
(element, context) ->
new IndexRequest("my-index")
.id(element.f0.toString())
.source(element.f1));
Comment on lines +67 to +69
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm.. Since we have to implement a DocSerdeRequest, should we consider exposing this in the interface instead of OpenSearch classes? This might be helpful in the event OpenSearch's interface changes.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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


source.sinkTo(osSinkBuilder.build());

env.execute("Opensearch end to end async sink test example");
}
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,15 @@
org.apache.flink.connector.opensearch.sink.OpensearchAsyncSinkITCase does not satisfy: only one of the following predicates match:\
* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
org.apache.flink.connector.opensearch.sink.OpensearchAsyncWriterITCase does not satisfy: only one of the following predicates match:\
* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
org.apache.flink.connector.opensearch.sink.OpensearchSinkITCase does not satisfy: only one of the following predicates match:\
* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
Expand Down
8 changes: 8 additions & 0 deletions flink-connector-opensearch/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,14 @@ under the License.
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<!-- Opensearch table descriptor testing -->
<dependency>
<groupId>org.apache.flink</groupId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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.
*/
@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 <T> DocSerdeRequest from(DocWriteRequest<T> request) {
return new DocSerdeRequest(request);
}

static DocSerdeRequest readFrom(long requestSize, DataInputStream in) throws IOException {
try (final StreamInput stream = new InputStreamStreamInput(in, requestSize)) {
return new DocSerdeRequest(readDocumentRequest(stream));
}
}

void writeTo(DataOutputStream out) throws IOException {
try (BytesStreamOutput stream = new BytesStreamOutput()) {
writeDocumentRequest(stream, request);
out.write(BytesReference.toBytes(stream.bytes()));
}
}

/** Read a document write (index/delete/update) request. */
private static DocWriteRequest<?> readDocumentRequest(StreamInput in) throws IOException {
byte type = in.readByte();
DocWriteRequest<?> docWriteRequest;
if (type == 0) {
docWriteRequest = new IndexRequest(in);
} else if (type == 1) {
docWriteRequest = new DeleteRequest(in);
} else if (type == 2) {
docWriteRequest = new UpdateRequest(in);
} else {
throw new IllegalStateException("Invalid request type [" + type + " ]");
}
return docWriteRequest;
Comment on lines +73 to +85
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These methods are untested. Should we add unit tests for them?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They are tested in scope if integration test, OpensearchAsyncSinkITCase, both reading and writing side.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}

/** Write a document write (index/delete/update) request. */
private static void writeDocumentRequest(StreamOutput out, DocWriteRequest<?> request)
throws IOException {
if (request instanceof IndexRequest) {
out.writeByte((byte) 0);
((IndexRequest) request).writeTo(out);
} else if (request instanceof DeleteRequest) {
out.writeByte((byte) 1);
((DeleteRequest) request).writeTo(out);
} else if (request instanceof UpdateRequest) {
out.writeByte((byte) 2);
((UpdateRequest) request).writeTo(out);
} else {
throw new IllegalStateException(
"Invalid request [" + request.getClass().getSimpleName() + " ]");
}
}
}
Loading