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;
+ }
+ }
+}