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

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

                Author: ASF GitHub Bot
            Created on: 23/Apr/18 18:53
            Start Date: 23/Apr/18 18:53
    Worklog Time Spent: 10m 
      Work Description: jkff 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_r183501107
 
 

 ##########
 File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
 ##########
 @@ -361,7 +362,18 @@ public void setup() {
     public void processElement(final ProcessContext c) {
       final InputT element = c.element().getKey();
       invoker.invokeSplitRestriction(
-          element, c.element().getValue(), part -> c.output(KV.of(element, 
part)));
+          element, c.element().getValue(), new OutputReceiver<RestrictionT>() {
+            @Override
+            public void output(RestrictionT part) {
+              c.output(KV.of(element, part));
+            }
+
+            @Override
+            public void outputWithTimestamp(RestrictionT part, Instant 
timestamp) {
+              c.outputWithTimestamp(KV.of(element, part), timestamp);
 
 Review comment:
   I think this one should be an UnsupportedOperationException - it probably 
doesn't make sense to have different splits of a restriction have different 
timestamps.

----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 94244)
    Time Spent: 1h 20m  (was: 1h 10m)

> 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: 1h 20m
>  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