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

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

                Author: ASF GitHub Bot
            Created on: 15/Oct/18 18:13
            Start Date: 15/Oct/18 18:13
    Worklog Time Spent: 10m 
      Work Description: Noctune opened a new pull request #6696: [BEAM-5098] 
Fix withoutDefaults and withFanout clearing side inputs
URL: https://github.com/apache/beam/pull/6696
 
 
   Calling `withFanout` or `withFanout` on `Combine.Globally` would clear side 
input. This was due to those two functions using a constructor that did not 
accept side inputs. I have removed the offending constructor and rewritten the 
two functions to use the correct constructor.
   
   I have also added a regression test in a separate commit. If it's too much 
for such a simple change, you can just merge commit with the fix.
   
   Post-Commit Tests Status (on master branch)
   
------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/)
 | --- | --- | --- | --- | --- | ---
   Java | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/)
   Python | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/)
 | --- | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)
 </br> [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/)
 | --- | --- | ---
   
   
   
   
   

----------------------------------------------------------------
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: 154404)
            Time Spent: 10m
    Remaining Estimate: 0h

> Combine.Globally::asSingletonView clears side inputs
> ----------------------------------------------------
>
>                 Key: BEAM-5098
>                 URL: https://issues.apache.org/jira/browse/BEAM-5098
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>    Affects Versions: 2.5.0
>            Reporter: Mike Pedersen
>            Assignee: Kenneth Knowles
>            Priority: Critical
>              Labels: beginner, starter
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> It seems like calling .asSingletonView on Combine.Globally clears all side 
> inputs. Take this code for example:
>  
> {code:java}
> public class Main {
>     public static void main(String[] args) {
>         PipelineOptions options = PipelineOptionsFactory.create();
>         Pipeline p = Pipeline.create(options);
>         PCollection<Integer> a = p.apply(Create.of(1, 2, 3));
>         PCollectionView<Integer> b = 
> p.apply(Create.of(10)).apply(View.asSingleton());
>         a
>                 .apply(Combine.globally(new 
> CombineWithContext.CombineFnWithContext<Integer, Integer, Integer>() {
>                     @Override
>                     public Integer 
> createAccumulator(CombineWithContext.Context c) {
>                         return c.sideInput(b);
>                     }
>                     @Override
>                     public Integer addInput(Integer accumulator, Integer 
> input, CombineWithContext.Context c) {
>                         return accumulator + input;
>                     }
>                     @Override
>                     public Integer mergeAccumulators(Iterable<Integer> 
> accumulators, CombineWithContext.Context c) {
>                         int sum = 0;
>                         for (int i : accumulators) {
>                             sum += i;
>                         }
>                         return sum;
>                     }
>                     @Override
>                     public Integer extractOutput(Integer accumulator, 
> CombineWithContext.Context c) {
>                         return accumulator;
>                     }
>                     @Override
>                     public Integer defaultValue() {
>                         return 0;
>                     }
>                 }).withSideInputs(b).asSingletonView());
>         p.run().waitUntilFinish();
>     }
> }{code}
> This fails with the following exception:
> {code:java}
> Exception in thread "main" 
> org.apache.beam.sdk.Pipeline$PipelineExecutionException: 
> java.lang.IllegalArgumentException: calling sideInput() with unknown view
>     at 
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:349)
>     at 
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:319)
>     at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:210)
>     at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:66)
>     at org.apache.beam.sdk.Pipeline.run(Pipeline.java:311)
>     at org.apache.beam.sdk.Pipeline.run(Pipeline.java:297)
>     at Main.main(Main.java:287)
> Caused by: java.lang.IllegalArgumentException: calling sideInput() with 
> unknown view
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.runners.core.SimpleDoFnRunner.sideInput(SimpleDoFnRunner.java:212)
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.runners.core.SimpleDoFnRunner.access$900(SimpleDoFnRunner.java:69)
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.sideInput(SimpleDoFnRunner.java:489)
>     at 
> org.apache.beam.sdk.transforms.Combine$GroupedValues$1$1.sideInput(Combine.java:2137)
>     at Main$1.createAccumulator(Main.java:258)
>     at Main$1.createAccumulator(Main.java:255)
>     at 
> org.apache.beam.sdk.transforms.CombineWithContext$CombineFnWithContext.apply(CombineWithContext.java:120)
>     at 
> org.apache.beam.sdk.transforms.Combine$GroupedValues$1.processElement(Combine.java:2129){code}
> But if you change
> {code:java}
> .withSideInputs(b).asSingletonView()){code}
> to
> {code:java}
> .withSideInputs(b)).apply(View.asSingleton()){code}
> then it works just fine.
>  



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

Reply via email to