Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

...

Page properties

Document the state by adding a label to the FLIP page with one of "discussion", "accepted", "released", "rejected".

Discussion threadhttps:

...

...


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki

...

JIRA: tbd

Released: tbd

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast). 

...

There are two user-facing aspects of the generic sink. First, 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
languagejava
titleAsyncSinkWriter
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
     * destination. It is implemented when support for a new destination is
     * added.
     * <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>
     * During checkpointing, Thethe methodsink returnsneeds ato futureensure 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
     * outstanding in-flight requests. Ie, that all futures returned by this@param requestResult  a ResultFuture that needs to be completed once all
     * method      are completed.
     *
     * @param requestEntries a set ofrequest requestsentries that shouldhave bebeen sentpassed to the APImethod
     *                       on invocation have either been endpointsuccessfully
     * @return a future that completes when all request entries                     persisted in the destination or have been
     * successfully persisted to the API or were                       re-queued
     */
    protected abstract CompletableFuture<?>void submitRequestEntries(List<RequestEntryT> requestEntries, ResultFuture<RequestEntryT> requestResult);

    ...
}

Internally, the AsyncSinkWriter buffers RequestEntryTs and invokes the submitRequestEntries method with a set of RequestEntryTs 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 RequestEntryTs are persisted in the application state snapshot.

Code Block
languagejava
titleAsyncSinkWriter Internals
    /**
     * The ElementConverter provides a mapping between for the elements of a
     * stream 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> bufferedRequestEntries = 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 removeddecrementing from 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 is enforced by checking the
     * size of this queue before issuing 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 Queue<CompletableFuture<?>> inFlightRequests = new ConcurrentLinkedQueue<>(...);

int inFlightRequestsCount;


    @Override
    public void write(InputT element, Context context) throws IOException, InterruptedException {
        
// blocks if too many elements have  bufferedRequestEntries.putLast(elementConverter.apply(element, context));

		
been buffered
    }


    /**
     * The entire request may fail or single request entries that are part of
while (bufferedRequestEntries.size() >= MAX_BUFFERED_REQUESTS_ENTRIES) {
            mailboxExecutor.yield();
      * the request}

 may not be persisted successfully, eg, because of network bufferedRequestEntries.add(elementConverter.apply(element, context));

     * issues or service side throttling. All request entries that failed with// blocks if too many async requests are in flight
     * transient failures needflush();
 to be re-queued with this method so that aren't }


    /**
     * lost and can be retried later. Persists buffered RequestsEntries into the destination by invoking {@code
     * <p>
     * Request entries that are causing the same error in a reproducible manner,submitRequestEntries} with batches according to the user specified
     * buffering hints.
     *
     * eg, ill-formed request entries, must not be re-queued but the error needsThe method blocks if too many async requests are in flight.
     */
 to be handled in the logic of {@code submitRequestEntries}. Otherwiseprivate void flush() throws InterruptedException {
     * these request entries will be retried indefinitely, always causing the
while (bufferedRequestEntries.size() >= MAX_BATCH_SIZE) {

        * same error.
  // create a */
batch of request entries protectedthat voidshould requeueFailedRequestEntry(RequestEntryT requestEntry) {be persisted in the destination
        bufferedRequestEntries.offerFirst(requestEntry);
    }


ArrayList<RequestEntryT> batch =  /**new ArrayList<>(MAX_BATCH_SIZE);

     * In-flight requests may fail, but they will be retried if the sink iswhile (batch.size() <= MAX_BATCH_SIZE && !bufferedRequestEntries.isEmpty()) {
     * still healthy.
     * <p>
   try {
 * To not lose any requests, there cannot be any outstanding in-flight
     * requests when a commit is initialized. To this end, all in-flight
 batch.add(bufferedRequestEntries.remove());
             * requests need to} becatch completed(NoSuchElementException ase) part{
 of the pre commit.
     */
    @Override
    public List<Collection<CompletableFuture<?>>> prepareCommit(boolean flush) throws IOException {

        // reuse current inFlightRequests as commitable and create an empty queue 
 // if there are not enough elements, just create a smaller batch
                   // tobreak;
 avoid copy and clearing
        List<Collection<CompletableFuture<?>>>  committable = Collections.singletonList(inFlightRequests);

 }
        // all in-flight requests are}

 handled by the AsyncSinkCommiter and new 
     ResultFuture<RequestEntryT> requestResult =
 // elements cannot be added to the queue during a commit, so it's save to 
    failedRequestEntries -> mailboxExecutor.execute(
  // create a new queue
        inFlightRequests = new ConcurrentLinkedQueue<>();

        return committable;
  ()  }


-> completeRequest(failedRequestEntries),
     /**
     * All in-flight requests have been completed, but there may still be
     * request entries"Mark in-flight therequest internalas buffercompleted thatand arerequeue yet%d to be sent to the
request entries",
          * 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.
     */ failedRequestEntries.size());

            while (inFlightRequestsCount >= MAX_IN_FLIGHT_REQUESTS) {
    @Override
    public List<Collection<RequestEntryT>> snapshotState() throws IOException {
      mailboxExecutor.yield();
            }

            inFlightRequestsCount++;
          return  Collections.singletonList(bufferedRequestEntries);submitRequestEntries(batch, requestResult);
        }
    }

Limitations

...

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.

...


    /**
     * Marks an in-flight request as completed and prepends failed requestEntries back to the
     * internal requestEntry buffer for later retry.
     *
     * @param failedRequestEntries requestEntries that need to be retried
     */
    private void completeRequest(Collection<RequestEntryT> failedRequestEntries) {
        inFlightRequestsCount--;

        // By just iterating through failedRequestEntries, it reverses the order of the
        // 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
languagejava
titleElementConverter for Kinesis Data Streams

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

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
languagejava
titleElementConverter for Kinesis Data StreamsAmazonKinesisDataStreamWriter
private class AmazonKinesisDataStreamWriter extends AsyncSinkWriter<InputT
new ElementConverter<InputT, PutRecordsRequestEntry>() {

    @Override
    publicprotected PutRecordsRequestEntryvoid applysubmitRequestEntries(InputTList<PutRecordsRequestEntry> elementrequestEntries, SinkWriter.ContextResultFuture<PutRecordsRequestEntry> contextrequestResult) {
        return PutRecordsRequestEntry
        // create a      .builder()batch request
        PutRecordsRequest batchRequest       .data(SdkBytes.fromUtf8String(element.toString()))= PutRecordsRequest
                .partitionKey(String.valueOf(element.hashCodebuilder()))
                .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
languagejava
titleAmazonKinesisDataStreamWriter
private class AmazonKinesisDataStreamWriter extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> {

    @Override
    protected CompletableFuture<?> submitRequestEntries(List<PutRecordsRequestEntry> requestEntries) {.records(requestEntries)
                .streamName(streamName)
                .build();

        // call createapi awith batch request
        PutRecordsRequestCompletableFuture<PutRecordsResponse> batchRequestfuture = PutRecordsRequest
client.putRecords(batchRequest);

        // re-queue elements of failed requests
        future.builder()
whenComplete((response, err) -> {
                if (response.recordsfailedRecordCount(requestEntries)
 >       0) {
        .streamName(streamName)
            ArrayList<PutRecordsRequestEntry> failedRequestEntries = new ArrayList<>(response.buildfailedRecordCount());

        // call api with batch request
        CompletableFuture<PutRecordsResponse>List<PutRecordsResultEntry> futurerecords = clientresponse.putRecordsrecords(batchRequest);

        // re-queue elements of failed requests

            CompletableFuture<PutRecordsResponse> handleResponse = future
     for (int i = 0; i < records.whenComplete((response, err) ->size(); i++) {
                        if (response.failedRecordCount(records.get(i).errorCode() >!= 0null) {
                         List<PutRecordsResultEntry> records = response.records();

   failedRequestEntries.add(requestEntries.get(i));
                 for (int i = 0; i < records.size(); i++) { }
                        if (records.get(i).errorCode() != null) {}
    
                            requeueFailedRequest(requestEntries.get(i)requestResult.complete(failedRequestEntries);
                        }} else {
                    }requestResult.complete(Collections.emptyList());
                }

                //TODO: handle errors of the entire request...
            });

    }

    // return future to track completion of async request
        return handleResponse;
    }

    ...
}

...

...
}


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
languagejava
titleisAvailable pattern
    /**
     * 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
     * additional FLIP.
     *
     * @return a future that will be completed once a request against the
     * destination can be made
     */
    public CompletableFuture<Void> isAvailable() {
        ...
    }

...