You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2022/08/08 23:04:34 UTC

[flink] branch master updated: [FLINK-28865] Add new PrintSink

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

chesnay 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 aaea1adc155 [FLINK-28865] Add new PrintSink
aaea1adc155 is described below

commit aaea1adc155122f066736a4e2a4a287a40a77969
Author: Chesnay Schepler <ch...@apache.org>
AuthorDate: Tue Aug 9 01:04:25 2022 +0200

    [FLINK-28865] Add new PrintSink
---
 .../functions/util/PrintSinkOutputWriter.java      |  17 +-
 .../streaming/api/functions/sink/PrintSink.java    |  97 +++++++++
 .../streaming/api/functions/PrintSinkTest.java     | 232 +++++++++++++++++++++
 3 files changed, 345 insertions(+), 1 deletion(-)

diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/PrintSinkOutputWriter.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/PrintSinkOutputWriter.java
index 15f19424334..4ecce807f6c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/PrintSinkOutputWriter.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/PrintSinkOutputWriter.java
@@ -18,13 +18,15 @@
 package org.apache.flink.api.common.functions.util;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink2.SinkWriter;
 
+import java.io.IOException;
 import java.io.PrintStream;
 import java.io.Serializable;
 
 /** Print sink output writer for DataStream and DataSet print API. */
 @Internal
-public class PrintSinkOutputWriter<IN> implements Serializable {
+public class PrintSinkOutputWriter<IN> implements Serializable, SinkWriter<IN> {
 
     private static final long serialVersionUID = 1L;
 
@@ -71,6 +73,19 @@ public class PrintSinkOutputWriter<IN> implements Serializable {
         stream.println(completedPrefix + record.toString());
     }
 
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        write(element);
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException, InterruptedException {
+        stream.flush();
+    }
+
+    @Override
+    public void close() throws Exception {}
+
     @Override
     public String toString() {
         return "Print to " + (target == STD_OUT ? "System.out" : "System.err");
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSink.java
new file mode 100644
index 00000000000..7755268c161
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSink.java
@@ -0,0 +1,97 @@
+/*
+ * 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.api.functions.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.util.PrintSinkOutputWriter;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+
+import java.io.IOException;
+
+/**
+ * Sink implementation writing every element to the standard output or standard error stream.
+ *
+ * <p>Four possible format options:
+ *
+ * <ul>
+ *   <li>{@code sinkIdentifier}:subtaskIndex> output <- {@code sinkIdentifier} provided and
+ *       parallelism > 1,
+ *   <li>{@code sinkIdentifier}> output <- {@code sinkIdentifier} provided and parallelism == 1
+ *   <li>subtaskIndex> output <- no {@code sinkIdentifier} provided and parallelism > 1
+ *   <li>output <- no {@code sinkIdentifier} provided and parallelism == 1
+ * </ul>
+ *
+ * @param <IN> Input record type
+ */
+@PublicEvolving
+public class PrintSink<IN> implements Sink<IN> {
+
+    private static final long serialVersionUID = 1L;
+    private final String sinkIdentifier;
+    private final boolean stdErr;
+
+    /** Instantiates a print sink function that prints to STDOUT. */
+    public PrintSink() {
+        this("");
+    }
+
+    /**
+     * Instantiates a print sink that prints to STDOUT or STDERR.
+     *
+     * @param stdErr True, if the format should print to standard error instead of standard out.
+     */
+    public PrintSink(final boolean stdErr) {
+        this("", stdErr);
+    }
+
+    /**
+     * Instantiates a print sink that prints to STDOUT and gives a sink identifier.
+     *
+     * @param sinkIdentifier Message that identifies the sink and is prefixed to the output of the
+     *     value
+     */
+    public PrintSink(final String sinkIdentifier) {
+        this(sinkIdentifier, false);
+    }
+
+    /**
+     * Instantiates a print sink that prints to STDOUT or STDERR and gives a sink identifier.
+     *
+     * @param sinkIdentifier Message that identifies the sink and is prefixed to the output of the
+     *     value
+     * @param stdErr True if the sink should print to STDERR instead of STDOUT.
+     */
+    public PrintSink(final String sinkIdentifier, final boolean stdErr) {
+        this.sinkIdentifier = sinkIdentifier;
+        this.stdErr = stdErr;
+    }
+
+    @Override
+    public SinkWriter<IN> createWriter(InitContext context) throws IOException {
+        final PrintSinkOutputWriter<IN> writer =
+                new PrintSinkOutputWriter<>(sinkIdentifier, stdErr);
+        writer.open(context.getSubtaskId(), context.getNumberOfParallelSubtasks());
+        return writer;
+    }
+
+    @Override
+    public String toString() {
+        return "Print to " + (stdErr ? "System.err" : "System.out");
+    }
+}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java
new file mode 100644
index 00000000000..2a86e20fcdd
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkTest.java
@@ -0,0 +1,232 @@
+/*
+ * 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.api.functions;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.streaming.api.functions.sink.PrintSink;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.SimpleUserCodeClassLoader;
+import org.apache.flink.util.UserCodeClassLoader;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.OptionalLong;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests for the {@link PrintSink}. */
+class PrintSinkTest {
+
+    private final PrintStream originalSystemOut = System.out;
+    private final PrintStream originalSystemErr = System.err;
+
+    private final ByteArrayOutputStream arrayOutputStream = new ByteArrayOutputStream();
+    private final ByteArrayOutputStream arrayErrorStream = new ByteArrayOutputStream();
+
+    private final String line = System.lineSeparator();
+
+    @BeforeEach
+    void setUp() {
+        System.setOut(new PrintStream(arrayOutputStream));
+        System.setErr(new PrintStream(arrayErrorStream));
+    }
+
+    @AfterEach
+    void tearDown() {
+        if (System.out != originalSystemOut) {
+            System.out.close();
+        }
+        if (System.err != originalSystemErr) {
+            System.err.close();
+        }
+        System.setOut(originalSystemOut);
+        System.setErr(originalSystemErr);
+    }
+
+    @Test
+    void testPrintSinkStdOut() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>();
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(1))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.out", printSink.toString());
+            assertEquals("hello world!" + line, arrayOutputStream.toString());
+        }
+    }
+
+    @Test
+    void testPrintSinkStdErr() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>(true);
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(1))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.err", printSink.toString());
+            assertEquals("hello world!" + line, arrayErrorStream.toString());
+        }
+    }
+
+    @Test
+    void testPrintSinkStdErrWithIdentifier() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>("mySink", true);
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(1))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.err", printSink.toString());
+            assertEquals("mySink> hello world!" + line, arrayErrorStream.toString());
+        }
+    }
+
+    @Test
+    void testPrintSinkWithPrefix() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>();
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(2))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.out", printSink.toString());
+            assertEquals("1> hello world!" + line, arrayOutputStream.toString());
+        }
+    }
+
+    @Test
+    void testPrintSinkWithIdentifierAndPrefix() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>("mySink");
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(2))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.out", printSink.toString());
+            assertEquals("mySink:1> hello world!" + line, arrayOutputStream.toString());
+        }
+    }
+
+    @Test
+    void testPrintSinkWithIdentifierButNoPrefix() throws Exception {
+        PrintSink<String> printSink = new PrintSink<>("mySink");
+
+        try (SinkWriter<String> writer = printSink.createWriter(new MockInitContext(1))) {
+            writer.write("hello world!", new MockContext());
+
+            assertEquals("Print to System.out", printSink.toString());
+            assertEquals("mySink> hello world!" + line, arrayOutputStream.toString());
+        }
+    }
+
+    private static class MockContext implements SinkWriter.Context {
+
+        @Override
+        public long currentWatermark() {
+            return 0;
+        }
+
+        @Override
+        public Long timestamp() {
+            return System.currentTimeMillis();
+        }
+    }
+
+    private static class MockInitContext
+            implements Sink.InitContext, SerializationSchema.InitializationContext {
+
+        private final int numSubtasks;
+
+        private MockInitContext(int numSubtasks) {
+            this.numSubtasks = numSubtasks;
+        }
+
+        @Override
+        public UserCodeClassLoader getUserCodeClassLoader() {
+            return SimpleUserCodeClassLoader.create(PrintSinkTest.class.getClassLoader());
+        }
+
+        @Override
+        public MailboxExecutor getMailboxExecutor() {
+            return new DummyMailboxExecutor();
+        }
+
+        @Override
+        public ProcessingTimeService getProcessingTimeService() {
+            return new TestProcessingTimeService();
+        }
+
+        @Override
+        public int getSubtaskId() {
+            return 0;
+        }
+
+        @Override
+        public int getNumberOfParallelSubtasks() {
+            return numSubtasks;
+        }
+
+        @Override
+        public SinkWriterMetricGroup metricGroup() {
+            return InternalSinkWriterMetricGroup.mock(new UnregisteredMetricsGroup());
+        }
+
+        @Override
+        public MetricGroup getMetricGroup() {
+            return metricGroup();
+        }
+
+        @Override
+        public OptionalLong getRestoredCheckpointId() {
+            return OptionalLong.empty();
+        }
+
+        @Override
+        public SerializationSchema.InitializationContext
+                asSerializationSchemaInitializationContext() {
+            return this;
+        }
+    }
+
+    private static class DummyMailboxExecutor implements MailboxExecutor {
+
+        @Override
+        public void execute(
+                ThrowingRunnable<? extends Exception> command,
+                String descriptionFormat,
+                Object... descriptionArgs) {}
+
+        @Override
+        public void yield() throws InterruptedException, FlinkRuntimeException {}
+
+        @Override
+        public boolean tryYield() throws FlinkRuntimeException {
+            return false;
+        }
+    }
+}