Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-6298

Local execution is not setting RuntimeContext for RichOutputFormat

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 1.1.0, 1.2.0
    • Fix Version/s: 1.3.0, 1.2.2
    • Component/s: DataStream API
    • Labels:
      None

      Description

      RuntimeContext is never set in RichOutputFormat. I tested it in local execution. RichMapFunction is setup correctly.

      Following code will never print "//////Context set in RichOutputFormat"

      import org.apache.flink.api.common.functions.{RichMapFunction, RuntimeContext}
      import org.apache.flink.api.common.io.RichOutputFormat
      import org.apache.flink.api.scala._
      import org.apache.flink.configuration.Configuration
      import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
      
      object Startup {
        def main(args: Array[String]): Unit = {
      
          val mapFunction = new RichMapFunction[String, String] {
            def open(taskNumber: Int, numTasks: Int) { getRuntimeContext }
            def map(event: String) = { event }
            override def setRuntimeContext(t: RuntimeContext) = {
              println("//////Context set in RichMapFunction")
              super.setRuntimeContext(t)
            }
          }
      
          val outputFormat = new RichOutputFormat[String] {
            override def setRuntimeContext(t: RuntimeContext) = {
              println("//////Context set in RichOutputFormat")
              super.setRuntimeContext(t)
            }
            def open(taskNumber: Int, numTasks: Int) {}
            def writeRecord(event: String) {
              println(event)
            }
            def configure(parameters: Configuration): Unit = {}
            def close(): Unit = {}
          }
      
          val see = StreamExecutionEnvironment.getExecutionEnvironment
          val eventsStream = see.fromElements[String]("A", "B", "C").map(mapFunction)
          eventsStream.writeUsingOutputFormat(outputFormat)
          see.execute("test-job")
        }
      }
      

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user wenlong88 opened a pull request:

          https://github.com/apache/flink/pull/3716

          FLINK-6298Local execution is not setting RuntimeContext for RichOutputFormat

          call set RuntimeContext OutputFormat when the OutputFormat is RichOutputFormat

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/wenlong88/flink jira-6298

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3716.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3716


          commit 050d3262b2882cb6ca1a6b6a23ad7d49f76b1e94
          Author: wenlong.lwl <wenlong.lwl@alibaba-inc.com>
          Date: 2017-04-13T09:23:15Z

          Set runtimeContext to output format when it is a RichOutputFormat


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user wenlong88 opened a pull request: https://github.com/apache/flink/pull/3716 FLINK-6298 Local execution is not setting RuntimeContext for RichOutputFormat call set RuntimeContext OutputFormat when the OutputFormat is RichOutputFormat You can merge this pull request into a Git repository by running: $ git pull https://github.com/wenlong88/flink jira-6298 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3716.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3716 commit 050d3262b2882cb6ca1a6b6a23ad7d49f76b1e94 Author: wenlong.lwl <wenlong.lwl@alibaba-inc.com> Date: 2017-04-13T09:23:15Z Set runtimeContext to output format when it is a RichOutputFormat
          Hide
          wenlong.lwl Wenlong Lyu added a comment -

          It is a bug in all execution environenment, I think we should set runtime context when the output format is a RichOutputFormat.

          Show
          wenlong.lwl Wenlong Lyu added a comment - It is a bug in all execution environenment, I think we should set runtime context when the output format is a RichOutputFormat.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user greghogan commented on the issue:

          https://github.com/apache/flink/pull/3716

          +1

          Show
          githubbot ASF GitHub Bot added a comment - Github user greghogan commented on the issue: https://github.com/apache/flink/pull/3716 +1
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3716

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3716
          Hide
          greghogan Greg Hogan added a comment -

          Fixed in 1.3.0: 1dde7327058cb973bd4ff94bf5531d5093fb60a2
          and 1.2.2: 5061e7e28dd3688801406c3cb1ebb4bfac012729

          Show
          greghogan Greg Hogan added a comment - Fixed in 1.3.0: 1dde7327058cb973bd4ff94bf5531d5093fb60a2 and 1.2.2: 5061e7e28dd3688801406c3cb1ebb4bfac012729

            People

            • Assignee:
              wenlong.lwl Wenlong Lyu
              Reporter:
              Matzz Mateusz Zakarczemny
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development