Status
...
Page properties | ||
---|---|---|
Document the state by adding a label to the FLIP page with one of "discussion", "accepted", "released", "rejected".
|
...
...
|
Please keep the discussion on the mailing list rather
...
JIRA: here (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)
Released: <Flink Version>
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
...
Apache Flink has a rich connector ecosystem that can persist data in various destinations. Flink natively supports Apache Kafka, Amazon Kinesis Data Streams, Elasticsearch, HBase, and many more destinations. Additional connectors are maintained in Apache Bahir or directly on GitHub. The basic functionality of these sinks is quite similar. They batch events according to user defined buffering hints, sign requests and send them to the respective endpoint, retry unsuccessful or throttled requests, and participate in checkpointing. They primarily only just differ in the way they interface with the destination. Yet, all the above-mentioned sinks are developed and maintained independently.
...
There are two user-facing aspects of the generic sink. First, the interface an abstract class that is used to implement a new sink for a concrete destination. Second, the interface that is used by end-users, who want to leverage an existing sink to persist events in a destination. Appendix A contains a simplified sample implementation for a Kinesis Data Stream sink.
The async sink is based on FLIP-143 and FLIP-177. It is based on the following generic types to be extensible and remain agnostic to the destination.
...
Code Block | ||||
---|---|---|---|---|
| ||||
public abstract class AsyncSinkWriter<InputT, RequestEntryT extends Serializable> implements SinkWriter<InputT, Collection<CompletableFuture<?>>Void, Collection<RequestEntryT>> { /** * This method specifies how to persist buffered request entries into the * sinkdestination. It is implemented when support for a new destination is * implementedadded. * <p> * The method is invoked with a set of request entries according to the * buffering hints (and the valid limits of the destination). The logic then * needs to create and execute the request against the destination (ideally * by batching together multiple request entries to increase efficiency). * The logic also needs to identify individual request entries that were not * persisted successfully and resubmit them using the {@code * requeueFailedRequestEntryrequestResult} method. * <p> * The method returns a future During checkpointing, the sink needs to ensure that indicates,there once completed, that allare no * requestoutstanding entries that have been passed to the method on invocation have * either been successfully persisted in the destination or have beenin-flight requests. * * @param requestEntries a set of request entries that should be sent to the * re-queued. * <p> * During checkpointing, the sink needs to ensure that there are nodestination * @param outstandingrequestResult in-flight requests.a Ie,ResultFuture that allneeds to futuresbe returnedcompleted byonce thisall * method are completed. * * @param requestEntries a set ofrequest requestsentries that shouldhave bebeen sentpassed to the APImethod * endpoint on invocation have either been *successfully @return a future that* completes when all request entries have been * successfully persisted to the API or were re-queued persisted in */ the destination or have protectedbeen abstract * CompletableFuture<?> submitRequestEntries(List<RequestEntryT> requestEntries); /** * The ElementConverter provides a mapping between for the elements of are-queued */ stream to request entriesprotected thatabstract canvoid be sent to the destination.submitRequestEntries(List<RequestEntryT> requestEntries, ResultFuture<RequestEntryT> requestResult); * <p> * The resulting request * entry is buffered by the AsyncSinkWriter and sent * to the destination when * the {@code submitRequestEntries} method is... } |
Internally, the AsyncSinkWriter
buffers RequestEntryT
s and invokes the submitRequestEntries
method with a set of RequestEntryT
s according to user specified buffering hints. The AsyncSinkWriter
also tracks in-flight requests, ie, calls to the API that have been sent but not completed. During a commit, the sink enforces that all in-flight requests have completed and currently buffered RequestEntryT
s are persisted in the application state snapshot.
Code Block | ||||
---|---|---|---|---|
| ||||
/** * invoked. */ The ElementConverter provides a mapping between for the elements of a private final* ElementConverter<InputT,stream RequestEntryT> elementConverter; ... } |
Internally, the AsyncSinkWriter
buffers RequestEntryT
s and invokes the submitRequestEntries
method with a set of RequestEntryT
s according to user specified buffering hints. The AsyncSinkWriter
also tracks in-flight requests, ie, calls to the API that have been sent but not completed. During a commit, the sink enforces that all in-flight requests have completed and currently buffered RequestEntryT
s are persisted in the application state snapshot.
Code Block | ||||
---|---|---|---|---|
| ||||
to request entries that can be sent to the destination. * <p> * The resulting request entry is buffered by the AsyncSinkWriter and sent * to the destination when the {@code submitRequestEntries} method is * invoked. */ private final ElementConverter<InputT, RequestEntryT> elementConverter; /** * Buffer to hold request entries that should be persisted into the * destination. * <p> * A request entry contain all relevant details to make a call to the * destination. Eg, for Kinesis Data Streams a request entry contains the * payload and partition key. * <p> * It seems more natural to buffer InputT, ie, the events that should be * persisted, rather than RequestEntryT. However, in practice, the response * of a failed request call can make it very hard, if not impossible, to * reconstruct the original event. It is much easier, to just construct a * new (retry) request entry from the response and add that back to the * queue for later retry. */ private final Deque<RequestEntryT> bufferedRequestsbufferedRequestEntries = new ConcurrentLinkedDeque<>ArrayDeque<>(); /** * Tracks all pending async calls that have been executed since the last * checkpoint. Calls that already completed (successfully or unsuccessfully) are * are automatically removed fromdecrementing the queuecounter. Any request entry that was not * successfully persisted needneeds to be handled and retried by the logic in * {@code submitRequestsToApi}. * <p> * There is a limit on the number of concurrent (async) requests that can be * handled by the client library. This limit mustis be checkedenforced by checking the * size of this queue before issuninissuing new requests. * <p> * To complete a checkpoint, we need to make sure that no requests are in * flight, as they may fail, which could then lead to data loss. */ private int inFlightRequestsCount; @Override Queue<CompletableFuture<?>> inFlightRequests =public newvoid ConcurrentLinkedQueue<>(); /**write(InputT element, Context context) throws IOException, InterruptedException { * Signals if enough RequestEntryTs // blocks if too many elements have been buffered according to the user * specified buffering hints to make a request against the destination. This * functionality will be added to the sink interface by means of an * additional FLIP. * * @return a future that will be completed once there is are record available * to make a request against the destinationwhile (bufferedRequestEntries.size() >= MAX_BUFFERED_REQUESTS_ENTRIES) { mailboxExecutor.yield(); } bufferedRequestEntries.add(elementConverter.apply(element, context)); // blocks if too many async requests are in flight flush(); } /** * Persists buffered RequestsEntries into the destination by invoking {@code */ submitRequestEntries} with batches according publicto CompletableFuture<Void>the isAvailable()user {specified * buffering .hints.. } * @Override * The method publicblocks voidif write(InputT element, Context context) throws IOException {too many async requests are in flight. */ bufferedRequests.offerLast(elementConverter.apply(element, context)); } private void flush() throws InterruptedException { /** * A request or single request entries of a request may fail, eg, because of * network issues or service side throttling. All request entries thatwhile (bufferedRequestEntries.size() >= MAX_BATCH_SIZE) { // create a batch of request entries that should be persisted in the destination * failed with transient failures need to beArrayList<RequestEntryT> re-queuedbatch with= this method sonew ArrayList<>(MAX_BATCH_SIZE); * that aren't lost and can be retriedwhile later. * <p>(batch.size() <= MAX_BATCH_SIZE && !bufferedRequestEntries.isEmpty()) { * Request entries that are causing the same error in a reproducibletry manner,{ * eg, ill-formed request entries, must not be re-queued but the error needs batch.add(bufferedRequestEntries.remove()); * to be handled in the logic of {@code submitRequestEntries}. Otherwise} catch (NoSuchElementException e) { * these request entries will be retried indefinitely, always causing the // if there are *not same error. */ enough elements, just create a smaller batch protected void requeueFailedRequestEntry(RequestEntryT requestEntry) { bufferedRequests.offerFirst(requestEntry) break; } /** * In-flight} requests may fail, but they will be retried if the sink is} * still healthy. ResultFuture<RequestEntryT> *requestResult <p>= * To not lose any requests, there cannot be any outstanding in-flight *failedRequestEntries requests when a commit is initialized. To this end, all in-flight -> mailboxExecutor.execute( * requests need to be completed as part of the pre commit. () */-> completeRequest(failedRequestEntries), @Override public List<Collection<CompletableFuture<?>>> prepareCommit(boolean flush) throws IOException { // reuse current inFlightRequests as commitable and create an empty queue "Mark in-flight request as completed and requeue %d request entries", // to avoid copy and clearing List<Collection<CompletableFuture<?>>> committable = CollectionsfailedRequestEntries.singletonListsize(inFlightRequests)); // all in-flight requests arewhile handled(inFlightRequestsCount by the AsyncSinkCommiter and new >= MAX_IN_FLIGHT_REQUESTS) { // elements cannot be added to the queue during a commit, so it's save to mailboxExecutor.yield(); } // create a new queueinFlightRequestsCount++; inFlightRequests = new ConcurrentLinkedQueue<>submitRequestEntries(batch, requestResult); return committable;} } /** * AllMarks an in-flight requestsrequest haveas been completed, butand there may still beprepends failed requestEntries back to the * request entries in the internalinternal requestEntry buffer thatfor arelater yetretry. to be sent to the* * endpoint.@param ThesefailedRequestEntries requestrequestEntries entriesthat areneed storedto in the snapshot state so thatbe retried */ they don't get lostprivate in case of a failure/restart of the application. void completeRequest(Collection<RequestEntryT> failedRequestEntries) { inFlightRequestsCount--; */ // By @Override just iterating through failedRequestEntries, publicit List<Collection<RequestEntryT>>reverses snapshotState()the throwsorder IOExceptionof {the return// Collections.singletonList(bufferedRequests); } |
Limitations
...
failedRequestEntries. It doesn't make a difference for kinesis:putRecords, as the api
// does not make any order guarantees, but may cause avoidable reorderings for other
// destinations.
failedRequestEntries.forEach(bufferedRequestEntries::addFirst);
}
/**
* In flight requests will be retried if the sink is still healthy. But if in-flight requests
* fail after a checkpoint has been triggered and Flink needs to recover from the checkpoint,
* the (failed) in-flight requests are gone and cannot be retried. Hence, there cannot be any
* outstanding in-flight requests when a commit is initialized.
*
* <p>To this end, all in-flight requests need to completed before proceeding with the commit.
*/
@Override
public List<Void> prepareCommit(boolean flush) throws IOException, InterruptedException {
if (flush) {
flush();
}
// wait until all in-flight requests completed
while (inFlightRequestsCount > 0) {
mailboxExecutor.yield();
}
return Collections.emptyList();
}
/**
* All in-flight requests have been completed, but there may still be
* request entries in the internal buffer that are yet to be sent to the
* endpoint. These request entries are stored in the snapshot state so that
* they don't get lost in case of a failure/restart of the application.
*/
@Override
public List<Collection<RequestEntryT>> snapshotState() throws IOException {
return Collections.singletonList(bufferedRequestEntries);
}
|
Limitations
- The sink is designed for destinations that provide an async client. Destinations that cannot ingest events in an async fashion cannot be supported by the sink.
The sink usually persist InputTs in the order they are added to the sink, but reorderings may occur, eg, when RequestEntryTs need to be retried.
- We are not focusing on support for exactly-once semantics beyond simple upsert capable and idempotent destinations at this point.
Appendix A – Simplified example implementation of a sink for Kinesis Data Streams
Example ElementConverter for Kinesis Data Streams
This is the functionality that needs to be implemented by end users of the sink. It specifies how an element of a DataStream is mapped to a PutRecordsRequestEntry
that can be submitted to the Kinesis Data Streams API.
Code Block | ||||
---|---|---|---|---|
| ||||
new ElementConverter<InputT, PutRecordsRequestEntry>() {
@Override
public PutRecordsRequestEntry apply(InputT element, SinkWriter.Context context) {
return PutRecordsRequestEntry
.builder()
.data(SdkBytes.fromUtf8String(element.toString()))
.partitionKey(String.valueOf(element.hashCode()))
.build();
}
} |
Simplified AsyncSinkWriter for Kinesis Data Streams
This is a simplified sample implementation of the AsyncSinkWriter
for Kinesis Data Streams. Given a set of buffered PutRecordRequestEntries
, it creates and submits a batch request against the Kinesis Data Stream API using the KinesisAsyncClient
. The response of the API call is then checked for events that were not persisted successfully (eg, because of throttling or network failures) and those events are added back to the internal buffer of the AsyncSinkWriter
.
Code Block | ||||
---|---|---|---|---|
| ||||
private class AmazonKinesisDataStreamWriter extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> {
@Override
protected void submitRequestEntries(List<PutRecordsRequestEntry> requestEntries, ResultFuture<PutRecordsRequestEntry> requestResult) {
// create a batch request
PutRecordsRequest batchRequest = PutRecordsRequest
.builder()
.records(requestEntries)
.streamName(streamName)
.build();
// call api with batch request
CompletableFuture<PutRecordsResponse> future = client.putRecords(batchRequest);
|
Appendix A – Simplified example implementation of a sink for Kinesis Data Streams
Example ElementConverter for Kinesis Data Streams
This is the functionality that needs to be implemented by end users of the sink. It specifies how an element of a DataStream is mapped to a PutRecordsRequestEntry
that can be submitted to the Kinesis Data Streams API.
Code Block | ||||
---|---|---|---|---|
| ||||
new ElementConverter<InputT, PutRecordsRequestEntry>() { @Override public PutRecordsRequestEntry apply(InputT element, SinkWriter.Context context) { return PutRecordsRequestEntry // re-queue elements of failed .builder()requests .data(SdkBytes.fromUtf8String(element.toString())) future.whenComplete((response, err) -> { if .partitionKey(String.valueOf(element.hashCoderesponse.failedRecordCount())) > 0) { .build(); } } |
Simplified AsyncSinkWriter for Kinesis Data Streams
This is a simplified sample implementation of the AsyncSinkWriter
for Kinesis Data Streams. Given a set of buffered PutRecordRequestEntries
, it creates and submits a batch request against the Kinesis Data Stream API using the KinesisAsyncClient
. The response of the API call is then checked for events that were not persisted successfully (eg, because of throttling or network failures) and those events are added back to the internal buffer of the AsyncSinkWriter
.
Code Block | ||||
---|---|---|---|---|
| ||||
public class AmazonKinesisDataStreamSink<InputT> extends AsyncSink<InputT, PutRecordsRequestEntry> { @Override protected CompletableFuture<?> submitRequestEntries(List<PutRecordsRequestEntry> requestEntries) { ArrayList<PutRecordsRequestEntry> failedRequestEntries = new ArrayList<>(response.failedRecordCount()); List<PutRecordsResultEntry> records = response.records(); // create a batch request for PutRecordsRequest(int batchRequesti = PutRecordsRequest 0; i < records.size(); i++) { .builder() if .records(requestEntries) (records.get(i).errorCode() != null) { .streamName(streamName) failedRequestEntries.buildadd(requestEntries.get(i)); // call api with batch request CompletableFuture<PutRecordsResponse> future = client.putRecords(batchRequest); } // re-queue elements of failed requests } CompletableFuture<PutRecordsResponse> handleResponse = future .whenComplete((response, err) -> { requestResult.complete(failedRequestEntries); if (response.failedRecordCount() > 0) { } else { List<PutRecordsResultEntry> records = response.records(); requestResult.complete(Collections.emptyList()); for (int i} = 0; i < records.size(); i++) { //TODO: handle errors of the entire request... if (records.get(i).errorCode() != null}); { requeueFailedRequest(requestEntries.get(i));} ... } |
Rejected Alternatives
The sink needs a way to build back pressure, eg, if the throughput limit of the destination is exceeded. Initially we were planning to adopt the isAvailable pattern from the source interface. But the benefits are too vague at this point and it would require substantial changes to the sink API. We'll hence start with a blocking implementation of the write
function and see how far we get.
Code Block | ||||
---|---|---|---|---|
| ||||
/** * Signals if enough RequestEntryTs have been buffered according to the }user * specified buffering hints to make a request against the destination. This } * functionality will be added to the sink interface by means of }an //TODO: handle errors of the entire request...* additional FLIP. * * @return a future that will be completed }); once a request against the // return* futuredestination tocan trackbe completionmade of async request */ public CompletableFuture<Void> return handleResponse;isAvailable() { } ... } |