You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Austin Cawley-Edwards <au...@gmail.com> on 2019/02/27 03:59:21 UTC

okio and okhttp not shaded in the Flink Uber Jar on EMR

Hi,

I recently experienced versioning clashes with the okio and okhttp when
trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
investigating and talking to the okio team (see this issue)
<https://github.com/square/okio/issues/559>, I found that both okio and
okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
jar. The okio team suggested that Flink should shade the uber jar to fix
the issue, but I'm wondering if there is something I can do on my end to
have all versions exist simultaneously.

From the issue, here are the okio contents of the uber jar:

*jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*

META-INF/maven/com.squareup.okio/
META-INF/maven/com.squareup.okio/okio/
META-INF/maven/com.squareup.okio/okio/pom.properties
META-INF/maven/com.squareup.okio/okio/pom.xml
okio/
okio/AsyncTimeout$1.class
okio/AsyncTimeout$2.class
okio/AsyncTimeout$Watchdog.class
okio/AsyncTimeout.class
okio/Base64.class
okio/Buffer$1.class
okio/Buffer$2.class
okio/Buffer.class
okio/BufferedSink.class
okio/BufferedSource.class
okio/ByteString.class
okio/DeflaterSink.class
okio/ForwardingSink.class
okio/ForwardingSource.class
okio/ForwardingTimeout.class
okio/GzipSink.class
okio/GzipSource.class
okio/InflaterSource.class
okio/Okio$1.class
okio/Okio$2.class
okio/Okio$3.class
okio/Okio.class
okio/RealBufferedSink$1.class
okio/RealBufferedSink.class
okio/RealBufferedSource$1.class
okio/RealBufferedSource.class
okio/Segment.class
okio/SegmentPool.class
okio/SegmentedByteString.class
okio/Sink.class
okio/Source.class
okio/Timeout$1.class
okio/Timeout.class
okio/Util.class

Thank you,
Austin Cawley-Edwards

Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Chesnay Schepler <ch...@apache.org>.
There is no (accurate) reference of included dependencies for the 
flink-shaded-hadoop uber jars. The contained NOTICE file is a good 
starting point, but for the time being we're using a generalized version 
that we apply to all hadoop versions (so some things may be missing).

I believe for hadoop 2.8.4 the following dependencies are contained, 
based on the output of the maven-shade-plugin:
     com.github.stephenc.jcip:jcip-annotations:1.0-1
     com.google.code.gson:gson:2.2.4
     com.google.guava:guava:11.0.2
     com.google.guava:guava:11.0.2
     com.google.protobuf:protobuf-java:2.5.0
     com.google.protobuf:protobuf-java:2.5.0
     com.jamesmurty.utils:java-xmlbuilder:0.4
     com.jcraft:jsch:0.1.54
     com.nimbusds:nimbus-jose-jwt:4.41.1
     com.squareup.okhttp:okhttp:2.4.0
     com.squareup.okio:okio:1.4.0
     com.sun.jersey:jersey-client:1.9
     com.thoughtworks.paranamer:paranamer:2.7
     commons-beanutils:commons-beanutils:1.9.3
     commons-cli:commons-cli:1.3.1
     commons-codec:commons-codec:1.10
     commons-collections:commons-collections:3.2.2
     commons-configuration:commons-configuration:1.7
     commons-daemon:commons-daemon:1.0.13
     commons-digester:commons-digester:1.8.1
     commons-io:commons-io:2.4
     commons-lang:commons-lang:2.6
     commons-logging:commons-logging:1.1.3
     commons-net:commons-net:3.1
     io.netty:netty-all:4.0.23.Final
     io.netty:netty-all:4.0.23.Final
     io.netty:netty:3.6.2.Final
     io.netty:netty:3.6.2.Final
     javax.activation:activation:1.1
     javax.servlet:servlet-api:2.5
     javax.xml.bind:jaxb-api:2.2.2
     javax.xml.stream:stax-api:1.0-2
     net.java.dev.jets3t:jets3t:0.9.0
     net.java.dev.jets3t:jets3t:0.9.0
     net.minidev:accessors-smart:1.2
     net.minidev:json-smart:2.3
     org.apache.avro:avro:1.8.2
     org.apache.commons:commons-compress:1.18
     org.apache.commons:commons-math3:3.5
     org.apache.curator:curator-client:2.7.1
     org.apache.curator:curator-client:2.7.1
     org.apache.curator:curator-framework:2.7.1
     org.apache.curator:curator-framework:2.7.1
     org.apache.curator:curator-recipes:2.7.1
     org.apache.curator:curator-recipes:2.7.1
     org.apache.directory.api:api-asn1-api:1.0.0-M20
     org.apache.directory.api:api-util:1.0.0-M20
     org.apache.directory.server:apacheds-i18n:2.0.0-M15
     org.apache.directory.server:apacheds-kerberos-codec:2.0.0-M15
     org.apache.hadoop:hadoop-annotations:2.8.4
     org.apache.hadoop:hadoop-annotations:2.8.4
     org.apache.hadoop:hadoop-auth:2.8.4
     org.apache.hadoop:hadoop-auth:2.8.4
     org.apache.hadoop:hadoop-common:2.8.4
     org.apache.hadoop:hadoop-common:2.8.4
     org.apache.hadoop:hadoop-hdfs-client:2.8.4
     org.apache.hadoop:hadoop-hdfs-client:2.8.4
     org.apache.hadoop:hadoop-hdfs:2.8.4
     org.apache.hadoop:hadoop-hdfs:2.8.4
     org.apache.hadoop:hadoop-mapreduce-client-core:2.8.4
     org.apache.hadoop:hadoop-mapreduce-client-core:2.8.4
     org.apache.hadoop:hadoop-yarn-api:2.8.4
     org.apache.hadoop:hadoop-yarn-api:2.8.4
     org.apache.hadoop:hadoop-yarn-client:2.8.4
     org.apache.hadoop:hadoop-yarn-client:2.8.4
     org.apache.hadoop:hadoop-yarn-common:2.8.4
     org.apache.hadoop:hadoop-yarn-common:2.8.4
     org.apache.htrace:htrace-core4:4.0.1-incubating
     org.apache.httpcomponents:httpclient:4.5.3
     org.apache.httpcomponents:httpclient:4.5.3
     org.apache.httpcomponents:httpcore:4.4.6
     org.apache.httpcomponents:httpcore:4.4.6
     org.apache.zookeeper:zookeeper:3.4.10
     org.codehaus.jackson:jackson-core-asl:1.9.13
     org.codehaus.jackson:jackson-core-asl:1.9.13
     org.codehaus.jackson:jackson-jaxrs:1.9.13
     org.codehaus.jackson:jackson-jaxrs:1.9.13
     org.codehaus.jackson:jackson-mapper-asl:1.9.13
     org.codehaus.jackson:jackson-mapper-asl:1.9.13
     org.codehaus.jackson:jackson-xc:1.9.13
     org.codehaus.jackson:jackson-xc:1.9.13
     org.fusesource.leveldbjni:leveldbjni-all:1.8
     org.mortbay.jetty:jetty-sslengine:6.1.26
     org.tukaani:xz:1.5
     org.xerial.snappy:snappy-java:1.1.4
     xerces:xercesImpl:2.9.1
     xml-apis:xml-apis:1.3.04
     xmlenc:xmlenc:0.52

On 28.02.2019 15:54, Austin Cawley-Edwards wrote:
> Hi Gary,
>
> No, I am running a YARN session (which I start 
> with: flink-yarn-session --slots 4 --taskManagerMemory 16GB 
> --jobManagerMemory 3GB --detached) and submit jobs through the REST 
> interface. Thank you for the tips - I will probably shade it on my 
> side. Is there an official location that the uber jar dependencies are 
> documented that I can reference for future dependency additions?
>
> Best,
> Austin
>
> On Thu, Feb 28, 2019 at 7:13 AM Gary Yao <gary@ververica.com 
> <ma...@ververica.com>> wrote:
>
>     Hi Austin,
>
>     Are you running your job detached in a per-job cluster? In that
>     case inverted
>     class loading does not work. This is because we add the user jar
>     to the system
>     class path, and there is no dynamic class loading involved at the
>     moment [1].
>
>     You can try the YARN session mode, or – as Chesnay already
>     suggested – shade
>     the dependency on your side.
>
>     Best,
>     Gary
>
>     [1]
>     https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#overview-of-classloading-in-flink
>
>
>     On Wed, Feb 27, 2019 at 8:57 PM Austin Cawley-Edwards
>     <austin.cawley@gmail.com <ma...@gmail.com>> wrote:
>
>         Thanks Gary,
>
>         I will try to look into why the child-first strategy seems to
>         have failed for this dependency.
>
>         Best,
>         Austin
>
>         On Wed, Feb 27, 2019 at 12:25 PM Gary Yao <gary@ververica.com
>         <ma...@ververica.com>> wrote:
>
>             Hi,
>
>             Actually Flink's inverted class loading feature was
>             designed to mitigate
>             problems with different versions of libraries that are not
>             compatible with
>             each other [1]. You may want to debug why it does not work
>             for you.
>
>             You can also try to use the Hadoop free Flink
>             distribution, and export the
>             HADOOP_CLASSPATH variable [2].
>
>             Best,
>             Gary
>
>             [1]
>             https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#inverted-class-loading-and-classloader-resolution-order
>             [2]
>             https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/deployment/hadoop.html#configuring-flink-with-hadoop-classpaths
>
>             On Wed, Feb 27, 2019 at 5:23 AM Austin Cawley-Edwards
>             <austin.cawley@gmail.com <ma...@gmail.com>>
>             wrote:
>
>                 Hi,
>
>                 I recently experienced versioning clashes with the
>                 okio and okhttp when trying to deploy a Flink 1.6.0
>                 app to AWS EMR on Hadoop 2.8.4. After investigating
>                 and talking to the okio team (see this issue)
>                 <https://github.com/square/okio/issues/559>, I found
>                 that both okio and okhttp exist in the Flink uber jar
>                 with versions 1.4.0 and 2.4.0, respectively, whereas
>                 I'm including versions 2.2.2 and 3.13.1 in my shaded
>                 jar. The okio team suggested that Flink should shade
>                 the uber jar to fix the issue, but I'm wondering if
>                 there is something I can do on my end to have all
>                 versions exist simultaneously.
>
>                 From the issue, here are the okio contents of the uber
>                 jar:
>
>                 *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>
>                 META-INF/maven/com.squareup.okio/
>                 META-INF/maven/com.squareup.okio/okio/
>                 META-INF/maven/com.squareup.okio/okio/pom.properties
>                 META-INF/maven/com.squareup.okio/okio/pom.xml
>                 okio/
>                 okio/AsyncTimeout$1.class
>                 okio/AsyncTimeout$2.class
>                 okio/AsyncTimeout$Watchdog.class
>                 okio/AsyncTimeout.class
>                 okio/Base64.class
>                 okio/Buffer$1.class
>                 okio/Buffer$2.class
>                 okio/Buffer.class
>                 okio/BufferedSink.class
>                 okio/BufferedSource.class
>                 okio/ByteString.class
>                 okio/DeflaterSink.class
>                 okio/ForwardingSink.class
>                 okio/ForwardingSource.class
>                 okio/ForwardingTimeout.class
>                 okio/GzipSink.class
>                 okio/GzipSource.class
>                 okio/InflaterSource.class
>                 okio/Okio$1.class
>                 okio/Okio$2.class
>                 okio/Okio$3.class
>                 okio/Okio.class
>                 okio/RealBufferedSink$1.class
>                 okio/RealBufferedSink.class
>                 okio/RealBufferedSource$1.class
>                 okio/RealBufferedSource.class
>                 okio/Segment.class
>                 okio/SegmentPool.class
>                 okio/SegmentedByteString.class
>                 okio/Sink.class
>                 okio/Source.class
>                 okio/Timeout$1.class
>                 okio/Timeout.class
>                 okio/Util.class
>
>                 Thank you,
>                 Austin Cawley-Edwards
>


Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Austin Cawley-Edwards <au...@gmail.com>.
Hi Gary,

No, I am running a YARN session (which I start with: flink-yarn-session
--slots 4 --taskManagerMemory 16GB --jobManagerMemory 3GB --detached) and
submit jobs through the REST interface. Thank you for the tips - I will
probably shade it on my side. Is there an official location that the uber
jar dependencies are documented that I can reference for future dependency
additions?

Best,
Austin

On Thu, Feb 28, 2019 at 7:13 AM Gary Yao <ga...@ververica.com> wrote:

> Hi Austin,
>
> Are you running your job detached in a per-job cluster? In that case
> inverted
> class loading does not work. This is because we add the user jar to the
> system
> class path, and there is no dynamic class loading involved at the moment
> [1].
>
> You can try the YARN session mode, or – as Chesnay already suggested –
> shade
> the dependency on your side.
>
> Best,
> Gary
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#overview-of-classloading-in-flink
>
>
> On Wed, Feb 27, 2019 at 8:57 PM Austin Cawley-Edwards <
> austin.cawley@gmail.com> wrote:
>
>> Thanks Gary,
>>
>> I will try to look into why the child-first strategy seems to have failed
>> for this dependency.
>>
>> Best,
>> Austin
>>
>> On Wed, Feb 27, 2019 at 12:25 PM Gary Yao <ga...@ververica.com> wrote:
>>
>>> Hi,
>>>
>>> Actually Flink's inverted class loading feature was designed to mitigate
>>> problems with different versions of libraries that are not compatible
>>> with
>>> each other [1]. You may want to debug why it does not work for you.
>>>
>>> You can also try to use the Hadoop free Flink distribution, and export
>>> the
>>> HADOOP_CLASSPATH variable [2].
>>>
>>> Best,
>>> Gary
>>>
>>> [1]
>>> https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#inverted-class-loading-and-classloader-resolution-order
>>> [2]
>>> https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/deployment/hadoop.html#configuring-flink-with-hadoop-classpaths
>>>
>>> On Wed, Feb 27, 2019 at 5:23 AM Austin Cawley-Edwards <
>>> austin.cawley@gmail.com> wrote:
>>>
>>>> Hi,
>>>>
>>>> I recently experienced versioning clashes with the okio and okhttp when
>>>> trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
>>>> investigating and talking to the okio team (see this issue)
>>>> <https://github.com/square/okio/issues/559>, I found that both okio
>>>> and okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
>>>> respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
>>>> jar. The okio team suggested that Flink should shade the uber jar to fix
>>>> the issue, but I'm wondering if there is something I can do on my end to
>>>> have all versions exist simultaneously.
>>>>
>>>> From the issue, here are the okio contents of the uber jar:
>>>>
>>>> *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>>>>
>>>> META-INF/maven/com.squareup.okio/
>>>> META-INF/maven/com.squareup.okio/okio/
>>>> META-INF/maven/com.squareup.okio/okio/pom.properties
>>>> META-INF/maven/com.squareup.okio/okio/pom.xml
>>>> okio/
>>>> okio/AsyncTimeout$1.class
>>>> okio/AsyncTimeout$2.class
>>>> okio/AsyncTimeout$Watchdog.class
>>>> okio/AsyncTimeout.class
>>>> okio/Base64.class
>>>> okio/Buffer$1.class
>>>> okio/Buffer$2.class
>>>> okio/Buffer.class
>>>> okio/BufferedSink.class
>>>> okio/BufferedSource.class
>>>> okio/ByteString.class
>>>> okio/DeflaterSink.class
>>>> okio/ForwardingSink.class
>>>> okio/ForwardingSource.class
>>>> okio/ForwardingTimeout.class
>>>> okio/GzipSink.class
>>>> okio/GzipSource.class
>>>> okio/InflaterSource.class
>>>> okio/Okio$1.class
>>>> okio/Okio$2.class
>>>> okio/Okio$3.class
>>>> okio/Okio.class
>>>> okio/RealBufferedSink$1.class
>>>> okio/RealBufferedSink.class
>>>> okio/RealBufferedSource$1.class
>>>> okio/RealBufferedSource.class
>>>> okio/Segment.class
>>>> okio/SegmentPool.class
>>>> okio/SegmentedByteString.class
>>>> okio/Sink.class
>>>> okio/Source.class
>>>> okio/Timeout$1.class
>>>> okio/Timeout.class
>>>> okio/Util.class
>>>>
>>>> Thank you,
>>>> Austin Cawley-Edwards
>>>>
>>>

Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Gary Yao <ga...@ververica.com>.
Hi Austin,

Are you running your job detached in a per-job cluster? In that case
inverted
class loading does not work. This is because we add the user jar to the
system
class path, and there is no dynamic class loading involved at the moment
[1].

You can try the YARN session mode, or – as Chesnay already suggested – shade
the dependency on your side.

Best,
Gary

[1]
https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#overview-of-classloading-in-flink


On Wed, Feb 27, 2019 at 8:57 PM Austin Cawley-Edwards <
austin.cawley@gmail.com> wrote:

> Thanks Gary,
>
> I will try to look into why the child-first strategy seems to have failed
> for this dependency.
>
> Best,
> Austin
>
> On Wed, Feb 27, 2019 at 12:25 PM Gary Yao <ga...@ververica.com> wrote:
>
>> Hi,
>>
>> Actually Flink's inverted class loading feature was designed to mitigate
>> problems with different versions of libraries that are not compatible with
>> each other [1]. You may want to debug why it does not work for you.
>>
>> You can also try to use the Hadoop free Flink distribution, and export the
>> HADOOP_CLASSPATH variable [2].
>>
>> Best,
>> Gary
>>
>> [1]
>> https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#inverted-class-loading-and-classloader-resolution-order
>> [2]
>> https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/deployment/hadoop.html#configuring-flink-with-hadoop-classpaths
>>
>> On Wed, Feb 27, 2019 at 5:23 AM Austin Cawley-Edwards <
>> austin.cawley@gmail.com> wrote:
>>
>>> Hi,
>>>
>>> I recently experienced versioning clashes with the okio and okhttp when
>>> trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
>>> investigating and talking to the okio team (see this issue)
>>> <https://github.com/square/okio/issues/559>, I found that both okio and
>>> okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
>>> respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
>>> jar. The okio team suggested that Flink should shade the uber jar to fix
>>> the issue, but I'm wondering if there is something I can do on my end to
>>> have all versions exist simultaneously.
>>>
>>> From the issue, here are the okio contents of the uber jar:
>>>
>>> *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>>>
>>> META-INF/maven/com.squareup.okio/
>>> META-INF/maven/com.squareup.okio/okio/
>>> META-INF/maven/com.squareup.okio/okio/pom.properties
>>> META-INF/maven/com.squareup.okio/okio/pom.xml
>>> okio/
>>> okio/AsyncTimeout$1.class
>>> okio/AsyncTimeout$2.class
>>> okio/AsyncTimeout$Watchdog.class
>>> okio/AsyncTimeout.class
>>> okio/Base64.class
>>> okio/Buffer$1.class
>>> okio/Buffer$2.class
>>> okio/Buffer.class
>>> okio/BufferedSink.class
>>> okio/BufferedSource.class
>>> okio/ByteString.class
>>> okio/DeflaterSink.class
>>> okio/ForwardingSink.class
>>> okio/ForwardingSource.class
>>> okio/ForwardingTimeout.class
>>> okio/GzipSink.class
>>> okio/GzipSource.class
>>> okio/InflaterSource.class
>>> okio/Okio$1.class
>>> okio/Okio$2.class
>>> okio/Okio$3.class
>>> okio/Okio.class
>>> okio/RealBufferedSink$1.class
>>> okio/RealBufferedSink.class
>>> okio/RealBufferedSource$1.class
>>> okio/RealBufferedSource.class
>>> okio/Segment.class
>>> okio/SegmentPool.class
>>> okio/SegmentedByteString.class
>>> okio/Sink.class
>>> okio/Source.class
>>> okio/Timeout$1.class
>>> okio/Timeout.class
>>> okio/Util.class
>>>
>>> Thank you,
>>> Austin Cawley-Edwards
>>>
>>

Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Austin Cawley-Edwards <au...@gmail.com>.
Thanks Gary,

I will try to look into why the child-first strategy seems to have failed
for this dependency.

Best,
Austin

On Wed, Feb 27, 2019 at 12:25 PM Gary Yao <ga...@ververica.com> wrote:

> Hi,
>
> Actually Flink's inverted class loading feature was designed to mitigate
> problems with different versions of libraries that are not compatible with
> each other [1]. You may want to debug why it does not work for you.
>
> You can also try to use the Hadoop free Flink distribution, and export the
> HADOOP_CLASSPATH variable [2].
>
> Best,
> Gary
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#inverted-class-loading-and-classloader-resolution-order
> [2]
> https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/deployment/hadoop.html#configuring-flink-with-hadoop-classpaths
>
> On Wed, Feb 27, 2019 at 5:23 AM Austin Cawley-Edwards <
> austin.cawley@gmail.com> wrote:
>
>> Hi,
>>
>> I recently experienced versioning clashes with the okio and okhttp when
>> trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
>> investigating and talking to the okio team (see this issue)
>> <https://github.com/square/okio/issues/559>, I found that both okio and
>> okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
>> respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
>> jar. The okio team suggested that Flink should shade the uber jar to fix
>> the issue, but I'm wondering if there is something I can do on my end to
>> have all versions exist simultaneously.
>>
>> From the issue, here are the okio contents of the uber jar:
>>
>> *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>>
>> META-INF/maven/com.squareup.okio/
>> META-INF/maven/com.squareup.okio/okio/
>> META-INF/maven/com.squareup.okio/okio/pom.properties
>> META-INF/maven/com.squareup.okio/okio/pom.xml
>> okio/
>> okio/AsyncTimeout$1.class
>> okio/AsyncTimeout$2.class
>> okio/AsyncTimeout$Watchdog.class
>> okio/AsyncTimeout.class
>> okio/Base64.class
>> okio/Buffer$1.class
>> okio/Buffer$2.class
>> okio/Buffer.class
>> okio/BufferedSink.class
>> okio/BufferedSource.class
>> okio/ByteString.class
>> okio/DeflaterSink.class
>> okio/ForwardingSink.class
>> okio/ForwardingSource.class
>> okio/ForwardingTimeout.class
>> okio/GzipSink.class
>> okio/GzipSource.class
>> okio/InflaterSource.class
>> okio/Okio$1.class
>> okio/Okio$2.class
>> okio/Okio$3.class
>> okio/Okio.class
>> okio/RealBufferedSink$1.class
>> okio/RealBufferedSink.class
>> okio/RealBufferedSource$1.class
>> okio/RealBufferedSource.class
>> okio/Segment.class
>> okio/SegmentPool.class
>> okio/SegmentedByteString.class
>> okio/Sink.class
>> okio/Source.class
>> okio/Timeout$1.class
>> okio/Timeout.class
>> okio/Util.class
>>
>> Thank you,
>> Austin Cawley-Edwards
>>
>

Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Gary Yao <ga...@ververica.com>.
Hi,

Actually Flink's inverted class loading feature was designed to mitigate
problems with different versions of libraries that are not compatible with
each other [1]. You may want to debug why it does not work for you.

You can also try to use the Hadoop free Flink distribution, and export the
HADOOP_CLASSPATH variable [2].

Best,
Gary

[1]
https://ci.apache.org/projects/flink/flink-docs-release-1.6/monitoring/debugging_classloading.html#inverted-class-loading-and-classloader-resolution-order
[2]
https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/deployment/hadoop.html#configuring-flink-with-hadoop-classpaths

On Wed, Feb 27, 2019 at 5:23 AM Austin Cawley-Edwards <
austin.cawley@gmail.com> wrote:

> Hi,
>
> I recently experienced versioning clashes with the okio and okhttp when
> trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
> investigating and talking to the okio team (see this issue)
> <https://github.com/square/okio/issues/559>, I found that both okio and
> okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
> respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
> jar. The okio team suggested that Flink should shade the uber jar to fix
> the issue, but I'm wondering if there is something I can do on my end to
> have all versions exist simultaneously.
>
> From the issue, here are the okio contents of the uber jar:
>
> *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>
> META-INF/maven/com.squareup.okio/
> META-INF/maven/com.squareup.okio/okio/
> META-INF/maven/com.squareup.okio/okio/pom.properties
> META-INF/maven/com.squareup.okio/okio/pom.xml
> okio/
> okio/AsyncTimeout$1.class
> okio/AsyncTimeout$2.class
> okio/AsyncTimeout$Watchdog.class
> okio/AsyncTimeout.class
> okio/Base64.class
> okio/Buffer$1.class
> okio/Buffer$2.class
> okio/Buffer.class
> okio/BufferedSink.class
> okio/BufferedSource.class
> okio/ByteString.class
> okio/DeflaterSink.class
> okio/ForwardingSink.class
> okio/ForwardingSource.class
> okio/ForwardingTimeout.class
> okio/GzipSink.class
> okio/GzipSource.class
> okio/InflaterSource.class
> okio/Okio$1.class
> okio/Okio$2.class
> okio/Okio$3.class
> okio/Okio.class
> okio/RealBufferedSink$1.class
> okio/RealBufferedSink.class
> okio/RealBufferedSource$1.class
> okio/RealBufferedSource.class
> okio/Segment.class
> okio/SegmentPool.class
> okio/SegmentedByteString.class
> okio/Sink.class
> okio/Source.class
> okio/Timeout$1.class
> okio/Timeout.class
> okio/Util.class
>
> Thank you,
> Austin Cawley-Edwards
>

Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Chesnay Schepler <ch...@apache.org>.
Short-term I'd try relocating the okio/okhttp dependencies in your jar.

I'm not too keen on adding more relocations to the hadoop jar; I can't 
gauge the possible side-effects.

On 27.02.2019 14:54, Austin Cawley-Edwards wrote:
> Following up to add more info, I am building my app with maven based 
> on the sample Flink pom.xml
>
> My shade plugin config is:
> <groupId>org.apache.maven.plugins</groupId> <artifactId>maven-shade-plugin</artifactId> <version>3.0.0</version> <executions> <!-- Run shade goal on package phase --> 
> <execution> <phase>package</phase> <goals> <goal>shade</goal> </goals> <configuration> <artifactSet> <excludes> <exclude>org.apache.flink:force-shading</exclude> <exclude>com.google.code.findbugs:jsr305</exclude> <exclude>org.slf4j:*</exclude> <exclude>log4j:*</exclude> </excludes> </artifactSet> <filters> <filter> <!-- Do not 
> copy the signatures in the META-INF folder. Otherwise, this might 
> cause SecurityExceptions when using the JAR. --> <artifact>*:*</artifact> <excludes> <exclude>META-INF/*.SF</exclude> <exclude>META-INF/*.DSA</exclude> <exclude>META-INF/*.RSA</exclude> </excludes> </filter> </filters> <transformers> 
> <transformer 
> implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer"> 
> <mainClass>${main.class}</mainClass> </transformer> </transformers> </configuration> 
> </execution> </executions>
>
> Thanks again and let me know if there is more information I can provide.
>
> Austin
>
> On Tue, Feb 26, 2019 at 10:59 PM Austin Cawley-Edwards 
> <austin.cawley@gmail.com <ma...@gmail.com>> wrote:
>
>     Hi,
>
>     I recently experienced versioning clashes with the okio and
>     okhttp when trying to deploy a Flink 1.6.0 app to AWS EMR on
>     Hadoop 2.8.4. After investigating and talking to the okio team
>     (see this issue) <https://github.com/square/okio/issues/559>, I
>     found that both okio and okhttp exist in the Flink uber jar with
>     versions 1.4.0 and 2.4.0, respectively, whereas I'm including
>     versions 2.2.2 and 3.13.1 in my shaded jar. The okio team
>     suggested that Flink should shade the uber jar to fix the issue,
>     but I'm wondering if there is something I can do on my end to have
>     all versions exist simultaneously.
>
>     From the issue, here are the okio contents of the uber jar:
>
>     *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>
>     META-INF/maven/com.squareup.okio/
>     META-INF/maven/com.squareup.okio/okio/
>     META-INF/maven/com.squareup.okio/okio/pom.properties
>     META-INF/maven/com.squareup.okio/okio/pom.xml
>     okio/
>     okio/AsyncTimeout$1.class
>     okio/AsyncTimeout$2.class
>     okio/AsyncTimeout$Watchdog.class
>     okio/AsyncTimeout.class
>     okio/Base64.class
>     okio/Buffer$1.class
>     okio/Buffer$2.class
>     okio/Buffer.class
>     okio/BufferedSink.class
>     okio/BufferedSource.class
>     okio/ByteString.class
>     okio/DeflaterSink.class
>     okio/ForwardingSink.class
>     okio/ForwardingSource.class
>     okio/ForwardingTimeout.class
>     okio/GzipSink.class
>     okio/GzipSource.class
>     okio/InflaterSource.class
>     okio/Okio$1.class
>     okio/Okio$2.class
>     okio/Okio$3.class
>     okio/Okio.class
>     okio/RealBufferedSink$1.class
>     okio/RealBufferedSink.class
>     okio/RealBufferedSource$1.class
>     okio/RealBufferedSource.class
>     okio/Segment.class
>     okio/SegmentPool.class
>     okio/SegmentedByteString.class
>     okio/Sink.class
>     okio/Source.class
>     okio/Timeout$1.class
>     okio/Timeout.class
>     okio/Util.class
>
>     Thank you,
>     Austin Cawley-Edwards
>


Re: okio and okhttp not shaded in the Flink Uber Jar on EMR

Posted by Austin Cawley-Edwards <au...@gmail.com>.
Following up to add more info, I am building my app with maven based on the
sample Flink pom.xml

My shade plugin config is:

    <groupId>org.apache.maven.plugins</groupId>
    <artifactId>maven-shade-plugin</artifactId>
    <version>3.0.0</version>
    <executions>
        <!-- Run shade goal on package phase -->
        <execution>
            <phase>package</phase>
            <goals>
                <goal>shade</goal>
            </goals>
            <configuration>
                <artifactSet>
                    <excludes>
                        <exclude>org.apache.flink:force-shading</exclude>
                        <exclude>com.google.code.findbugs:jsr305</exclude>
                        <exclude>org.slf4j:*</exclude>
                        <exclude>log4j:*</exclude>
                    </excludes>
                </artifactSet>
                <filters>
                    <filter>
                        <!-- Do not copy the signatures in the META-INF folder.
                        Otherwise, this might cause SecurityExceptions
when using the JAR. -->
                        <artifact>*:*</artifact>
                        <excludes>
                            <exclude>META-INF/*.SF</exclude>
                            <exclude>META-INF/*.DSA</exclude>
                            <exclude>META-INF/*.RSA</exclude>
                        </excludes>
                    </filter>
                </filters>
                <transformers>
                    <transformer

implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
                        <mainClass>${main.class}</mainClass>
                    </transformer>
                </transformers>
            </configuration>
        </execution>
    </executions>


Thanks again and let me know if there is more information I can provide.

Austin

On Tue, Feb 26, 2019 at 10:59 PM Austin Cawley-Edwards <
austin.cawley@gmail.com> wrote:

> Hi,
>
> I recently experienced versioning clashes with the okio and okhttp when
> trying to deploy a Flink 1.6.0 app to AWS EMR on Hadoop 2.8.4. After
> investigating and talking to the okio team (see this issue)
> <https://github.com/square/okio/issues/559>, I found that both okio and
> okhttp exist in the Flink uber jar with versions 1.4.0 and 2.4.0,
> respectively, whereas I'm including versions 2.2.2 and 3.13.1 in my shaded
> jar. The okio team suggested that Flink should shade the uber jar to fix
> the issue, but I'm wondering if there is something I can do on my end to
> have all versions exist simultaneously.
>
> From the issue, here are the okio contents of the uber jar:
>
> *jar -tf flink-shaded-hadoop2-uber-1.6.0.jar | grep okio*
>
> META-INF/maven/com.squareup.okio/
> META-INF/maven/com.squareup.okio/okio/
> META-INF/maven/com.squareup.okio/okio/pom.properties
> META-INF/maven/com.squareup.okio/okio/pom.xml
> okio/
> okio/AsyncTimeout$1.class
> okio/AsyncTimeout$2.class
> okio/AsyncTimeout$Watchdog.class
> okio/AsyncTimeout.class
> okio/Base64.class
> okio/Buffer$1.class
> okio/Buffer$2.class
> okio/Buffer.class
> okio/BufferedSink.class
> okio/BufferedSource.class
> okio/ByteString.class
> okio/DeflaterSink.class
> okio/ForwardingSink.class
> okio/ForwardingSource.class
> okio/ForwardingTimeout.class
> okio/GzipSink.class
> okio/GzipSource.class
> okio/InflaterSource.class
> okio/Okio$1.class
> okio/Okio$2.class
> okio/Okio$3.class
> okio/Okio.class
> okio/RealBufferedSink$1.class
> okio/RealBufferedSink.class
> okio/RealBufferedSource$1.class
> okio/RealBufferedSource.class
> okio/Segment.class
> okio/SegmentPool.class
> okio/SegmentedByteString.class
> okio/Sink.class
> okio/Source.class
> okio/Timeout$1.class
> okio/Timeout.class
> okio/Util.class
>
> Thank you,
> Austin Cawley-Edwards
>