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

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

                Author: ASF GitHub Bot
            Created on: 16/Oct/18 15:57
            Start Date: 16/Oct/18 15:57
    Worklog Time Spent: 10m 
      Work Description: lukecwik closed pull request #6696: [BEAM-5098] Fix 
withoutDefaults and withFanout clearing side inputs
URL: https://github.com/apache/beam/pull/6696
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index cff3f6874a8..118f79e7c93 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -126,7 +126,7 @@ private Combine() {
   private static <InputT, OutputT> Globally<InputT, OutputT> globally(
       GlobalCombineFn<? super InputT, ?, OutputT> fn,
       DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
-    return new Globally<>(fn, fnDisplayData, true, 0);
+    return new Globally<>(fn, fnDisplayData, true, 0, ImmutableList.of());
   }
 
   /**
@@ -994,18 +994,6 @@ public final OutputT extractOutput(AccumT accumulator) {
     private final int fanout;
     private final List<PCollectionView<?>> sideInputs;
 
-    private Globally(
-        GlobalCombineFn<? super InputT, ?, OutputT> fn,
-        DisplayData.ItemSpec<? extends Class<?>> fnDisplayData,
-        boolean insertDefault,
-        int fanout) {
-      this.fn = fn;
-      this.fnDisplayData = fnDisplayData;
-      this.insertDefault = insertDefault;
-      this.fanout = fanout;
-      this.sideInputs = ImmutableList.of();
-    }
-
     private Globally(
         GlobalCombineFn<? super InputT, ?, OutputT> fn,
         DisplayData.ItemSpec<? extends Class<?>> fnDisplayData,
@@ -1040,7 +1028,7 @@ protected String getKindString() {
      * and the output is not being used as a side input.
      */
     public Globally<InputT, OutputT> withoutDefaults() {
-      return new Globally<>(fn, fnDisplayData, false, fanout);
+      return new Globally<>(fn, fnDisplayData, false, fanout, sideInputs);
     }
 
     /**
@@ -1050,7 +1038,7 @@ protected String getKindString() {
      * <p>The {@code fanout} parameter determines the number of intermediate 
keys that will be used.
      */
     public Globally<InputT, OutputT> withFanout(int fanout) {
-      return new Globally<>(fn, fnDisplayData, insertDefault, fanout);
+      return new Globally<>(fn, fnDisplayData, insertDefault, fanout, 
sideInputs);
     }
 
     /**
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
index 6464437d267..eba0fb7270b 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
@@ -964,6 +964,19 @@ public void testSimpleCombineWithContext() {
     public void testSimpleCombineWithContextEmpty() {
       runTestSimpleCombineWithContext(EMPTY_TABLE, 0, Collections.emptyList(), 
new String[] {});
     }
+
+    @Test
+    public void testWithDefaultsPreservesSideInputs() {
+      final PCollectionView<Integer> view =
+          
pipeline.apply(Create.of(1)).apply(Sum.integersGlobally().asSingletonView());
+
+      Combine.Globally<Integer, String> combine =
+          Combine.globally(new TestCombineFnWithContext(view))
+              .withSideInputs(view)
+              .withoutDefaults();
+
+      assertEquals(Collections.singletonList(view), combine.getSideInputs());
+    }
   }
 
   /** Tests validating windowing behaviors. */


 

----------------------------------------------------------------
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: 154949)
    Time Spent: 0.5h  (was: 20m)

> 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: Mike Pedersen
>            Priority: Critical
>              Labels: beginner, starter
>          Time Spent: 0.5h
>  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