Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 0C46C200B3C for ; Wed, 13 Jul 2016 20:57:51 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0AD6F160A6E; Wed, 13 Jul 2016 18:57:51 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 84487160A6A for ; Wed, 13 Jul 2016 20:57:49 +0200 (CEST) Received: (qmail 26128 invoked by uid 500); 13 Jul 2016 18:57:48 -0000 Mailing-List: contact user-help@beam.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@beam.incubator.apache.org Delivered-To: mailing list user@beam.incubator.apache.org Received: (qmail 26117 invoked by uid 99); 13 Jul 2016 18:57:48 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 Jul 2016 18:57:48 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 37977C9D39 for ; Wed, 13 Jul 2016 18:57:48 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.108 X-Spam-Level: X-Spam-Status: No, score=-0.108 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_LOW=-0.7, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.287, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=google.com Received: from mx2-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id UUGtUk5kVQEZ for ; Wed, 13 Jul 2016 18:57:45 +0000 (UTC) Received: from mail-oi0-f41.google.com (mail-oi0-f41.google.com [209.85.218.41]) by mx2-lw-eu.apache.org (ASF Mail Server at mx2-lw-eu.apache.org) with ESMTPS id DF57C5FB2C for ; Wed, 13 Jul 2016 18:57:44 +0000 (UTC) Received: by mail-oi0-f41.google.com with SMTP id w18so84217316oiw.3 for ; Wed, 13 Jul 2016 11:57:44 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20120113; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=d6fUscI0m9Lm0kRtONwNwI1eaPJBtaM10Tk4fW8rtA0=; b=PyD8zUZ/PwUEFB1p7H2u+GKCfDJwSdWuwdQa0vvecijZCqS0UJdMrwGprwVW20aclZ 3uGMqnuidI3JzP1zl3TNVT26iRDYc/xDlxRLFzZUepkW+d7bLdbOe63kgb0msHl2kgQ7 pNFJr8D+carODjIJhMiqZIWJRA4gyBZAk8BK0dxGK38bnMiGQmXe0r/8n+21JXZ1PkEc bcxxVaj2JCKLgEG2zAN4/fL5k12l9dmYSNVz+RCNPXHJBmRZmFxHlK+KhAjeIUTqttja x+MNN7HewEYNLKSYh9Jtn7rUaJif6AnqOByBT+CL2CJ5cTJLU2obmhmPBRQ9jp8XqwQV NmOQ== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20130820; h=x-gm-message-state:mime-version:in-reply-to:references:from:date :message-id:subject:to; bh=d6fUscI0m9Lm0kRtONwNwI1eaPJBtaM10Tk4fW8rtA0=; b=Fx9axjtfx2YAoLvxOLpTHTR4jPo2/I0OTazyJZI66z3T8edR+L4wkALMEkxQJhPXiQ oOshZQde1oXJEEx1KQQj0jcARPWUrxDF1SRvOk7L4BncoT8D/ZHo0fbydgQCOat7fKsP 7U5GvKiZ4RzmPtMGxYGrgOU/nn66b6Ok+HJUXQGwQ3G0+A27WAgrF3cSEQIgyt1Waowv BtBQYXcckcDv9Z6mqRrQ+slCZpiTVBbWDQFYUM1jCydyfnRwg5tE/1W+3ErzjrcGs7Lp qGB1pRIgdTIcHezHlGSO4fyVCpP3YQwxNCfFb9gkCEROjbYc7GTcMiwFF9W3PvQxF+yB 6hOA== X-Gm-Message-State: ALyK8tIOuWpo4vlyjDSXeSjWoKbYyziyOYvzEqQTn/oLhZ9mxm2sWmPRucdiZU8Q1K0Cv4Q1QBoZlsbOdL5GrsKE X-Received: by 10.157.4.21 with SMTP id 21mr5890378otc.182.1468436263209; Wed, 13 Jul 2016 11:57:43 -0700 (PDT) MIME-Version: 1.0 Received: by 10.157.16.25 with HTTP; Wed, 13 Jul 2016 11:57:42 -0700 (PDT) In-Reply-To: References: <94CA9455-074C-44CC-BD22-63A0DAC93301@shopkick.com> <576B7514.4020600@nanthrax.net> <92E16832-6632-442A-B9E1-27C33CD903B0@shopkick.com> From: Lukasz Cwik Date: Wed, 13 Jul 2016 14:57:42 -0400 Message-ID: Subject: Re: A quick demo of Apache Beam with Docker To: user@beam.incubator.apache.org Content-Type: multipart/alternative; boundary=001a113bf8ecd87321053788f56d archived-at: Wed, 13 Jul 2016 18:57:51 -0000 --001a113bf8ecd87321053788f56d Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable If you find out that Flink was setting the context class loader on the thread, then PipelineOptionsFactory could be updated to use it for proxy generation. On Wed, Jul 13, 2016 at 2:50 PM, Isma=C3=ABl Mej=C3=ADa = wrote: > Indeed I think it is a classloader problem, I am going to try to debug it > with the info you sent me, and I expect also to get some help from the > Flink guys, because I have the impression it could be something related t= o > their webapp that uses a different classpath (I don't know the inner > details of flink to check this). > > Thanks. > > > On Wed, Jul 13, 2016 at 6:26 PM, Lukasz Cwik wrote: > >> Could it be that multiple class loaders may be in play? >> >> PipelineOptionsFactory defaults to use the class loader that it was >> loaded by to create the proxies which could differ from the options >> interface class. >> >> Can you see who is the class loader for WordCountOptions and who is the >> class loader for PipelineOptionsFactory? >> >> https://docs.oracle.com/javase/7/docs/api/java/lang/Class.html#getClassL= oader() >> >> PipelineOptionsFactory proxy creation could be updated to use the thread >> context class loader similar to how it does the service loading here: >> >> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/= main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java#L518 >> >> >> On Wed, Jul 13, 2016 at 12:02 PM, Isma=C3=ABl Mej=C3=ADa wrote: >> >>> It was already public too, I made everything public and I also tried >>> with a jar with only the WordCount example and I had a similar excepti= on: >>> >>> Caused by: java.lang.IllegalArgumentException: interface >>> org.apache.beam.examples.WordCount$WordCountOptions is not visible from >>> class loader >>> at java.lang.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:581) >>> at java.lang.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:557) >>> at java.lang.reflect.WeakCache$Factory.get(WeakCache.java:230) >>> at java.lang.reflect.WeakCache.get(WeakCache.java:127) >>> at java.lang.reflect.Proxy.getProxyClass0(Proxy.java:419) >>> at java.lang.reflect.Proxy.getProxyClass(Proxy.java:371) >>> at >>> org.apache.beam.sdk.options.PipelineOptionsFactory.validateWellFormed(P= ipelineOptionsFactory.java:620) >>> at >>> org.apache.beam.sdk.options.PipelineOptionsFactory.parseObjects(Pipelin= eOptionsFactory.java:1374) >>> at >>> org.apache.beam.sdk.options.PipelineOptionsFactory.access$400(PipelineO= ptionsFactory.java:107) >>> at >>> org.apache.beam.sdk.options.PipelineOptionsFactory$Builder.as(PipelineO= ptionsFactory.java:292) >>> at org.apache.beam.examples.WordCount.main(WordCount.java:191) >>> >>> >>> >>> >>> On Wed, Jul 13, 2016 at 3:32 PM, Lukasz Cwik wrote: >>> >>>> I believe I had run into this before and it was because the outer clas= s >>>> was not public so to make sure, you >>>> made org.apache.beam.samples.EventsByLocation public? >>>> >>>> Have you tried moving >>>> the org.apache.beam.samples.EventsByLocation$Options inner class to it= s own >>>> file? >>>> >>>> >>>> On Wed, Jul 13, 2016 at 4:43 AM, Isma=C3=ABl Mej=C3=ADa >>>> wrote: >>>> >>>>> For context: >>>>> >>>>> I just changed the beam-runners-flink_2.10 pom file to add the shade >>>>> plugin to >>>>> pack the flink runner dependencies in an uber jar, and exclude the >>>>> flink classes >>>>> (that I assume are present on flink. My goal is to create a fat jar >>>>> and add it to >>>>> $FLINK_HOME/lib so I can deploy any pipeline on flink (with Beam >>>>> included): >>>>> >>>>> >>>>> org.apache.maven.plugins >>>>> maven-shade-plugin >>>>> 2.4.1 >>>>> >>>>> >>>>> package >>>>> >>>>> shade >>>>> >>>>> >>>>> >>>>> >>>>> org.apache.flink:force-shading >>>>> >>>>> org.apache.flink:flink-clients_2.10 >>>>> >>>>> org.apache.flink:flink-optimizer_2.10 >>>>> org.apache.flink:flink-java >>>>> >>>>> org.apache.flink:flink-shaded-hadoop2 >>>>> >>>>> org.apache.flink:flink-streaming-java_2.10 >>>>> org.apache.flink:flink-core >>>>> org.apache.flink:flink-annotations >>>>> >>>>> org.apache.flink:flink-runtime_2.10 >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> Then I build the beam-flink-runner uber jar: >>>>> >>>>> mvn clean package -DskipTests >>>>> >>>>> And I copy the produced jar into the $FLINK_HOME/lib directory, >>>>> however when I try to submit a >>>>> simple test jar, via the Flink Web UI I get this exception: >>>>> >>>>> org.apache.flink.client.program.ProgramInvocationException: The main >>>>> method caused an error. >>>>> at >>>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(Packag= edProgram.java:520) >>>>> at >>>>> org.apache.flink.client.program.PackagedProgram.invokeInteractiveMode= ForExecution(PackagedProgram.java:403) >>>>> at >>>>> org.apache.flink.client.program.OptimizerPlanEnvironment.getOptimized= Plan(OptimizerPlanEnvironment.java:80) >>>>> at >>>>> org.apache.flink.client.program.Client.getOptimizedPlan(Client.java:2= 15) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.getJobG= raphAndClassLoader(JarActionHandler.java:95) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.handlers.JarPlanHandler.handleReq= uest(JarPlanHandler.java:42) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.respondAsLe= ader(RuntimeMonitorHandler.java:135) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.channelRead= 0(RuntimeMonitorHandler.java:112) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.channelRead= 0(RuntimeMonitorHandler.java:60) >>>>> at >>>>> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChanne= lInboundHandler.java:105) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(Abst= ractChannelHandlerContext.java:339) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(Abstra= ctChannelHandlerContext.java:324) >>>>> at >>>>> io.netty.handler.codec.http.router.Handler.routed(Handler.java:62) >>>>> at >>>>> io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(D= ualAbstractHandler.java:57) >>>>> at >>>>> io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(D= ualAbstractHandler.java:20) >>>>> at >>>>> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChanne= lInboundHandler.java:105) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(Abst= ractChannelHandlerContext.java:339) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(Abstra= ctChannelHandlerContext.java:324) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(H= ttpRequestHandler.java:104) >>>>> at >>>>> org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(H= ttpRequestHandler.java:65) >>>>> at >>>>> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChanne= lInboundHandler.java:105) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(Abst= ractChannelHandlerContext.java:339) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(Abstra= ctChannelHandlerContext.java:324) >>>>> at >>>>> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessage= Decoder.java:242) >>>>> at >>>>> io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedCha= nnelDuplexHandler.java:147) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(Abst= ractChannelHandlerContext.java:339) >>>>> at >>>>> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(Abstra= ctChannelHandlerContext.java:324) >>>>> at >>>>> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChanne= lPipeline.java:847) >>>>> at >>>>> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(Abstra= ctNioByteChannel.java:131) >>>>> at >>>>> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.jav= a:511) >>>>> at >>>>> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEve= ntLoop.java:468) >>>>> at >>>>> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.ja= va:382) >>>>> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) >>>>> at >>>>> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThread= EventExecutor.java:111) >>>>> at >>>>> io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorato= r.run(DefaultThreadFactory.java:137) >>>>> at java.lang.Thread.run(Thread.java:745) >>>>> Caused by: java.lang.IllegalArgumentException: interface >>>>> org.apache.beam.samples.EventsByLocation$Options is not visible from = class >>>>> loader >>>>> at java.lang.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:581= ) >>>>> at java.lang.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:557= ) >>>>> at java.lang.reflect.WeakCache$Factory.get(WeakCache.java:230) >>>>> at java.lang.reflect.WeakCache.get(WeakCache.java:127) >>>>> at java.lang.reflect.Proxy.getProxyClass0(Proxy.java:419) >>>>> at java.lang.reflect.Proxy.getProxyClass(Proxy.java:371) >>>>> at >>>>> org.apache.beam.sdk.options.PipelineOptionsFactory.validateWellFormed= (PipelineOptionsFactory.java:620) >>>>> at >>>>> org.apache.beam.sdk.options.PipelineOptionsFactory.parseObjects(Pipel= ineOptionsFactory.java:1374) >>>>> at >>>>> org.apache.beam.sdk.options.PipelineOptionsFactory.access$400(Pipelin= eOptionsFactory.java:107) >>>>> at >>>>> org.apache.beam.sdk.options.PipelineOptionsFactory$Builder.as(Pipelin= eOptionsFactory.java:292) >>>>> at >>>>> org.apache.beam.samples.EventsByLocation.main(EventsByLocation.java:6= 2) >>>>> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) >>>>> at >>>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.= java:62) >>>>> at >>>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAcces= sorImpl.java:43) >>>>> at java.lang.reflect.Method.invoke(Method.java:498) >>>>> at >>>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(Packag= edProgram.java:505) >>>>> ... 35 more >>>>> >>>>> Sorry for the long exception in the message but I wanted to show all >>>>> possible details so you can help me fix this. Notice that the interfa= ce >>>>> org.apache.beam.samples.EventsByLocation$Options is a classical Beam >>>>> Options interface created to configure the Pipeline, and I tried maki= ng it >>>>> public but access does not seem to be the case, but classloading. >>>>> >>>>> Thanks, >>>>> Ismael >>>>> >>>>> ps. >>>>> I saw this and I think maybe it could be related: >>>>> >>>>> https://stackoverflow.com/questions/211176/interface-is-not-visible-f= rom-classloader-when-using-a-proxy >>>>> >>>>> >>>>> On Thu, Jul 7, 2016 at 11:30 PM, Isma=C3=ABl Mej=C3=ADa >>>>> wrote: >>>>> >>>>>> =E2=80=8BBoth Max and Alojscha, thanks for writing, I have been a bi= t busy >>>>>> these last >>>>>> days with other Beam stuff and I have not been able to continue >>>>>> testing my Beam >>>>>> on flink integration, I will write back to you as soon as I have tim= e >>>>>> the check >>>>>> this again, thanks. >>>>>> >>>>>> >>>>> >>>> >>> >> > --001a113bf8ecd87321053788f56d Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
If you find out that Flink was setting the context class l= oader on the thread, then PipelineOptionsFactory could be updated to use it= for proxy generation.

On Wed, Jul 13, 2016 at 2:50 PM, Isma=C3=ABl Mej=C3=ADa <iemeji= a@gmail.com> wrote:
= Indeed I think it is a classloader problem, I am going to try to debug it w= ith the info you sent me, and I expect also to get some help from the Flink= guys, because I have the impression it could be something related to their= webapp that uses a different classpath (I don't know the inner details= of flink to check this).

Thanks.


On Wed, Jul 13, 2016 at 6:26 PM, Lukasz Cwik <lcwik@google.com> wrote:
Could it be = that multiple class loaders may be in play?

PipelineOpti= onsFactory defaults to use the class loader that it was loaded by to create= the proxies which could differ from the options interface class.

Can you see who is the class loader for WordCountOptions an= d who is the class loader for PipelineOptionsFactory?

PipelineOptio= nsFactory proxy creation could be updated to use the thread context class l= oader similar to how it does the service loading here:


On Wed, Jul 13, 2016 at 12:02 PM, Isma=C3= =ABl Mej=C3=ADa <iemejia@gmail.com> wrote:
It was already public too, I made everything public= and I also tried with a=C2=A0 jar with only the WordCount example and I ha= d a similar exception:

Caused by: java.lang.IllegalArgumentException= : interface org.apache.beam.examples.WordCount$WordCountOptions is not visi= ble from class loader
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.Prox= y$ProxyClassFactory.apply(Proxy.java:581)
=C2=A0=C2=A0=C2=A0 at java.lan= g.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:557)
=C2=A0=C2=A0=C2= =A0 at java.lang.reflect.WeakCache$Factory.get(WeakCache.java:230)
=C2= =A0=C2=A0=C2=A0 at java.lang.reflect.WeakCache.get(WeakCache.java:127)
= =C2=A0=C2=A0=C2=A0 at java.lang.reflect.Proxy.getProxyClass0(Proxy.java:419= )
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.Proxy.getProxyClass(Proxy.java= :371)
=C2=A0=C2=A0=C2=A0 at org.apache.beam.sdk.options.PipelineOptionsF= actory.validateWellFormed(PipelineOptionsFactory.java:620)
=C2=A0=C2=A0= =C2=A0 at org.apache.beam.sdk.options.PipelineOptionsFactory.parseObjects(P= ipelineOptionsFactory.java:1374)
=C2=A0=C2=A0=C2=A0 at org.apache.beam.s= dk.options.PipelineOptionsFactory.access$400(PipelineOptionsFactory.java:10= 7)
=C2=A0=C2=A0=C2=A0 at org.apache.beam.sdk.options.PipelineOptionsFact= ory$Builder.as(PipelineOptionsFactory.java:292)
=C2=A0=C2=A0=C2= =A0 at org.apache.beam.examples.WordCount.main(WordCount.java:191)

<= /div>

<= br>

On Wed, Jul 13, 2016 at 3:32 PM, Lukasz Cwik <= lcwik@google.com&= gt; wrote:
I= believe I had run into this before and it was because the outer class was = not public so to make sure, you made=C2=A0org.apache.beam.samples.EventsByL= ocation public?

Have you tried moving the=C2=A0org= .apache.beam.samples.EventsByLocation$Options inner class to its own file?<= /div>


On Wed, Jul 13, 2016 at 4:43 AM, Isma=C3=ABl Mej=C3=ADa = <iemejia@gmail.com> wrote:
For context:

I just changed the beam-runners-flink_2.10 pom = file to add the shade plugin to=C2=A0=C2=A0=C2=A0
pack the flink runner= dependencies in an uber jar, and exclude the flink classes
(that I assu= me are present on flink. My goal is to create a fat jar and add it to
$F= LINK_HOME/lib so I can deploy any pipeline on flink (with Beam included):
=C2=A0=C2=A0=C2=A0 <plugin>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0 <groupId>org.apache.maven.plugins</groupId>
=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <artifactId>maven-shade-plugi= n</artifactId>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <vers= ion>2.4.1</version>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 = <executions>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0 <execution>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 <phase>package</phase>
=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <goals>
=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <g= oal>shade</goal>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0 </goals>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <configuration>
=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <artifac= tSet>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0 <excludes>
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= <exclude>org.apache.flink:force-shading</exclude>
=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 <exclude>org.apache.flink:flink-clients_2.10</e= xclude>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <exclude>org.apache.flink:= flink-optimizer_2.10</exclude>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <e= xclude>org.apache.flink:flink-java</exclude>
=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 <exclude>org.apache.flink:flink-shaded-hadoop2</exclude&= gt;
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <exclude>org.apache.flink:flink-= streaming-java_2.10</exclude>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <excl= ude>org.apache.flink:flink-core</exclude>
=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0 <exclude>org.apache.flink:flink-annotations</exclude>=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 <exclude>org.apache.flink:flink-runtim= e_2.10</exclude>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 </excludes>
=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 </= artifactSet>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0 </configuration>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 </execution>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 </executions>
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 </plugin= >=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0

Then I build the beam-flink-runner u= ber jar:

=C2=A0=C2=A0=C2=A0 mvn clean package -DskipTests

And= I copy the produced jar into the $FLINK_HOME/lib directory, however when I= try to submit a
simple test jar, via the Flink Web UI I get this excep= tion:

org.apache.flink.client.program.ProgramInvocationException: Th= e main method caused an error.
=C2=A0=C2=A0=C2=A0 at org.apache.flink.cl= ient.program.PackagedProgram.callMainMethod(PackagedProgram.java:520)
= =C2=A0=C2=A0=C2=A0 at org.apache.flink.client.program.PackagedProgram.invok= eInteractiveModeForExecution(PackagedProgram.java:403)
=C2=A0=C2=A0=C2= =A0 at org.apache.flink.client.program.OptimizerPlanEnvironment.getOptimize= dPlan(OptimizerPlanEnvironment.java:80)
=C2=A0=C2=A0=C2=A0 at org.apache= .flink.client.program.Client.getOptimizedPlan(Client.java:215)
=C2=A0=C2= =A0=C2=A0 at org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.= getJobGraphAndClassLoader(JarActionHandler.java:95)
=C2=A0=C2=A0=C2=A0 a= t org.apache.flink.runtime.webmonitor.handlers.JarPlanHandler.handleRequest= (JarPlanHandler.java:42)
=C2=A0=C2=A0=C2=A0 at org.apache.flink.runtime.= webmonitor.RuntimeMonitorHandler.respondAsLeader(RuntimeMonitorHandler.java= :135)
=C2=A0=C2=A0=C2=A0 at org.apache.flink.runtime.webmonitor.RuntimeM= onitorHandler.channelRead0(RuntimeMonitorHandler.java:112)
=C2=A0=C2=A0= =C2=A0 at org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler.channel= Read0(RuntimeMonitorHandler.java:60)
=C2=A0=C2=A0=C2=A0 at io.netty.chan= nel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.jav= a:105)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.AbstractChannelHandlerCont= ext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
=C2=A0=C2= =A0=C2=A0 at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead= (AbstractChannelHandlerContext.java:324)
=C2=A0=C2=A0=C2=A0 at io.netty.= handler.codec.http.router.Handler.routed(Handler.java:62)
=C2=A0=C2=A0= =C2=A0 at io.netty.handler.codec.http.router.DualAbstractHandler.channelRea= d0(DualAbstractHandler.java:57)
=C2=A0=C2=A0=C2=A0 at io.netty.handler.c= odec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:= 20)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.SimpleChannelInboundHandler.c= hannelRead(SimpleChannelInboundHandler.java:105)
=C2=A0=C2=A0=C2=A0 at i= o.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractCha= nnelHandlerContext.java:339)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.Abst= ractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.jav= a:324)
=C2=A0=C2=A0=C2=A0 at org.apache.flink.runtime.webmonitor.HttpReq= uestHandler.channelRead0(HttpRequestHandler.java:104)
=C2=A0=C2=A0=C2=A0= at org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(Htt= pRequestHandler.java:65)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.SimpleCh= annelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
= =C2=A0=C2=A0=C2=A0 at io.netty.channel.AbstractChannelHandlerContext.invoke= ChannelRead(AbstractChannelHandlerContext.java:339)
=C2=A0=C2=A0=C2=A0 a= t io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractCh= annelHandlerContext.java:324)
=C2=A0=C2=A0=C2=A0 at io.netty.handler.cod= ec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242)
=C2= =A0=C2=A0=C2=A0 at io.netty.channel.CombinedChannelDuplexHandler.channelRea= d(CombinedChannelDuplexHandler.java:147)
=C2=A0=C2=A0=C2=A0 at io.netty.= channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHand= lerContext.java:339)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.AbstractChan= nelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)=C2=A0=C2=A0=C2=A0 at io.netty.channel.DefaultChannelPipeline.fireChannel= Read(DefaultChannelPipeline.java:847)
=C2=A0=C2=A0=C2=A0 at io.netty.cha= nnel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.j= ava:131)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.nio.NioEventLoop.process= SelectedKey(NioEventLoop.java:511)
=C2=A0=C2=A0=C2=A0 at io.netty.channe= l.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
= =C2=A0=C2=A0=C2=A0 at io.netty.channel.nio.NioEventLoop.processSelectedKeys= (NioEventLoop.java:382)
=C2=A0=C2=A0=C2=A0 at io.netty.channel.nio.NioEv= entLoop.run(NioEventLoop.java:354)
=C2=A0=C2=A0=C2=A0 at io.netty.util.c= oncurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:11= 1)
=C2=A0=C2=A0=C2=A0 at io.netty.util.concurrent.DefaultThreadFactory$D= efaultRunnableDecorator.run(DefaultThreadFactory.java:137)
=C2=A0=C2=A0= =C2=A0 at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.Ill= egalArgumentException: interface org.apache.beam.samples.EventsByLocation$O= ptions is not visible from class loader
=C2=A0=C2=A0=C2=A0 at java.lang.= reflect.Proxy$ProxyClassFactory.apply(Proxy.java:581)
=C2=A0=C2=A0=C2=A0= at java.lang.reflect.Proxy$ProxyClassFactory.apply(Proxy.java:557)
=C2= =A0=C2=A0=C2=A0 at java.lang.reflect.WeakCache$Factory.get(WeakCache.java:2= 30)
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.WeakCache.get(WeakCache.java= :127)
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.Proxy.getProxyClass0(Proxy= .java:419)
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.Proxy.getProxyClass(P= roxy.java:371)
=C2=A0=C2=A0=C2=A0 at org.apache.beam.sdk.options.Pipelin= eOptionsFactory.validateWellFormed(PipelineOptionsFactory.java:620)
=C2= =A0=C2=A0=C2=A0 at org.apache.beam.sdk.options.PipelineOptionsFactory.parse= Objects(PipelineOptionsFactory.java:1374)
=C2=A0=C2=A0=C2=A0 at org.apac= he.beam.sdk.options.PipelineOptionsFactory.access$400(PipelineOptionsFactor= y.java:107)
=C2=A0=C2=A0=C2=A0 at org.apache.beam.sdk.options.PipelineOp= tionsFactory$Builder.as(PipelineOptionsFactory.java:292)
=C2=A0=C2=A0=C2= =A0 at org.apache.beam.samples.EventsByLocation.main(EventsByLocation.java:= 62)
=C2=A0=C2=A0=C2=A0 at sun.reflect.NativeMethodAccessorImpl.invoke0(N= ative Method)
=C2=A0=C2=A0=C2=A0 at sun.reflect.NativeMethodAccessorImpl= .invoke(NativeMethodAccessorImpl.java:62)
=C2=A0=C2=A0=C2=A0 at sun.refl= ect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:4= 3)
=C2=A0=C2=A0=C2=A0 at java.lang.reflect.Method.invoke(Method.java:498= )
=C2=A0=C2=A0=C2=A0 at org.apache.flink.client.program.PackagedProgram.= callMainMethod(PackagedProgram.java:505)
=C2=A0=C2=A0=C2=A0 ... 35 more<= br>
Sorry for the long exception in the message but I wanted to show all= possible details so you can help me fix this. Notice that the interface or= g.apache.beam.samples.EventsByLocation$Options is a classical Beam Options = interface created to configure the Pipeline, and I tried making it public b= ut access does not seem to be the case, but classloading.

Thanks,Ismael

ps.
I saw this and I think maybe it could be related:
= https://stackove= rflow.com/questions/211176/interface-is-not-visible-from-classloader-when-u= sing-a-proxy

On Thu, Jul 7, 2016 at 11:30 PM, Isma=C3=ABl M= ej=C3=ADa <iemejia@gmail.com> wrote:
=E2=80=8BBoth Max and Alojscha, thanks for writing, I hav= e been a bit busy these last
days with other Beam stuff and I have not b= een able to continue testing my Beam
on flink integration, I will write = back to you as soon as I have time the check
this again, thanks.

=






--001a113bf8ecd87321053788f56d--