[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-11-09 Thread Erik van Oosten (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16245474#comment-16245474
 ] 

Erik van Oosten commented on FLINK-5633:


bq. Just curious, why are you creating a new reader for each record?

Its just a bit easier then caching a reader for each writer/reader schema 
combination.

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
>   at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>   at 
> 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-11-07 Thread Stephan Ewen (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16242418#comment-16242418
 ] 

Stephan Ewen commented on FLINK-5633:
-

[~erikvanoosten] Okay, I assume that the reveres class loading should fix that.

Just curious, why are you creating a new reader for each record?

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
>   at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>   at 
> 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-11-07 Thread Erik van Oosten (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16241701#comment-16241701
 ] 

Erik van Oosten commented on FLINK-5633:


[~StephanEwen] We need to process 130K msg/s, I guess that can be called often 
:) . Our process is CPU bound and parsing Avro is ±15% of that. Any improvement 
means we can run with fewer machines.

For every message we create a new SpecificDatumReader. If I follow the code 
correctly that should _not_ give a large overhead. The Schema instances we pass 
to it _are_ cached.

Then we call {SpecificDatumReader.read}} to parse each Avro message. In that 
call you eventually end up in {{SpecificData.newInstance}} to create a new 
instance of the target class. The constructor of that class is looked up in a 
cache. That cache is declared as {{static}}. I do not understand how 
instantiating a new {{SpecificData}} for every call to {{read}} helps because 
it would still use the same cache. The code I pasted above also uses a 
constructor cache but the cache is not {{static}}. Reversing the class loader 
order should also work.

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-11-06 Thread Stephan Ewen (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16240218#comment-16240218
 ] 

Stephan Ewen commented on FLINK-5633:
-

Few updates here
  - Flink 1.4 will actually integrate Avro properly with Flink's type 
information, meaning that for Avro types in-flight, schema will be transported 
via the serializer's parameters
  - Flink 1.4 by default also inverts class loading order. If Avro is part of 
the user code's dependencies, then there is no JVM wide singleton any more, but 
only a per-job singleton. However, that requires that Avro is strictly part of 
the user code.

[~erikvanoosten] In what cases do you use that SpecificData? Can you elaborate 
a bit more? As far as I know, Avro does not reconstruct the schema every time, 
but only when you create a DatumReader or DatumWriter. Are you doing that very 
often such that schema caching is critical?

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-08-17 Thread Svend Vanderveken (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16130917#comment-16130917
 ] 

Svend Vanderveken commented on FLINK-5633:
--

Hi all, 

I am hitting exactly the same issue, although I am not using the 
`SpecificDatumReader` directly but rather using Confluent's Avro deserializer. 
We opted for this one since it has the advantage of being integrated with the 
Schema registry, and thus validating the schemas at runtime. 

I am not the first person to encounter this exact situation and I found and 
commented the  PR below, which essentially suggests to apply the fix mentioned 
by [~gcaliari] to that deserializer.

https://github.com/confluentinc/schema-registry/pull/509

[~StephanEwen], [~gcaliari], feel free to add comments on that PR if you think 
it should move forward, I think such kind of update would greatly help to 
integrate Kafka and Flink through Avro and the Schema Registry. 


> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-06-21 Thread Erik van Oosten (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16058706#comment-16058706
 ] 

Erik van Oosten commented on FLINK-5633:


In case you need throughput (like we do), the caching is indispensable. In 
those cases you can use the following {{SpecificData}} implementation. Simply 
instantiate it once and then pass that singleton instance to every 
{{SpecificDatumReader}}.

{code:scala|title=LocalCachingSpecificData.scala}
import java.lang.reflect.Constructor
import java.util.concurrent.ConcurrentHashMap

import org.apache.avro.Schema
import org.apache.avro.specific.SpecificData
import scala.collection.JavaConverters._

/**
  * This can be used instead of [[SpecificData]] in multi-classloader 
environments like Flink.
  * This variation removes the JVM singleton constructor cache and replaces it 
with a
  * cache that is local to the current class loader.
  *
  * If two Flink jobs use the same generated Avro code, they will still have 
separate instances of the classes because
  * they live in separate class loaders.
  * However, a JVM-wide singleton cache keeps reference to the class in the 
first class loader that was loaded. Any
  * subsequent jobs will fail with a [[ClassCastException]] because they will 
get incompatible classes.
  */
class LocalCachingSpecificData extends SpecificData {
  private val NO_ARG: Array[Class[_]] = Array.empty
  private val SCHEMA_ARG: Array[Class[_]] = Array(classOf[Schema])
  private val CTOR_CACHE: scala.collection.concurrent.Map[Class[_], 
Constructor[_]] =
new ConcurrentHashMap[Class[_], Constructor[_]]().asScala

  /** Create an instance of a class.
* If the class implements 
[[org.apache.avro.specific.SpecificData.SchemaConstructable]], call a 
constructor with a
* [[org.apache.avro.Schema]] parameter, otherwise use a no-arg constructor.
*/
  private def newInstance(c: Class[_], s: Schema): AnyRef = {
val useSchema = 
classOf[SpecificData.SchemaConstructable].isAssignableFrom(c)
val constructor = CTOR_CACHE.getOrElseUpdate(c, {
  val ctor = c.getDeclaredConstructor((if (useSchema) SCHEMA_ARG else 
NO_ARG): _*)
  ctor.setAccessible(true)
  ctor
})
if (useSchema) constructor.newInstance(s).asInstanceOf[AnyRef]
else constructor.newInstance().asInstanceOf[AnyRef]
  }

  override def createFixed(old: AnyRef, schema: Schema): AnyRef = {
val c = getClass(schema)
if (c == null) super.createFixed(old, schema) // delegate to generic
else if (c.isInstance(old)) old
else newInstance(c, schema)
  }

  override def newRecord(old: AnyRef, schema: Schema): AnyRef = {
val c = getClass(schema)
if (c == null) super.newRecord(old, schema) // delegate to generic
else if (c.isInstance(old)) {old }
else {newInstance(c, schema) }
  }
}
{code}

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 

[jira] [Commented] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-01-25 Thread Stephan Ewen (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15837475#comment-15837475
 ] 

Stephan Ewen commented on FLINK-5633:
-

This is covered in that section of the 
docs:https://ci.apache.org/projects/flink/flink-docs-release-1.2/monitoring/debugging_classloading.html#x-cannot-be-cast-to-x-exceptions

It is an "unavoidable issue" sessions. If this helps with the problem, can you 
close the issue?

> ClassCastException: X cannot be cast to X when re-submitting a job.
> ---
>
> Key: FLINK-5633
> URL: https://issues.apache.org/jira/browse/FLINK-5633
> Project: Flink
>  Issue Type: Bug
>  Components: Job-Submission, YARN
>Affects Versions: 1.1.4
>Reporter: Giuliano Caliari
>Priority: Minor
>
> I’m running a job on my local cluster and the first time I submit the job 
> everything works but whenever I cancel and re-submit the same job it fails 
> with:
> {quote}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
>   at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:101)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:400)
>   at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>   at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:634)
>   at au.com.my.package.pTraitor.OneTrait.execute(Traitor.scala:147)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.delayedEndpoint$au$com$my$package$pTraitor$TraitorAppOneTrait$1(TraitorApp.scala:22)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$delayedInit$body.apply(TraitorApp.scala:21)
>   at scala.Function0$class.apply$mcV$sp(Function0.scala:34)
>   at 
> scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.App$$anonfun$main$1.apply(App.scala:76)
>   at scala.collection.immutable.List.foreach(List.scala:381)
>   at 
> scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
>   at scala.App$class.main(App.scala:76)
>   at 
> au.com.my.package.pTraitor.TraitorAppOneTrait$.main(TraitorApp.scala:21)
>   at au.com.my.package.pTraitor.TraitorAppOneTrait.main(TraitorApp.scala)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:528)
>   at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:419)
>   at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:339)
>   at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:831)
>   at org.apache.flink.client.CliFrontend.run(CliFrontend.java:256)
>   at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1073)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1120)
>   at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1117)
>   at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:29)
>   at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1116)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
>   at 
> scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
>   at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
>