Yes, but I think Sessions uses IntervalWindows, so that should not cause an exception.
On Thu, 9 Feb 2017 at 17:17 Ismaël Mejía <[email protected]> wrote: > Hi Aljoscha, > > It is a SessionWindow. I am going to try to create a simpler Test (the > actual code is full of extra quirks to copy here), but in the meantime, > this is actually intended (from looking at the flink runner source code): > > > https://github.com/apache/beam/blob/master/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java#L247 > > > Thanks, > Ismaël. > > > On Thu, Feb 9, 2017 at 4:17 PM, Aljoscha Krettek <[email protected]> > wrote: > > Hi, > what window type is this using? I think Sessions WindowFn should in fact > produce IntervalWindows. > > Cheers, > Aljoscha > > On Wed, 8 Feb 2017 at 12:16 Ismaël Mejía <[email protected]> wrote: > > Hello, > > I was testing a pipeline that produces SessionWindows and then calculates > a Mean afterwards in 'batch' mode and I found this issue while running with > the Flink Runner. > > 17/02/08 09:27:24 INFO > org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator: | > | | | visitPrimitiveTransform- 19b75b2bGroupByKey > Exception in thread "main" java.lang.UnsupportedOperationException: > Merging WindowFn with windows other than IntervalWindow are not supported. > at > org.apache.beam.runners.flink.translation.FlinkBatchTransformTranslators$GroupByKeyTranslatorBatch.translateNode(FlinkBatchTransformTranslators.java:247) > at > org.apache.beam.runners.flink.translation.FlinkBatchTransformTranslators$GroupByKeyTranslatorBatch.translateNode(FlinkBatchTransformTranslators.java:180) > at > org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator.applyBatchTransform(FlinkBatchPipelineTranslator.java:116) > > I rapidly look through the JIRAs, and I could not find if there was a JIRA > tracking this case, is there any work on this ? Is there a conceptual > problem on flink to do this, or it is just that some work is needed ? > > Thanks, > Ismaël > > >
