You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gr...@apache.org on 2017/04/14 01:33:56 UTC

flink git commit: [FLINK-6298] [java api] Set context in RichOutputFormat

Repository: flink
Updated Branches:
  refs/heads/release-1.2 6e903bfad -> 5061e7e28


[FLINK-6298] [java api] Set context in RichOutputFormat

This closes #3716


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5061e7e2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5061e7e2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5061e7e2

Branch: refs/heads/release-1.2
Commit: 5061e7e28dd3688801406c3cb1ebb4bfac012729
Parents: 6e903bf
Author: wenlong.lwl <we...@alibaba-inc.com>
Authored: Thu Apr 13 17:23:15 2017 +0800
Committer: Greg Hogan <co...@greghogan.com>
Committed: Thu Apr 13 20:34:11 2017 -0400

----------------------------------------------------------------------
 .../sink/OutputFormatSinkFunction.java          |  9 +++++
 .../sink/OutputFormatSinkFunctionTest.java      | 42 ++++++++++++++++++++
 2 files changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5061e7e2/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java
index 16ec3f5..3f40095 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.io.CleanupWhenUnsuccessful;
 import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.configuration.Configuration;
@@ -61,6 +62,14 @@ public class OutputFormatSinkFunction<IN> extends RichSinkFunction<IN> implement
 	}
 
 	@Override
+	public void setRuntimeContext(RuntimeContext context) {
+		super.setRuntimeContext(context);
+		if (format instanceof RichOutputFormat) {
+			((RichOutputFormat) format).setRuntimeContext(context);
+		}
+	}
+
+	@Override
 	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
 		if (format instanceof InputTypeConfigurable) {
 			InputTypeConfigurable itc = (InputTypeConfigurable) format;

http://git-wip-us.apache.org/repos/asf/flink/blob/5061e7e2/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunctionTest.java
new file mode 100644
index 0000000..d3a9d3d
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunctionTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class OutputFormatSinkFunctionTest {
+
+	@Test
+	public void setRuntimeContext() throws Exception {
+		RuntimeContext mockRuntimeContext = Mockito.mock(RuntimeContext.class);
+
+		// Make sure setRuntimeContext of the rich output format is called
+		RichOutputFormat<?> mockRichOutputFormat = Mockito.mock(RichOutputFormat.class);
+		new OutputFormatSinkFunction<>(mockRichOutputFormat).setRuntimeContext(mockRuntimeContext);
+		Mockito.verify(mockRichOutputFormat, Mockito.times(1)).setRuntimeContext(Mockito.eq(mockRuntimeContext));
+
+		// Make sure setRuntimeContext work well when output format is not RichOutputFormat
+		OutputFormat<?> mockOutputFormat = Mockito.mock(OutputFormat.class);
+		new OutputFormatSinkFunction<>(mockOutputFormat).setRuntimeContext(mockRuntimeContext);
+	}
+}