NoClassDefFoundError with ElasticsearchSink on Yarn

classic Classic list List threaded Threaded
5 messages Options
Reply | Threaded
Open this post in threaded view
|

NoClassDefFoundError with ElasticsearchSink on Yarn

Steffen Hausmann
Hi there,

I’m running a flink program that reads from a Kinesis stream and
eventually writes to an Elasticsearch2 sink. When I’m running the
program locally from the IDE, everything seems to work fine, but when
I’m executing the same program on an EMR cluster with Yarn, a
NoClassDefFoundError occurs:java.lang.NoSuchMethodError:

com.google.common.util.concurrent.MoreExecutors.directExecutor()Ljava/util/concurrent/Executor;
        at
org.elasticsearch.threadpool.ThreadPool.<clinit>(ThreadPool.java:190)
        at
org.elasticsearch.client.transport.TransportClient$Builder.build(TransportClient.java:133)
        at
org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:164)
        at
org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
        at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:256)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)

I’ve installed flink on an EMR cluster from the binary distribution
flink-1.1.1-bin-hadoop27-scala_2.10.tgzand the jar file that is executed
on the cluster is build with mvn clean package(I’ve attached the pom.xml
for reference).

There is a thread on this list that seems to be related, but I’m afraid
I couldn’t draw any conclusions from it:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/classpath-issue-on-yarn-tt6442.html#none

Any idea, what’s wrong?

Thanks,
Steffen


pom.xml (18K) Download Attachment
Reply | Threaded
Open this post in threaded view
|

Re: NoClassDefFoundError with ElasticsearchSink on Yarn

aris kol

Classic problem of every uber-jar containing Hadoop dependencies and being deployed on Yarn.

What actually happens is that some Hadoop dependency relies on an old version of guava (11 in this case), which doesn't have the method. You may have assembled your fat-jar properly, but because Hadoop deps get introduced to your classpath before your own, you invoke the method using the guava 11 version of the class.

I fixed that by adding this line:


++ Seq(assemblyShadeRules in assembly := Seq(ShadeRule.rename("com.google.common.**" -> "shaded.@1").inAll))


on the artefact that gets deployed on flink.

What this practically does is to shade the guava dependencies. Instead of containing references to com.google.common your build will reference shaded.com.google.common and as a result it will use the proper class in your fat jar.
Get a bit creative with the name (ie use shadedhausmann instead of shaded) to avoid colliding with external deps shading stuff (something you have to do when using guava, joda, jackson etc).

Let me know if this helped.

Aris


From: Steffen Hausmann <[hidden email]>
Sent: Thursday, September 1, 2016 8:58 AM
To: [hidden email]
Subject: NoClassDefFoundError with ElasticsearchSink on Yarn
 
Hi there,

I’m running a flink program that reads from a Kinesis stream and
eventually writes to an Elasticsearch2 sink. When I’m running the
program locally from the IDE, everything seems to work fine, but when
I’m executing the same program on an EMR cluster with Yarn, a
NoClassDefFoundError occurs:java.lang.NoSuchMethodError:

com.google.common.util.concurrent.MoreExecutors.directExecutor()Ljava/util/concurrent/Executor;
        at
org.elasticsearch.threadpool.ThreadPool.<clinit>(ThreadPool.java:190)
        at
org.elasticsearch.client.transport.TransportClient$Builder.build(TransportClient.java:133)
        at
org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:164)
        at
org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
        at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:256)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)

I’ve installed flink on an EMR cluster from the binary distribution
flink-1.1.1-bin-hadoop27-scala_2.10.tgzand the jar file that is executed
on the cluster is build with mvn clean package(I’ve attached the pom.xml
for reference).

There is a thread on this list that seems to be related, but I’m afraid
I couldn’t draw any conclusions from it:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/classpath-issue-on-yarn-tt6442.html#none

Any idea, what’s wrong?

Thanks,
Steffen

Reply | Threaded
Open this post in threaded view
|

Re: NoClassDefFoundError with ElasticsearchSink on Yarn

Steffen Hausmann
Thanks Aris for your explanation!

A guava version mismatch was indeed the problem. But in addition to
shading the guava dependencies, I encountered another issue caused by
conflicting files in META-INF/services:

> RemoteTransportException[[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]]; nested: TransportSerializationException[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]; nested: ExceptionInInitializerError; nested: IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist.  You need to add the corresponding JAR file supporting this SPI to your classpath.  The current classpath supports the following names: [es090, completion090, XBloomFilter]];

By adding the following bits to my pom.xml file, the conflicting files
are appended instead of overwritten and hence the ElasticSearch sink
works as expected:

> <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
>     <resource>META-INF/services/org.apache.lucene.codecs.Codec</resource>
> </transformer>
> <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
>     <resource>META-INF/services/org.apache.lucene.codecs.DocValuesFormat</resource>
> </transformer>
> <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
>    <resource>META-INF/services/org.apache.lucene.codecs.PostingsFormat</resource>
> </transformer>

Maybe this is something that can be added to the documentation?

Thanks,
Steffen

On 01/09/2016 12:22, aris kol wrote:

> Classic problem of every uber-jar containing Hadoop dependencies and
> being deployed on Yarn.
>
> What actually happens is that some Hadoop dependency relies on an
> old version of guava (11 in this case), which doesn't have the method.
> You may have assembled your fat-jar properly, but because Hadoop deps
> get introduced to your classpath before your own, you invoke the method
> using the guava 11 version of the class.
>
> I fixed that by adding this line:
>
>
> ++ Seq(assemblyShadeRules in assembly :=
> Seq(ShadeRule.rename("com.google.common.**" -> "shaded.@1").inAll))
>
> on the artefact that gets deployed on flink.
>
> What this practically does is to shade the guava dependencies. Instead
> of containing references to com.google.common your build will reference
> shaded.com.google.common and as a result it will use the proper class in
> your fat jar.
> Get a bit creative with the name (ie use shadedhausmann instead of
> shaded) to avoid colliding with external deps shading stuff (something
> you have to do when using guava, joda, jackson etc).
>
> Let me know if this helped.
>
> Aris
>
> ------------------------------------------------------------------------
> *From:* Steffen Hausmann <[hidden email]>
> *Sent:* Thursday, September 1, 2016 8:58 AM
> *To:* [hidden email]
> *Subject:* NoClassDefFoundError with ElasticsearchSink on Yarn
>
> Hi there,
>
> I’m running a flink program that reads from a Kinesis stream and
> eventually writes to an Elasticsearch2 sink. When I’m running the
> program locally from the IDE, everything seems to work fine, but when
> I’m executing the same program on an EMR cluster with Yarn, a
> NoClassDefFoundError occurs:java.lang.NoSuchMethodError:
>
> com.google.common.util.concurrent.MoreExecutors.directExecutor()Ljava/util/concurrent/Executor;
>         at
> org.elasticsearch.threadpool.ThreadPool.<clinit>(ThreadPool.java:190)
>         at
> org.elasticsearch.client.transport.TransportClient$Builder.build(TransportClient.java:133)
>         at
> org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:164)
>         at
> org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
>         at
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:256)
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>         at java.lang.Thread.run(Thread.java:745)
>
> I’ve installed flink on an EMR cluster from the binary distribution
> flink-1.1.1-bin-hadoop27-scala_2.10.tgzand the jar file that is executed
> on the cluster is build with mvn clean package(I’ve attached the pom.xml
> for reference).
>
> There is a thread on this list that seems to be related, but I’m afraid
> I couldn’t draw any conclusions from it:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/classpath-issue-on-yarn-tt6442.html#none
>
> Any idea, what’s wrong?
>
> Thanks,
> Steffen
>
Reply | Threaded
Open this post in threaded view
|

Re: NoClassDefFoundError with ElasticsearchSink on Yarn

rmetzger0
Hi Steffen,

I think it would be good to add it to the documentation.
Would you like to open a pull request?


Regards,
Robert


On Mon, Sep 5, 2016 at 10:26 PM, Steffen Hausmann <[hidden email]> wrote:
Thanks Aris for your explanation!

A guava version mismatch was indeed the problem. But in addition to shading the guava dependencies, I encountered another issue caused by conflicting files in META-INF/services:

RemoteTransportException[[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]]; nested: TransportSerializationException[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]; nested: ExceptionInInitializerError; nested: IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist.  You need to add the corresponding JAR file supporting this SPI to your classpath.  The current classpath supports the following names: [es090, completion090, XBloomFilter]];

By adding the following bits to my pom.xml file, the conflicting files are appended instead of overwritten and hence the ElasticSearch sink works as expected:

<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
    <resource>META-INF/services/org.apache.lucene.codecs.Codec</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
    <resource>META-INF/services/org.apache.lucene.codecs.DocValuesFormat</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
   <resource>META-INF/services/org.apache.lucene.codecs.PostingsFormat</resource>
</transformer>

Maybe this is something that can be added to the documentation?

Thanks,
Steffen

On 01/09/2016 12:22, aris kol wrote:
Classic problem of every uber-jar containing Hadoop dependencies and
being deployed on Yarn.

What actually happens is that some Hadoop dependency relies on an
old version of guava (11 in this case), which doesn't have the method.
You may have assembled your fat-jar properly, but because Hadoop deps
get introduced to your classpath before your own, you invoke the method
using the guava 11 version of the class.

I fixed that by adding this line:


++ Seq(assemblyShadeRules in assembly :=
Seq(ShadeRule.rename("com.google.common.**" -> "shaded.@1").inAll))

on the artefact that gets deployed on flink.

What this practically does is to shade the guava dependencies. Instead
of containing references to com.google.common your build will reference
shaded.com.google.common and as a result it will use the proper class in
your fat jar.
Get a bit creative with the name (ie use shadedhausmann instead of
shaded) to avoid colliding with external deps shading stuff (something
you have to do when using guava, joda, jackson etc).

Let me know if this helped.

Aris

------------------------------------------------------------------------
*From:* Steffen Hausmann <[hidden email]>
*Sent:* Thursday, September 1, 2016 8:58 AM
*To:* [hidden email]
*Subject:* NoClassDefFoundError with ElasticsearchSink on Yarn


Hi there,

I’m running a flink program that reads from a Kinesis stream and
eventually writes to an Elasticsearch2 sink. When I’m running the
program locally from the IDE, everything seems to work fine, but when
I’m executing the same program on an EMR cluster with Yarn, a
NoClassDefFoundError occurs:java.lang.NoSuchMethodError:

com.google.common.util.concurrent.MoreExecutors.directExecutor()Ljava/util/concurrent/Executor;
        at
org.elasticsearch.threadpool.ThreadPool.<clinit>(ThreadPool.java:190)
        at
org.elasticsearch.client.transport.TransportClient$Builder.build(TransportClient.java:133)
        at
org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:164)
        at
org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
        at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:256)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)

I’ve installed flink on an EMR cluster from the binary distribution
flink-1.1.1-bin-hadoop27-scala_2.10.tgzand the jar file that is executed
on the cluster is build with mvn clean package(I’ve attached the pom.xml
for reference).

There is a thread on this list that seems to be related, but I’m afraid
I couldn’t draw any conclusions from it:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/classpath-issue-on-yarn-tt6442.html#none

Any idea, what’s wrong?

Thanks,
Steffen


Reply | Threaded
Open this post in threaded view
|

Re: NoClassDefFoundError with ElasticsearchSink on Yarn

Fabian Hueske-2
+1
I ran into that issue as well. Would be great to have that in the docs!

2016-09-09 11:49 GMT+02:00 Robert Metzger <[hidden email]>:
Hi Steffen,

I think it would be good to add it to the documentation.
Would you like to open a pull request?


Regards,
Robert


On Mon, Sep 5, 2016 at 10:26 PM, Steffen Hausmann <[hidden email]> wrote:
Thanks Aris for your explanation!

A guava version mismatch was indeed the problem. But in addition to shading the guava dependencies, I encountered another issue caused by conflicting files in META-INF/services:

RemoteTransportException[[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]]; nested: TransportSerializationException[Failed to deserialize response of type [org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse]]; nested: ExceptionInInitializerError; nested: IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist.  You need to add the corresponding JAR file supporting this SPI to your classpath.  The current classpath supports the following names: [es090, completion090, XBloomFilter]];

By adding the following bits to my pom.xml file, the conflicting files are appended instead of overwritten and hence the ElasticSearch sink works as expected:

<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
    <resource>META-INF/services/org.apache.lucene.codecs.Codec</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
    <resource>META-INF/services/org.apache.lucene.codecs.DocValuesFormat</resource>
</transformer>
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
   <resource>META-INF/services/org.apache.lucene.codecs.PostingsFormat</resource>
</transformer>

Maybe this is something that can be added to the documentation?

Thanks,
Steffen

On 01/09/2016 12:22, aris kol wrote:
Classic problem of every uber-jar containing Hadoop dependencies and
being deployed on Yarn.

What actually happens is that some Hadoop dependency relies on an
old version of guava (11 in this case), which doesn't have the method.
You may have assembled your fat-jar properly, but because Hadoop deps
get introduced to your classpath before your own, you invoke the method
using the guava 11 version of the class.

I fixed that by adding this line:


++ Seq(assemblyShadeRules in assembly :=
Seq(ShadeRule.rename("com.google.common.**" -> "shaded.@1").inAll))

on the artefact that gets deployed on flink.

What this practically does is to shade the guava dependencies. Instead
of containing references to com.google.common your build will reference
shaded.com.google.common and as a result it will use the proper class in
your fat jar.
Get a bit creative with the name (ie use shadedhausmann instead of
shaded) to avoid colliding with external deps shading stuff (something
you have to do when using guava, joda, jackson etc).

Let me know if this helped.

Aris

------------------------------------------------------------------------
*From:* Steffen Hausmann <[hidden email]>
*Sent:* Thursday, September 1, 2016 8:58 AM
*To:* [hidden email]
*Subject:* NoClassDefFoundError with ElasticsearchSink on Yarn


Hi there,

I’m running a flink program that reads from a Kinesis stream and
eventually writes to an Elasticsearch2 sink. When I’m running the
program locally from the IDE, everything seems to work fine, but when
I’m executing the same program on an EMR cluster with Yarn, a
NoClassDefFoundError occurs:java.lang.NoSuchMethodError:

com.google.common.util.concurrent.MoreExecutors.directExecutor()Ljava/util/concurrent/Executor;
        at
org.elasticsearch.threadpool.ThreadPool.<clinit>(ThreadPool.java:190)
        at
org.elasticsearch.client.transport.TransportClient$Builder.build(TransportClient.java:133)
        at
org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink.open(ElasticsearchSink.java:164)
        at
org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:38)
        at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:91)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376)
        at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:256)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)

I’ve installed flink on an EMR cluster from the binary distribution
flink-1.1.1-bin-hadoop27-scala_2.10.tgzand the jar file that is executed
on the cluster is build with mvn clean package(I’ve attached the pom.xml
for reference).

There is a thread on this list that seems to be related, but I’m afraid
I couldn’t draw any conclusions from it:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/classpath-issue-on-yarn-tt6442.html#none

Any idea, what’s wrong?

Thanks,
Steffen