dannycranmer commented on a change in pull request #17068:
URL: https://github.com/apache/flink/pull/17068#discussion_r703044019



##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListAsyncSink.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.ResultFuture;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/** Dummy destination that records write events. */
+public class ArrayListAsyncSink extends AsyncSinkBase<String, Integer> {
+
+    /**
+     * SinkWriter implementing {@code submitRequestEntries} that is used to 
define the persistence
+     * logic into {@code ArrayListDestination}.
+     */
+    public static class ArrayListAsyncSinkWriter extends 
AsyncSinkWriter<String, Integer> {
+
+        protected ArrayListAsyncSinkWriter(
+                Sink.InitContext context,
+                int maxBatchSize,
+                int maxInFlightRequests,
+                int maxBufferedRequests) {
+            super(
+                    (element, x) -> Integer.parseInt(element),
+                    context,
+                    maxBatchSize,
+                    maxInFlightRequests,
+                    maxBufferedRequests);
+        }
+
+        @Override
+        protected void submitRequestEntries(
+                List<Integer> requestEntries, ResultFuture<Integer> 
requestResult) {
+            ArrayListDestination.putRecords(requestEntries);
+            requestResult.complete(Arrays.asList());
+        }
+    }
+
+    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) {
+        return new ArrayListAsyncSinkWriter(

Review comment:
       Instead of creating a static class, is it possible to return a lambda 
here?

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/ArrayListAsyncSink.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.ResultFuture;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/** Dummy destination that records write events. */
+public class ArrayListAsyncSink extends AsyncSinkBase<String, Integer> {
+
+    /**
+     * SinkWriter implementing {@code submitRequestEntries} that is used to 
define the persistence
+     * logic into {@code ArrayListDestination}.
+     */
+    public static class ArrayListAsyncSinkWriter extends 
AsyncSinkWriter<String, Integer> {
+
+        protected ArrayListAsyncSinkWriter(
+                Sink.InitContext context,
+                int maxBatchSize,
+                int maxInFlightRequests,
+                int maxBufferedRequests) {
+            super(
+                    (element, x) -> Integer.parseInt(element),
+                    context,
+                    maxBatchSize,
+                    maxInFlightRequests,
+                    maxBufferedRequests);
+        }
+
+        @Override
+        protected void submitRequestEntries(
+                List<Integer> requestEntries, ResultFuture<Integer> 
requestResult) {
+            ArrayListDestination.putRecords(requestEntries);
+            requestResult.complete(Arrays.asList());
+        }
+    }
+
+    private final int maxBatchSize;
+    private final int maxInFlightRequests;
+    private final int maxBufferedRequests;
+
+    public ArrayListAsyncSink() {
+        this(25, 1, 100);
+    }

Review comment:
       nit: Move to top of file

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.Callable;
+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 
numOfRecordsIsAMultipleOfBatchSizeResultsInThatNumberOfRecordsBeingWritten()
+            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 unwrittenRecordsInBufferArePersistedWhenSnapshotIsTaken()
+            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 
preparingCommitAtSnapshotTimeEnsuresTheBufferedRecordsArePersistedToDestination()
+            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 
snapshotsAreTakenOfBufferCorrectlyBeforeAndAfterAutomaticFlush()
+            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 snapshotsAreTakenOfBufferCorrectlyBeforeAndAfterManualFlush()
+            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 
runtimeErrorsInSubmitRequestEntriesEndUpAsIOExceptionsWithNumberOfFailedRequests()
+            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 
retryableErrorsDoNotResultInViolationOfAtLeastOnceSemanticsDueToRequeueOfFailures()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 
1, 100, true);
+
+        sink.write("25");
+        assertEquals(Arrays.asList(), res);
+        assertEquals(Arrays.asList(25), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("55");
+        assertEquals(Arrays.asList(), res);
+        assertEquals(Arrays.asList(25, 55), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("965");
+        assertEquals(Arrays.asList(25, 55), res); // 25, 55 persisted; 965 
failed
+        assertEquals(Arrays.asList(), new 
ArrayList<>(sink.snapshotState().get(0))); // 965 inflight
+
+        sink.write("75");
+        assertEquals(Arrays.asList(25, 55), res);
+        assertEquals(Arrays.asList(75), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("95");
+        assertEquals(Arrays.asList(25, 55), res);
+        assertEquals(Arrays.asList(75, 95), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        /*
+         * 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.
+         */
+        sink.write("955");
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95), res);
+        assertEquals(Arrays.asList(955), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("550");
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95), res);
+        assertEquals(Arrays.asList(955, 550), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        /*
+         * [955, 550, 45] are attempted to be persisted
+         */
+        sink.write("45");
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95, 45), res);
+        assertEquals(Arrays.asList(), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("35");
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95, 45), res);
+        assertEquals(Arrays.asList(35), new 
ArrayList<>(sink.snapshotState().get(0)));
+
+        sink.write("535");
+
+        // [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
+        assertEquals(Arrays.asList(35, 535), new 
ArrayList<>(sink.snapshotState().get(0)));
+        assertEquals(Arrays.asList(25, 55, 965, 75, 95, 45), res);
+
+        // 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 
failedEntriesAreRetriedInTheNextPossiblePersistRequestAndNoLater()
+            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(545));

Review comment:
       Duplicate assertion here

##########
File path: 
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 writeTwentyThousandRecordsToGenericSink() throws Exception {

Review comment:
       The standard in Flink is for tests to start with `test`, unless you have 
seen otherwise? Can you please update for consistency, unless you believe this 
is ok?

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.Callable;
+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 
numOfRecordsIsAMultipleOfBatchSizeResultsInThatNumberOfRecordsBeingWritten()
+            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 unwrittenRecordsInBufferArePersistedWhenSnapshotIsTaken()
+            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 
preparingCommitAtSnapshotTimeEnsuresTheBufferedRecordsArePersistedToDestination()
+            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 
snapshotsAreTakenOfBufferCorrectlyBeforeAndAfterAutomaticFlush()
+            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 snapshotsAreTakenOfBufferCorrectlyBeforeAndAfterManualFlush()
+            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 
runtimeErrorsInSubmitRequestEntriesEndUpAsIOExceptionsWithNumberOfFailedRequests()
+            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 
retryableErrorsDoNotResultInViolationOfAtLeastOnceSemanticsDueToRequeueOfFailures()
+            throws IOException, InterruptedException {
+        AsyncSinkWriterImpl sink = new AsyncSinkWriterImpl(sinkInitContext, 3, 
1, 100, true);
+
+        sink.write("25");
+        assertEquals(Arrays.asList(), res);
+        assertEquals(Arrays.asList(25), new 
ArrayList<>(sink.snapshotState().get(0)));

Review comment:
       nit: A lot of repetition here, method is long. Consider extracting the 3 
lines to a method if it makes it more readable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to