[ 
https://issues.apache.org/jira/browse/BEAM-3979?focusedWorklogId=86642&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-86642
 ]

ASF GitHub Bot logged work on BEAM-3979:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Apr/18 16:44
            Start Date: 02/Apr/18 16:44
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #4989: 
[BEAM-3979] Start completing the new DoFn vision: plumb context parameters into 
process functions.
URL: https://github.com/apache/beam/pull/4989#discussion_r178582698
 
 

 ##########
 File path: 
runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
 ##########
 @@ -609,6 +724,31 @@ public TimeDomain timeDomain() {
       throw new UnsupportedOperationException("ProcessContext parameters are 
not supported.");
     }
 
+    @Override
+    public InputT element(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("Element parameters are not 
supported.");
+    }
+
+    @Override
+    public Instant timestamp(DoFn<InputT, OutputT> doFn) {
+      return timestamp();
+    }
+
+    @Override
+    public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
+      return timeDomain();
+    }
+
+    @Override
+    public OutputReceiver<OutputT> outputReceiver(DoFn<InputT, OutputT> doFn) {
+      return new WindowedContextOutputReceiver<>(this);
 
 Review comment:
   Why not return `this`, it effectively implements `OutputReceiver` already.
   
   Ditto elsewhere.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 86642)
    Time Spent: 0.5h  (was: 20m)

> New DoFn should allow injecting of all parameters in ProcessContext
> -------------------------------------------------------------------
>
>                 Key: BEAM-3979
>                 URL: https://issues.apache.org/jira/browse/BEAM-3979
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>    Affects Versions: 2.4.0
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>             Fix For: 2.5.0
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> This was intended in the past, but never completed. Ideally all primitive 
> parameters in ProcessContext should be injectable, and OutputReceiver 
> parameters can be used to collection output. So, we should be able to write a 
> DoFn as follows
> @ProcessElement
> public void process(@Element String word, OutputReceiver<String> receiver) {
>   receiver.output(word.toUpperCase());
> }



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to