Hi, the root cause seems to be that the Flink streaming runner does not support side-inputs (Views). Inside the CountWords it uses a Combine somewhere which uses side-inputs to sneak in a default value in case there is nothing to combine.
Good news is that I have almost finished work on making side-inputs work in Flink. I'll let you know once that is in master. Cheers, Aljoscha On Mon, 25 Jul 2016 at 00:33 Emanuele Cesena <[email protected]> wrote: > Hi, > > I’ve tried creating a small stream word count with FlinkRunner and KafkaIO. > I get the exception below. > > Is this related to: > https://issues.apache.org/jira/browse/BEAM-102 > (in which case I’ll wait) or am I doing something wrong? > > Thank you! > > > Code is here: > > https://github.com/ecesena/beam-starter/blob/master/src/main/java/com/dataradiant/beam/examples/StreamWordCount.java#L138 > > Exception details: > org.apache.flink.client.program.ProgramInvocationException: The main > method caused an error. > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:520) > at > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:403) > at > org.apache.flink.client.program.OptimizerPlanEnvironment.getOptimizedPlan(OptimizerPlanEnvironment.java:80) > at > org.apache.flink.client.program.Client.getOptimizedPlan(Client.java:215) > at > org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.getJobGraphAndClassLoader(JarActionHandler.java:95) > at > org.apache.flink.runtime.webmonitor.handlers.JarPlanHandler.handleRequest(JarPlanHandler.java:42) > at > org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.respondAsLeader(RuntimeMonitorHandler.java:135) > at > org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.channelRead0(RuntimeMonitorHandler.java:112) > at > org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.channelRead0(RuntimeMonitorHandler.java:60) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > at > io.netty.handler.codec.http.router.Handler.routed(Handler.java:62) > at > io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:57) > at > io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:20) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > at > org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:104) > at > org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:65) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242) > at > io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:147) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > at > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > at > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) > at > io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.IllegalStateException: Unable to create a side-input > view from input > at > org.apache.beam.sdk.transforms.View$AsIterable.validate(View.java:276) > at > org.apache.beam.sdk.transforms.View$AsIterable.validate(View.java:267) > at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:400) > at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:308) > at > org.apache.beam.sdk.values.PCollection.apply(PCollection.java:154) > at org.apache.beam.sdk.io.Write$Bound.createWrite(Write.java:381) > at org.apache.beam.sdk.io.Write$Bound.apply(Write.java:111) > at org.apache.beam.sdk.io.Write$Bound.apply(Write.java:98) > at > org.apache.beam.sdk.runners.PipelineRunner.apply(PipelineRunner.java:76) > at > org.apache.beam.runners.flink.FlinkRunner.apply(FlinkRunner.java:140) > at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:401) > at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:324) > at > org.apache.beam.sdk.values.PCollection.apply(PCollection.java:167) > at org.apache.beam.sdk.io.TextIO$Write$Bound.apply(TextIO.java:617) > at org.apache.beam.sdk.io.TextIO$Write$Bound.apply(TextIO.java:463) > at > org.apache.beam.sdk.runners.PipelineRunner.apply(PipelineRunner.java:76) > at > org.apache.beam.runners.flink.FlinkRunner.apply(FlinkRunner.java:140) > at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:401) > at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:324) > at > org.apache.beam.sdk.values.PCollection.apply(PCollection.java:167) > at > com.dataradiant.beam.examples.StreamWordCount.main(StreamWordCount.java:138) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:505) > ... 35 more > Caused by: java.lang.IllegalStateException: GroupByKey cannot be applied > to non-bounded PCollection in the GlobalWindow without a trigger. Use a > Window.into or Window.triggering transform prior to GroupByKey. > at > org.apache.beam.sdk.transforms.GroupByKey.applicableTo(GroupByKey.java:173) > at > org.apache.beam.sdk.transforms.View$AsIterable.validate(View.java:274) > ... 60 more > > > -- > Emanuele Cesena, Data Eng. > http://www.shopkick.com > > Il corpo non ha ideali > > > > >
