-
Notifications
You must be signed in to change notification settings - Fork 28
[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
base: main
Are you sure you want to change the base?
Changes from all commits
cfa50af
cf0992f
7213a0c
5eb2fa4
418c0e0
a029985
0d1c957
4cd6423
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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)); | ||
|
||
source.sinkTo(osSinkBuilder.build()); | ||
|
||
env.execute("Opensearch end to end async sink test example"); | ||
} | ||
} |
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
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These methods are untested. Should we add unit tests for them? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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() + " ]"); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
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 ofOpenSearch
classes? This might be helpful in the event OpenSearch's interface changes.There was a problem hiding this comment.
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 requiresSerializable
), we should export in the places when it is inevitable but in general we should only operate over OpenSearch APIs.There was a problem hiding this comment.
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