[FLINK-24041][connectors] First draft of API, uses FLIP-143: Unified Sink API, Increase robustness for flush, Add license header, Update JavaDoc comments, added unit and integration tests for the generic sink implementation. Corrected an inconsistency that led to prepareCommit() not fully flushing the entire contents of the buffer at checkpoint time, removed catching of RuntimeExceptions in concrete implementations, added @PublicEvolving to classes, rearranged cosmetics of classes, corrected spelling errors, changed tests to reflect removal of RuntimeException, added checks on the parameters passed to AsyncSinkWriter at construction time, simplified batch creation part of Sink, amended tests, and test harness now guarantees ordering, added Javadoc for AsyncSinkWriter, removed redundant flushIfFull, removed duplicate assertion, removed redundant overriding in SinkInitContext, refactored static class into anonymous function, cleaned up imports, all test methods now begin with "test", refactored duplication in tests, Fixed spelling error, removed ResultFuture and all references to this construct, Now using Consumer<T> to accept requeue candidates
diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/AsyncSinkBase.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/AsyncSinkBase.java
new file mode 100644
index 0000000..77797f7
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/AsyncSinkBase.java
@@ -0,0 +1,71 @@
+/*
+ * 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.base.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/**
+ * A generic sink for destinations that provide an async client to persist data.
+ *
+ * <p>The design of the sink focuses on extensibility and a broad support of destinations. The core
+ * of the sink is kept generic and free of any connector specific dependencies. The sink is designed
+ * to participate in checkpointing to provide at-least once semantics, but it is limited to
+ * destinations that provide a client that supports async requests.
+ *
+ * <p>Limitations:
+ *
+ * <ul>
+ *   <li>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.
+ *   <li>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.
+ *   <li>We are not considering support for exactly-once semantics at this point.
+ * </ul>
+ */
+@PublicEvolving
+public abstract class AsyncSinkBase<InputT, RequestEntryT extends Serializable>
+        implements Sink<InputT, Void, Collection<RequestEntryT>, Void> {
+
+    @Override
+    public Optional<Committer<Void>> createCommitter() {
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<GlobalCommitter<Void, Void>> createGlobalCommitter() {
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<SimpleVersionedSerializer<Void>> getCommittableSerializer() {
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<SimpleVersionedSerializer<Void>> getGlobalCommittableSerializer() {
+        return Optional.empty();
+    }
+}
diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java
new file mode 100644
index 0000000..1d17d2f
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java
@@ -0,0 +1,238 @@
+/*
+ * 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.base.sink.writer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Consumer;
+
+/**
+ * A generic sink writer that handles the general behaviour of a sink such as batching and flushing,
+ * and allows extenders to implement the logic for persisting individual request elements, with
+ * allowance for retries.
+ *
+ * <p>At least once semantics is supported through {@code prepareCommit} as outstanding requests are
+ * flushed or completed prior to checkpointing.
+ *
+ * <p>Designed to be returned at {@code createWriter} time by an {@code AsyncSinkBase}.
+ *
+ * <p>There are configuration options to customize the buffer size etc.
+ */
+@PublicEvolving
+public abstract class AsyncSinkWriter<InputT, RequestEntryT extends Serializable>
+        implements SinkWriter<InputT, Void, Collection<RequestEntryT>> {
+
+    private final MailboxExecutor mailboxExecutor;
+    private final Sink.ProcessingTimeService timeService;
+
+    private final int maxBatchSize;
+    private final int maxInFlightRequests;
+    private final int maxBufferedRequests;
+
+    /**
+     * 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 ArrayDeque<>();
+
+    /**
+     * Tracks all pending async calls that have been executed since the last checkpoint. Calls that
+     * completed (successfully or unsuccessfully) are automatically decrementing the counter. Any
+     * request entry that was not successfully persisted needs 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 queue size before accepting a new
+     * element into the queue.
+     *
+     * <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;
+
+    /**
+     * 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 requeueFailedRequestEntry} method.
+     *
+     * <p>During checkpointing, the sink needs to ensure that there are no outstanding in-flight
+     * requests.
+     *
+     * @param requestEntries a set of request entries that should be sent to the destination
+     * @param requestResult the {@code accept} method should be called on this Consumer once the
+     *     processing of the {@code requestEntries} are complete. Any entries that encountered
+     *     difficulties in persisting should be re-queued through {@code requestResult} by including
+     *     that element in the collection of {@code RequestEntryT}s passed to the {@code accept}
+     *     method. All other elements are assumed to have been successfully persisted.
+     */
+    protected abstract void submitRequestEntries(
+            List<RequestEntryT> requestEntries, Consumer<Collection<RequestEntryT>> requestResult);
+
+    public AsyncSinkWriter(
+            ElementConverter<InputT, RequestEntryT> elementConverter,
+            Sink.InitContext context,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests) {
+        this.elementConverter = elementConverter;
+        this.mailboxExecutor = context.getMailboxExecutor();
+        this.timeService = context.getProcessingTimeService();
+
+        Preconditions.checkNotNull(elementConverter);
+        Preconditions.checkArgument(maxBatchSize > 0);
+        Preconditions.checkArgument(maxBufferedRequests > 0);
+        Preconditions.checkArgument(maxInFlightRequests > 0);
+        Preconditions.checkArgument(
+                maxBufferedRequests > maxBatchSize,
+                "The maximum number of requests that may be buffered should be strictly"
+                        + " greater than the maximum number of requests per batch.");
+        this.maxBatchSize = maxBatchSize;
+        this.maxInFlightRequests = maxInFlightRequests;
+        this.maxBufferedRequests = maxBufferedRequests;
+    }
+
+    @Override
+    public void write(InputT element, Context context) throws IOException, InterruptedException {
+        while (bufferedRequestEntries.size() >= maxBufferedRequests) {
+            mailboxExecutor.yield();
+        }
+
+        bufferedRequestEntries.add(elementConverter.apply(element, context));
+
+        flushIfFull();
+    }
+
+    private void flushIfFull() throws InterruptedException {
+        while (bufferedRequestEntries.size() >= maxBatchSize) {
+            flush();
+        }
+    }
+
+    /**
+     * Persists buffered RequestsEntries into the destination by invoking {@code
+     * submitRequestEntries} with batches according to the user specified buffering hints.
+     *
+     * <p>The method blocks if too many async requests are in flight.
+     */
+    private void flush() throws InterruptedException {
+        while (inFlightRequestsCount >= maxInFlightRequests) {
+            mailboxExecutor.yield();
+        }
+
+        List<RequestEntryT> batch = new ArrayList<>(maxBatchSize);
+
+        int batchSize = Math.min(maxBatchSize, bufferedRequestEntries.size());
+        for (int i = 0; i < batchSize; i++) {
+            batch.add(bufferedRequestEntries.remove());
+        }
+
+        if (batch.size() == 0) {
+            return;
+        }
+
+        Consumer<Collection<RequestEntryT>> requestResult =
+                failedRequestEntries ->
+                        mailboxExecutor.execute(
+                                () -> completeRequest(failedRequestEntries),
+                                "Mark in-flight request as completed and requeue %d request entries",
+                                failedRequestEntries.size());
+
+        inFlightRequestsCount++;
+        submitRequestEntries(batch, requestResult);
+    }
+
+    /**
+     * 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--;
+        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 InterruptedException {
+        while (inFlightRequestsCount > 0 || bufferedRequestEntries.size() > 0) {
+            mailboxExecutor.yield();
+            if (flush) {
+                flush();
+            }
+        }
+
+        return Collections.emptyList();
+    }
+
+    /**
+     * All in-flight requests that are relevant for the snapshot have been completed, but there may
+     * still be request entries in the internal buffers 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() {
+        return Arrays.asList(bufferedRequestEntries);
+    }
+
+    @Override
+    public void close() {}
+}
diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/ElementConverter.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/ElementConverter.java
new file mode 100644
index 0000000..94334e0
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/ElementConverter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.base.sink.writer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink.SinkWriter;
+
+import java.io.Serializable;
+
+/**
+ * This interface specifies the mapping between elements of a stream to request entries that can be
+ * sent to the destination. The mapping is provided by the end-user of a sink, not the sink creator.
+ *
+ * <p>The request entries contain all relevant information required to create and sent the actual
+ * request. Eg, for Kinesis Data Streams, the request entry includes the payload and the partition
+ * key.
+ */
+@PublicEvolving
+public interface ElementConverter<InputT, RequestEntryT> extends Serializable {
+    RequestEntryT apply(InputT element, SinkWriter.Context context);
+}
diff --git a/flink-connectors/flink-connector-base/src/main/resources/log4j2.properties b/flink-connectors/flink-connector-base/src/main/resources/log4j2.properties
new file mode 100644
index 0000000..32c696e
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/main/resources/log4j2.properties
@@ -0,0 +1,25 @@
+################################################################################
+#  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.
+################################################################################
+
+rootLogger.level = INFO
+rootLogger.appenderRef.console.ref = ConsoleAppender
+
+appender.console.name = ConsoleAppender
+appender.console.type = CONSOLE
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListAsyncSink.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListAsyncSink.java
new file mode 100644
index 0000000..c4d6639
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListAsyncSink.java
@@ -0,0 +1,72 @@
+/*
+ * 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.base.sink;
+
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/** Dummy destination that records write events. */
+public class ArrayListAsyncSink extends AsyncSinkBase<String, Integer> {
+
+    private final int maxBatchSize;
+    private final int maxInFlightRequests;
+    private final int maxBufferedRequests;
+
+    public ArrayListAsyncSink() {
+        this(25, 1, 100);
+    }
+
+    public ArrayListAsyncSink(int maxBatchSize, int maxInFlightRequests, int maxBufferedRequests) {
+        this.maxBatchSize = maxBatchSize;
+        this.maxInFlightRequests = maxInFlightRequests;
+        this.maxBufferedRequests = maxBufferedRequests;
+    }
+
+    @Override
+    public SinkWriter<String, Void, Collection<Integer>> createWriter(
+            InitContext context, List<Collection<Integer>> states) {
+        /* SinkWriter implementing {@code submitRequestEntries} that is used to define the persistence
+         * logic into {@code ArrayListDestination}.
+         */
+        return new AsyncSinkWriter<String, Integer>(
+                (element, x) -> Integer.parseInt(element),
+                context,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests) {
+            @Override
+            protected void submitRequestEntries(
+                    List<Integer> requestEntries, Consumer<Collection<Integer>> requestResult) {
+                ArrayListDestination.putRecords(requestEntries);
+                requestResult.accept(Arrays.asList());
+            }
+        };
+    }
+
+    @Override
+    public Optional<SimpleVersionedSerializer<Collection<Integer>>> getWriterStateSerializer() {
+        return Optional.empty();
+    }
+}
diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListDestination.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListDestination.java
new file mode 100644
index 0000000..5b68da0
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListDestination.java
@@ -0,0 +1,43 @@
+/*
+ * 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.base.sink;
+
+import java.util.List;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.stream.Collectors;
+
+/** Dummy destination where the imaginary concrete sink is writing to. */
+public class ArrayListDestination {
+
+    private static BlockingDeque<Integer> store = new LinkedBlockingDeque<>();
+
+    /**
+     * Returns a list of indices of elements that failed to insert, fails to insert if the integer
+     * value of the {@code newRecord} is greater than 1000.
+     */
+    protected static List<Integer> putRecords(List<Integer> newRecords) {
+        store.addAll(
+                newRecords.stream().filter(record -> record <= 1000).collect(Collectors.toList()));
+        if (newRecords.contains(1_000_000)) {
+            throw new RuntimeException(
+                    "Intentional error on persisting 1_000_000 to ArrayListDestination");
+        }
+        return newRecords.stream().filter(record -> record > 1000).collect(Collectors.toList());
+    }
+}
diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/AsyncSinkBaseITCase.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/AsyncSinkBaseITCase.java
new file mode 100644
index 0000000..63d51e3
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/AsyncSinkBaseITCase.java
@@ -0,0 +1,62 @@
+/*
+ * 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.base.sink;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+/** Integration tests of a baseline generic sink that implements the AsyncSinkBase. */
+public class AsyncSinkBaseITCase {
+
+    final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    @Test
+    public void testWriteTwentyThousandRecordsToGenericSink() throws Exception {
+        env.fromSequence(1, 20000).map(Object::toString).sinkTo(new ArrayListAsyncSink());
+        env.execute("Integration Test: AsyncSinkBaseITCase").getJobExecutionResult();
+    }
+
+    @Test
+    public void testFailuresOnPersistingToDestinationAreCaughtAndRaised() {
+        env.fromSequence(999_999, 1_000_100)
+                .map(Object::toString)
+                .sinkTo(new ArrayListAsyncSink(1, 1, 2));
+        Exception e =
+                assertThrows(
+                        JobExecutionException.class,
+                        () -> env.execute("Integration Test: AsyncSinkBaseITCase"));
+        assertEquals(
+                "Intentional error on persisting 1_000_000 to ArrayListDestination",
+                e.getCause().getCause().getMessage());
+    }
+
+    @Test
+    public void testThatNoIssuesOccurWhenCheckpointingIsEnabled() throws Exception {
+        env.enableCheckpointing(20);
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, Time.milliseconds(200)));
+        env.fromSequence(1, 10_000).map(Object::toString).sinkTo(new ArrayListAsyncSink());
+        env.execute("Integration Test: AsyncSinkBaseITCase");
+    }
+}
diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java
new file mode 100644
index 0000000..378198d
--- /dev/null
+++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java
@@ -0,0 +1,368 @@
+/*
+ * 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.base.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor;
+import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl;
+import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl;
+import org.apache.flink.util.UserCodeClassLoader;
+import org.apache.flink.util.function.RunnableWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Unit Tests the functionality of AsyncSinkWriter without any assumptions of what a concrete
+ * implementation might do.
+ */
+public class AsyncSinkWriterTest {
+
+    private final List<Integer> res = new ArrayList<>();
+    private final SinkInitContext sinkInitContext = new SinkInitContext();
+
+    @Before
+    public void before() {
+        res.clear();
+    }
+
+    @Test
+    public void testNumberOfRecordsIsAMultipleOfBatchSizeResultsInThatNumberOfRecordsBeingWritten()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 10, 1, 100, false);
+        for (int i = 0; i < 80; i++) {
+            sink.write(String.valueOf(i));
+        }
+        assertEquals(80, res.size());
+    }
+
+    @Test
+    public void testThatUnwrittenRecordsInBufferArePersistedWhenSnapshotIsTaken()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 10, 1, 100, false);
+        for (int i = 0; i < 23; i++) {
+            sink.write(String.valueOf(i));
+        }
+        assertEquals(20, res.size());
+        assertEquals(Arrays.asList(20, 21, 22), new ArrayList<>(sink.snapshotState().get(0)));
+    }
+
+    @Test
+    public void testPreparingCommitAtSnapshotTimeEnsuresBufferedRecordsArePersistedToDestination()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 10, 1, 100, false);
+        for (int i = 0; i < 23; i++) {
+            sink.write(String.valueOf(i));
+        }
+        sink.prepareCommit(true);
+        assertEquals(23, res.size());
+    }
+
+    @Test
+    public void testThatSnapshotsAreTakenOfBufferCorrectlyBeforeAndAfterAutomaticFlush()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 1, 100, false);
+
+        sink.write("25");
+        sink.write("55");
+        assertEquals(Arrays.asList(25, 55), new ArrayList<>(sink.snapshotState().get(0)));
+        assertEquals(0, res.size());
+
+        sink.write("75");
+        assertEquals(Arrays.asList(), new ArrayList<>(sink.snapshotState().get(0)));
+        assertEquals(3, res.size());
+    }
+
+    @Test
+    public void testThatSnapshotsAreTakenOfBufferCorrectlyBeforeAndAfterManualFlush()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 1, 100, false);
+        sink.write("25");
+        sink.write("55");
+        sink.write("75");
+        sink.write("95");
+        sink.write("955");
+        assertEquals(Arrays.asList(95, 955), new ArrayList<>(sink.snapshotState().get(0)));
+        sink.prepareCommit(true);
+        assertEquals(Arrays.asList(), new ArrayList<>(sink.snapshotState().get(0)));
+        assertEquals(5, res.size());
+    }
+
+    @Test
+    public void testRuntimeErrorsInSubmitRequestEntriesEndUpAsIOExceptionsWithNumOfFailedRequests()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 1, 100, true);
+        sink.write("25");
+        sink.write("55");
+        sink.write("75");
+        sink.write("95");
+        sink.write("35");
+        Exception e = assertThrows(RuntimeException.class, () -> sink.write("135"));
+        assertEquals(
+                "Deliberate runtime exception occurred in SinkWriterImplementation.",
+                e.getMessage());
+        assertEquals(3, res.size());
+    }
+
+    @Test
+    public void testRetryableErrorsDoNotViolateAtLeastOnceSemanticsDueToRequeueOfFailures()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 1, 100, true);
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "25", Arrays.asList(), Arrays.asList(25));
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "55", Arrays.asList(), Arrays.asList(25, 55));
+
+        // 25, 55 persisted; 965 failed and inflight
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "965", Arrays.asList(25, 55), Arrays.asList());
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "75", Arrays.asList(25, 55), Arrays.asList(75));
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "95", Arrays.asList(25, 55), Arrays.asList(75, 95));
+
+        /*
+         * Writing 955 to the sink increases the buffer to size 3 containing [75, 95, 955]. This
+         * triggers the outstanding in flight request with the failed 965 to be run, and 965 is
+         * placed at the front of the queue. The first {@code maxBatchSize = 3} elements are
+         * persisted, with 965 succeeding this (second) time. 955 remains in the buffer.
+         */
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "955", Arrays.asList(25, 55, 965, 75, 95), Arrays.asList(955));
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "550", Arrays.asList(25, 55, 965, 75, 95), Arrays.asList(955, 550));
+
+        /*
+         * [955, 550, 45] are attempted to be persisted
+         */
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "45", Arrays.asList(25, 55, 965, 75, 95, 45), Arrays.asList());
+
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "35", Arrays.asList(25, 55, 965, 75, 95, 45), Arrays.asList(35));
+
+        /* [35, 535] should be in the bufferedRequestEntries
+         * [955, 550] should be in the inFlightRequest, ready to be added
+         * [25, 55, 965, 75, 95, 45] should be downstream already
+         */
+        writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+                sink, "535", Arrays.asList(25, 55, 965, 75, 95, 45), Arrays.asList(35, 535));
+
+        // Checkpoint occurs
+        sink.prepareCommit(true);
+
+        // Everything is saved
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95, 45, 550, 955, 35, 535), res);
+        assertEquals(0, sink.snapshotState().get(0).size());
+    }
+
+    @Test
+    public void testFailedEntriesAreRetriedInTheNextPossiblePersistRequestAndNoLater()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 1, 100, true);
+        sink.write("25");
+        sink.write("55");
+        sink.write("965");
+        sink.write("75");
+        sink.write("95");
+        sink.write("955");
+        assertTrue(res.contains(965));
+        sink.write("550");
+        sink.write("645");
+        sink.write("545");
+        sink.write("535");
+        sink.write("515");
+        assertTrue(res.contains(955));
+        sink.write("505");
+        assertTrue(res.contains(550));
+        assertTrue(res.contains(645));
+        sink.prepareCommit(true);
+        assertTrue(res.contains(545));
+        assertTrue(res.contains(535));
+        assertTrue(res.contains(515));
+    }
+
+    @Test
+    public void testThatMaxBufferSizeOfSinkShouldBeStrictlyGreaterThanMaxSizeOfEachBatch() {
+        Exception e =
+                assertThrows(
+                        IllegalArgumentException.class,
+                        () -> new AsyncSinkWriterImpl(sinkInitContext, 10, 1, 10, false));
+        assertEquals(
+                e.getMessage(),
+                "The maximum number of requests that may be buffered should be "
+                        + "strictly greater than the maximum number of requests per batch.");
+    }
+
+    private void writeXToSinkAssertDestinationIsInStateYAndBufferHasZ(
+            AsyncSinkWriterImpl sink, String x, List<Integer> y, List<Integer> z)
+            throws IOException, InterruptedException {
+        sink.write(x);
+        assertEquals(y, res);
+        assertEquals(z, new ArrayList<>(sink.snapshotState().get(0)));
+    }
+
+    private class AsyncSinkWriterImpl extends AsyncSinkWriter<String, Integer> {
+
+        private final Set<Integer> failedFirstAttempts = new HashSet<>();
+        private final boolean simulateFailures;
+
+        public AsyncSinkWriterImpl(
+                Sink.InitContext context,
+                int maxBatchSize,
+                int maxInFlightRequests,
+                int maxBufferedRequests,
+                boolean simulateFailures) {
+            super(
+                    (elem, ctx) -> Integer.parseInt(elem),
+                    context,
+                    maxBatchSize,
+                    maxInFlightRequests,
+                    maxBufferedRequests);
+            this.simulateFailures = simulateFailures;
+        }
+
+        public void write(String val) throws IOException, InterruptedException {
+            write(val, null);
+        }
+
+        /**
+         * Fails if any value is between 101 and 200. If {@code simulateFailures} is set, it will
+         * fail on the first attempt but succeeds upon retry on all others for entries strictly
+         * greater than 200.
+         *
+         * <p>A limitation of this basic implementation is that each element written must be unique.
+         *
+         * @param requestEntries a set of request entries that should be persisted to {@code res}
+         * @param requestResult a Consumer that needs to accept a collection of failure elements
+         *     once all request entries have been persisted
+         */
+        @Override
+        protected void submitRequestEntries(
+                List<Integer> requestEntries, Consumer<Collection<Integer>> requestResult) {
+            if (requestEntries.stream().anyMatch(val -> val > 100 && val <= 200)) {
+                throw new RuntimeException(
+                        "Deliberate runtime exception occurred in SinkWriterImplementation.");
+            }
+            if (simulateFailures) {
+                List<Integer> successfulRetries =
+                        failedFirstAttempts.stream()
+                                .filter(requestEntries::contains)
+                                .collect(Collectors.toList());
+                failedFirstAttempts.removeIf(successfulRetries::contains);
+
+                List<Integer> firstTimeFailed =
+                        requestEntries.stream()
+                                .filter(x -> !successfulRetries.contains(x))
+                                .filter(val -> val > 200)
+                                .collect(Collectors.toList());
+                failedFirstAttempts.addAll(firstTimeFailed);
+
+                requestEntries.removeAll(firstTimeFailed);
+                res.addAll(requestEntries);
+                requestResult.accept(firstTimeFailed);
+            } else {
+                res.addAll(requestEntries);
+                requestResult.accept(new ArrayList<>());
+            }
+        }
+    }
+
+    private static class SinkInitContext implements Sink.InitContext {
+
+        @Override
+        public UserCodeClassLoader getUserCodeClassLoader() {
+            return null;
+        }
+
+        @Override
+        public MailboxExecutor getMailboxExecutor() {
+            StreamTaskActionExecutor streamTaskActionExecutor =
+                    new StreamTaskActionExecutor() {
+                        @Override
+                        public void run(RunnableWithException e) throws Exception {
+                            e.run();
+                        }
+
+                        @Override
+                        public <E extends Throwable> void runThrowing(
+                                ThrowingRunnable<E> throwingRunnable) throws E {
+                            throwingRunnable.run();
+                        }
+
+                        @Override
+                        public <R> R call(Callable<R> callable) throws Exception {
+                            return callable.call();
+                        }
+                    };
+            return new MailboxExecutorImpl(
+                    new TaskMailboxImpl(Thread.currentThread()),
+                    Integer.MAX_VALUE,
+                    streamTaskActionExecutor);
+        }
+
+        @Override
+        public Sink.ProcessingTimeService getProcessingTimeService() {
+            return null;
+        }
+
+        @Override
+        public int getSubtaskId() {
+            return 0;
+        }
+
+        @Override
+        public int getNumberOfParallelSubtasks() {
+            return 0;
+        }
+
+        @Override
+        public SinkWriterMetricGroup metricGroup() {
+            return null;
+        }
+
+        @Override
+        public OptionalLong getRestoredCheckpointId() {
+            return OptionalLong.empty();
+        }
+    }
+}