[ 
https://issues.apache.org/jira/browse/BEAM-7478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16854456#comment-16854456
 ] 

Maximilian Michels commented on BEAM-7478:
------------------------------------------

The full stack trace:

{noformat}
org.apache.flink.runtime.rest.handler.RestHandlerException: Job submission 
failed.
        at 
org.apache.flink.runtime.rest.handler.job.JobSubmitHandler.lambda$handleRequest$2(JobSubmitHandler.java:119)
        at 
java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
        at 
java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
        at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
        at 
org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:770)
        at akka.dispatch.OnComplete.internal(Future.scala:258)
        at akka.dispatch.OnComplete.internal(Future.scala:256)
        at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
        at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
        at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
        at 
org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
        at 
scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
        at 
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
        at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
        at 
akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:20)
        at 
akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:18)
        at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:436)
        at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:435)
        at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
        at 
akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
        at 
akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
        at 
akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
        at 
akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
        at 
scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
        at 
akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
        at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)
        at 
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
        at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at 
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at 
scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at 
scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
Caused by: java.util.concurrent.CompletionException: 
org.apache.flink.runtime.client.JobSubmissionException: Failed to submit job.
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$submitJob$2(Dispatcher.java:256)
        at 
java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
        at 
java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
        at 
java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:561)
        at 
java.util.concurrent.CompletableFuture$UniRun.tryFire(CompletableFuture.java:690)
        at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:332)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:158)
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
        at 
akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
        at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
        at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
        at akka.actor.ActorCell.invoke(ActorCell.scala:495)
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
        at akka.dispatch.Mailbox.run(Mailbox.scala:224)
        at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
        ... 4 more
Caused by: org.apache.flink.runtime.client.JobSubmissionException: Failed to 
submit job.
        ... 24 more
Caused by: java.util.concurrent.CompletionException: 
java.lang.RuntimeException: 
org.apache.flink.runtime.client.JobExecutionException: Could not set up 
JobManager
        at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
        at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
        at 
java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:708)
        at 
java.util.concurrent.CompletableFuture$UniRun.tryFire(CompletableFuture.java:687)
        ... 18 more
Caused by: java.lang.RuntimeException: 
org.apache.flink.runtime.client.JobExecutionException: Could not set up 
JobManager
        at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:199)
        at 
org.apache.flink.util.function.ConsumerWithException.accept(ConsumerWithException.java:40)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$waitForTerminatingJobManager$29(Dispatcher.java:820)
        at 
java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705)
        ... 19 more
Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not set 
up JobManager
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:176)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher$DefaultJobManagerRunnerFactory.createJobManagerRunner(Dispatcher.java:936)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.createJobManagerRunner(Dispatcher.java:291)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.runJob(Dispatcher.java:281)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.persistAndRunJob(Dispatcher.java:266)
        at 
org.apache.flink.util.function.ConsumerWithException.accept(ConsumerWithException.java:38)
        ... 21 more
Caused by: org.apache.flink.runtime.client.JobExecutionException: Cannot 
initialize task 'DataSource (at TextIO.Read/Read 
(org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat))': 
Deserializing the InputFormat 
(org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat@3531f3ca) 
failed: class sun.reflect.GeneratedSerializationConstructorAccessor70 cannot 
access its superclass sun.reflect.SerializationConstructorAccessorImpl
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:220)
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:100)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createExecutionGraph(JobMaster.java:1152)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1132)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:295)
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
        ... 26 more
Caused by: java.lang.Exception: Deserializing the InputFormat 
(org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat@3531f3ca) 
failed: class sun.reflect.GeneratedSerializationConstructorAccessor70 cannot 
access its superclass sun.reflect.SerializationConstructorAccessorImpl
        at 
org.apache.flink.runtime.jobgraph.InputFormatVertex.initializeOnMaster(InputFormatVertex.java:66)
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:216)
        ... 31 more
Caused by: java.lang.IllegalAccessError: class 
sun.reflect.GeneratedSerializationConstructorAccessor70 cannot access its 
superclass sun.reflect.SerializationConstructorAccessorImpl
        at sun.misc.Unsafe.defineClass(Native Method)
        at sun.reflect.ClassDefiner.defineClass(ClassDefiner.java:63)
        at 
sun.reflect.MethodAccessorGenerator$1.run(MethodAccessorGenerator.java:399)
        at 
sun.reflect.MethodAccessorGenerator$1.run(MethodAccessorGenerator.java:394)
        at java.security.AccessController.doPrivileged(Native Method)
        at 
sun.reflect.MethodAccessorGenerator.generate(MethodAccessorGenerator.java:393)
        at 
sun.reflect.MethodAccessorGenerator.generateSerializationConstructor(MethodAccessorGenerator.java:112)
        at 
sun.reflect.ReflectionFactory.newConstructorForSerialization(ReflectionFactory.java:340)
        at 
java.io.ObjectStreamClass.getSerializableConstructor(ObjectStreamClass.java:1420)
        at java.io.ObjectStreamClass.access$1500(ObjectStreamClass.java:72)
        at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:497)
        at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:472)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.io.ObjectStreamClass.<init>(ObjectStreamClass.java:472)
        at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:369)
        at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:598)
        at 
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1630)
        at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1521)
        at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1781)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
        at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2018)
        at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1942)
        at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1808)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:373)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:502)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:489)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:477)
        at 
org.apache.flink.util.InstantiationUtil.readObjectFromConfig(InstantiationUtil.java:438)
        at 
org.apache.flink.runtime.operators.util.TaskConfig.getStubWrapper(TaskConfig.java:288)
        at 
org.apache.flink.runtime.jobgraph.InputFormatVertex.initializeOnMaster(InputFormatVertex.java:63)
        ... 32 more
{noformat}

> Remote cluster submission from Flink Runner broken due to staging issues
> ------------------------------------------------------------------------
>
>                 Key: BEAM-7478
>                 URL: https://issues.apache.org/jira/browse/BEAM-7478
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink, sdk-java-core
>    Affects Versions: 2.13.0
>            Reporter: Maximilian Michels
>            Priority: Major
>
> The usual way to submit pipelines with the Flink Runner is to build a fat jar 
> and use the {{bin/flink}} utility to submit the jar to a Flink cluster. This 
> works fine.
> Alternatively, the Flink Runner can use the {{flinkMaster}} pipeline option 
> to specify a remote cluster. Upon submitting an example we get the following 
> at Flink's JobManager.
> {noformat}
> Caused by: java.lang.IllegalAccessError: class 
> sun.reflect.GeneratedSerializationConstructorAccessor70 cannot access its 
> superclass sun.reflect.SerializationConstructorAccessorImpl
>       at sun.misc.Unsafe.defineClass(Native Method)
>       at sun.reflect.ClassDefiner.defineClass(ClassDefiner.java:63)
>       at 
> sun.reflect.MethodAccessorGenerator$1.run(MethodAccessorGenerator.java:399)
>       at 
> sun.reflect.MethodAccessorGenerator$1.run(MethodAccessorGenerator.java:394)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at 
> sun.reflect.MethodAccessorGenerator.generate(MethodAccessorGenerator.java:393)
>       at 
> sun.reflect.MethodAccessorGenerator.generateSerializationConstructor(MethodAccessorGenerator.java:112)
>       at 
> sun.reflect.ReflectionFactory.newConstructorForSerialization(ReflectionFactory.java:340)
>       at 
> java.io.ObjectStreamClass.getSerializableConstructor(ObjectStreamClass.java:1420)
>       at java.io.ObjectStreamClass.access$1500(ObjectStreamClass.java:72)
>       at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:497)
>       at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:472)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at java.io.ObjectStreamClass.<init>(ObjectStreamClass.java:472)
>       at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:369)
>       at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:598)
>       at 
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1630)
>       at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1521)
>       at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1781)
>       at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>       at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2018)
>       at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1942)
>       at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1808)
>       at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>       at java.io.ObjectInputStream.readObject(ObjectInputStream.java:373)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:502)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:489)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:477)
>       at 
> org.apache.flink.util.InstantiationUtil.readObjectFromConfig(InstantiationUtil.java:438)
>       at 
> org.apache.flink.runtime.operators.util.TaskConfig.getStubWrapper(TaskConfig.java:288)
>       at 
> org.apache.flink.runtime.jobgraph.InputFormatVertex.initializeOnMaster(InputFormatVertex.java:63)
>       ... 32 more
> {noformat}
> It appears there is an issue with the staging via {{PipelineResources}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to