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

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

                Author: ASF GitHub Bot
            Created on: 05/Oct/18 19:53
            Start Date: 05/Oct/18 19:53
    Worklog Time Spent: 10m 
      Work Description: jklukas commented on a change in pull request #6586: 
[BEAM-5638] Exception handling for Java single message transforms
URL: https://github.com/apache/beam/pull/6586#discussion_r223122319
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
 ##########
 @@ -162,4 +171,89 @@ public void populateDisplayData(DisplayData.Builder 
builder) {
       builder.include("fn", (HasDisplayData) originalFnForDisplayData);
     }
   }
+
+  /**
+   * Sets a {@link TupleTag} to associate with successes, converting this 
{@link PTransform} into
+   * one that returns a {@link PCollectionTuple}. This allows you to make 
subsequent {@link
+   * WithFailures#withFailureTag(TupleTag, Class, Class[])} calls to capture 
thrown exceptions to
+   * failure collections.
+   */
+  public WithFailures withSuccessTag(TupleTag<OutputT> successTag) {
+    return new WithFailures(successTag, TaggedExceptionsList.empty());
+  }
 
 Review comment:
   I had wanted to avoid that so that we could verify state at compile time. 
Calling `withFailureTag` without first setting the success tag would require us 
to check state at runtime and fail. 
   
   This situation seems similar to other places in the Java SDK where we use 
intermediate objects to constrain possible misconfigurations at compile time. 
I'm thinking about `FileIO.writeDynamic` as one example where methods must be 
called in a certain order and this allows compile-time checking.

----------------------------------------------------------------
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: 151829)
            Time Spent: 2h  (was: 1h 50m)
    Remaining Estimate: 166h  (was: 166h 10m)

> Add exception handling to single message transforms in Java SDK
> ---------------------------------------------------------------
>
>                 Key: BEAM-5638
>                 URL: https://issues.apache.org/jira/browse/BEAM-5638
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-core
>            Reporter: Jeff Klukas
>            Assignee: Jeff Klukas
>            Priority: Minor
>   Original Estimate: 168h
>          Time Spent: 2h
>  Remaining Estimate: 166h
>
> Add methods to MapElements, FlatMapElements, and Filter that allow users to 
> specify expected exceptions and tuple tags to associate with the with 
> collections of the successfully and unsuccessfully processed elements.
> See discussion on dev list:
> https://lists.apache.org/thread.html/936ed2a5f2c01be066fd903abf70130625e0b8cf4028c11b89b8b23f@%3Cdev.beam.apache.org%3E



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

Reply via email to