This is an automated email from the ASF dual-hosted git repository.

gaborgsomogyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new a4e3084ab92 [FLINK-37857][tests] Add a DSV2 sink implementation to 
tests which can collect data
a4e3084ab92 is described below

commit a4e3084ab9220b02bc3eefc70ea372251aec7f07
Author: Gabor Somogyi <[email protected]>
AuthorDate: Thu May 29 08:50:32 2025 +0200

    [FLINK-37857][tests] Add a DSV2 sink implementation to tests which can 
collect data
---
 .../streaming/util/testing/CollectingSink.java     | 106 +++++++++++++++++++++
 .../flink/runtime/util/CollectingSinkTest.java     |  71 ++++++++++++++
 2 files changed, 177 insertions(+)

diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/testing/CollectingSink.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/testing/CollectingSink.java
new file mode 100644
index 00000000000..a751d97eb7a
--- /dev/null
+++ 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/testing/CollectingSink.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.testing;
+
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.connector.sink2.WriterInitContext;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/** Sink for collecting output during testing. */
+public class CollectingSink<T> implements Sink<T> {
+    private static final long serialVersionUID = 1L;
+    private static final List<BlockingQueue<Object>> queues =
+            Collections.synchronizedList(new ArrayList<>());
+    private static final AtomicInteger numSinks = new AtomicInteger(-1);
+    private final int index;
+
+    public CollectingSink() {
+        this.index = numSinks.incrementAndGet();
+        queues.add(new LinkedBlockingQueue<>());
+    }
+
+    @Override
+    public SinkWriter<T> createWriter(WriterInitContext context) throws 
IOException {
+        return new CollectingElementWriter(index);
+    }
+
+    private class CollectingElementWriter implements SinkWriter<T> {
+        private final int index;
+
+        public CollectingElementWriter(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void write(T element, Context context) {
+            queues.get(this.index).add(element);
+        }
+
+        @Override
+        public void flush(boolean endOfInput) {}
+
+        @Override
+        public void close() {}
+    }
+
+    @SuppressWarnings("unchecked")
+    public List<T> getRemainingOutput() {
+        return new ArrayList<>((BlockingQueue<T>) queues.get(this.index));
+    }
+
+    public boolean isEmpty() {
+        return queues.get(this.index).isEmpty();
+    }
+
+    public T poll() throws TimeoutException {
+        return this.poll(Duration.ofSeconds(15L));
+    }
+
+    @SuppressWarnings("unchecked")
+    public T poll(Duration duration) throws TimeoutException {
+        Object element;
+
+        try {
+            element = queues.get(this.index).poll(duration.toMillis(), 
TimeUnit.MILLISECONDS);
+        } catch (InterruptedException var4) {
+            throw new RuntimeException(var4);
+        }
+
+        if (element == null) {
+            throw new TimeoutException();
+        } else {
+            return (T) element;
+        }
+    }
+
+    public void close() {
+        queues.get(this.index).clear();
+    }
+}
diff --git 
a/flink-tests/src/test/java/org/apache/flink/runtime/util/CollectingSinkTest.java
 
b/flink-tests/src/test/java/org/apache/flink/runtime/util/CollectingSinkTest.java
new file mode 100644
index 00000000000..d4ea45ab9c5
--- /dev/null
+++ 
b/flink-tests/src/test/java/org/apache/flink/runtime/util/CollectingSinkTest.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.runtime.util;
+
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.testing.CollectingSink;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests {@link CollectingSink}. */
+class CollectingSinkTest {
+
+    private CollectingSink<Integer> sink;
+
+    @BeforeEach
+    public void setup() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+
+        DataStreamSource<Integer> stream = env.fromData(0, 1, 
2).setParallelism(1);
+
+        sink = new CollectingSink<>();
+        stream.sinkTo(sink);
+        env.execute();
+    }
+
+    @Test
+    void testGetRemainingOutputGivesBackData() {
+        List<Integer> result = sink.getRemainingOutput();
+        
assertThat(result).containsExactlyInAnyOrderElementsOf(Arrays.asList(0, 1, 2));
+        closeAndAssertEmpty();
+    }
+
+    @Test
+    void testPollGivesBackData() throws TimeoutException {
+        for (int i = 0; i < 3; i++) {
+            assertThat(sink.poll()).isEqualTo(i);
+        }
+        closeAndAssertEmpty();
+    }
+
+    private void closeAndAssertEmpty() {
+        sink.close();
+        List<Integer> result = sink.getRemainingOutput();
+        assertThat(result).isEmpty();
+    }
+}

Reply via email to