You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/05 01:18:27 UTC

[GitHub] [beam] damccorm opened a new issue, #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

damccorm opened a new issue, #21696:
URL: https://github.com/apache/beam/issues/21696

   This flink load tests are affected by this error 
   - beam_LoadTests_Go_ParDo_Flink_Batch (FAILED)
   \-- beam_LoadTests_Go_SideInput_Flink_Batch (FAILED)
   \-- beam_LoadTests_Python_Combine_Flink_Batch (FAILED)
   \-- beam_LoadTests_Python_Combine_Flink_Streaming (FAILED)
   \-- beam_LoadTests_Python_ParDo_Flink_Batch (FAILED)
   \-- beam_LoadTests_Python_ParDo_Flink_Streaming (FAILED)
   
    
   
   Stacktrace of the error 
   
   
   14:03:35 root_transform_ids: "e11"
   14:03:35 root_transform_ids: "e12"
   14:03:35 root_transform_ids: "e13"
   14:03:35 root_transform_ids: "e14"
   14:03:35 requirements: "beam:requirement:org.apache.beam:pardo:splittable_dofn:v1"
   14:03:35 2022/03/07 20:03:33 Prepared job with id: load-tests-go-flink-batch-pardo-1-0307182650_12830e27-009a-44e5-8390-a53ce5f264f0 and staging token: load-tests-go-flink-batch-pardo-1-0307182650_12830e27-009a-44e5-8390-a53ce5f264f0
   14:03:35 2022/03/07 20:03:33 Using specified worker binary: 'linux_amd64/pardo'
   14:03:46 2022/03/07 20:03:46 Staged binary artifact with token: 
   14:03:47 2022/03/07 20:03:47 Submitted job: load0tests0go0flink0batch0pardo0100307182650-root-0307200346-d0077c8_166b7cfd-0805-481a-9a6a-52ba9bfddcbc
   14:03:47 2022/03/07 20:03:47 Job state: STOPPED
   14:03:47 2022/03/07 20:03:47 Job state: STARTING
   14:03:47 2022/03/07 20:03:47 Job state: RUNNING
   14:04:04 2022/03/07 20:04:04  (): org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: a8fc45aac0e16ee159027398311e082e)
   14:04:04 	at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:125)
   14:04:04 	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
   14:04:04 	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
   14:04:04 	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
   14:04:04 	at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
   14:04:04 	at org.apache.flink.client.program.rest.RestClusterClient.lambda$pollResourceAsync$22(RestClusterClient.java:665)
   14:04:04 	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
   14:04:04 	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
   14:04:04 	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
   14:04:04 	at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
   14:04:04 	at org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$9(FutureUtils.java:394)
   14:04:04 	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
   14:04:04 	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
   14:04:04 	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
   14:04:04 	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
   14:04:04 	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:943)
   14:04:04 	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
   14:04:04 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   14:04:04 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   14:04:04 	at java.lang.Thread.run(Thread.java:748)
   14:04:04 Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
   14:04:04 	at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144)
   14:04:04 	at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:123)
   14:04:04 	... 19 more
   14:04:04 Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
   14:04:04 	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:118)
   14:04:04 	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:80)
   14:04:04 	at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:233)
   14:04:04 	at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:224)
   14:04:04 	at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:215)
   14:04:04 	at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:666)
   14:04:04 	at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:89)
   14:04:04 	at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:446)
   14:04:04 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   14:04:04 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   14:04:04 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   14:04:04 	at java.lang.reflect.Method.invoke(Method.java:498)
   14:04:04 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305)
   14:04:04 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212)
   14:04:04 	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77)
   14:04:04 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
   14:04:04 	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
   14:04:04 	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
   14:04:04 	at scala.PartialFunction.applyOrElse(PartialFunction.scala:127)
   14:04:04 	at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126)
   14:04:04 	at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
   14:04:04 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175)
   14:04:04 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
   14:04:04 	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
   14:04:04 	at akka.actor.Actor.aroundReceive(Actor.scala:517)
   14:04:04 	at akka.actor.Actor.aroundReceive$(Actor.scala:515)
   14:04:04 	at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
   14:04:04 	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
   14:04:04 	at akka.actor.ActorCell.invoke(ActorCell.scala:561)
   14:04:04 	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
   14:04:04 	at akka.dispatch.Mailbox.run(Mailbox.scala:225)
   14:04:04 	at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
   14:04:04 	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
   14:04:04 	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
   14:04:04 	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
   14:04:04 	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
   14:04:04 Caused by: java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions
   14:04:04 	at java.io.ObjectStreamClass.hasStaticInitializer(Native Method)
   14:04:04 	at java.io.ObjectStreamClass.computeDefaultSUID(ObjectStreamClass.java:1955)
   14:04:04 	at java.io.ObjectStreamClass.access$100(ObjectStreamClass.java:79)
   14:04:04 	at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:275)
   14:04:04 	at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:273)
   14:04:04 	at java.security.AccessController.doPrivileged(Native Method)
   14:04:04 	at java.io.ObjectStreamClass.getSerialVersionUID(ObjectStreamClass.java:272)
   14:04:04 	at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:694)
   14:04:04 	at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:2005)
   14:04:04 	at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1852)
   14:04:04 	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2186)
   14:04:04 	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1669)
   14:04:04 	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2431)
   14:04:04 	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2355)
   14:04:04 	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2213)
   14:04:04 	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1669)
   14:04:04 	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:503)
   14:04:04 	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:461)
   14:04:04 	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:615)
   14:04:04 	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:600)
   14:04:04 	at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:587)
   14:04:04 	at org.apache.flink.util.InstantiationUtil.readObjectFromConfig(InstantiationUtil.java:541)
   14:04:04 	at org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory.readParametersFromConfig(RuntimeSerializerFactory.java:78)
   14:04:04 	at org.apache.flink.runtime.operators.util.TaskConfig.getTypeSerializerFactory(TaskConfig.java:1246)
   14:04:04 	at org.apache.flink.runtime.operators.util.TaskConfig.getOutputSerializer(TaskConfig.java:599)
   14:04:04 	at org.apache.flink.runtime.operators.DataSourceTask.initInputFormat(DataSourceTask.java:324)
   14:04:04 	at org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:106)
   14:04:04 	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:758)
   14:04:04 	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:573)
   14:04:04 	at java.lang.Thread.run(Thread.java:750)
   14:04:04 2022/03/07 20:04:04  (): java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions
   14:04:05 2022/03/07 20:04:05 Job state: FAILED
   14:04:05 2022/03/07 20:04:05 Failed to execute job: job load0tests0go0flink0batch0pardo0100307182650-root-0307200346-d0077c8_166b7cfd-0805-481a-9a6a-52ba9bfddcbc failed
   14:04:05 panic: Failed to execute job: job load0tests0go0flink0batch0pardo0100307182650-root-0307200346-d0077c8_166b7cfd-0805-481a-9a6a-52ba9bfddcbc failed
   14:04:05 
   14:04:05 goroutine 1 [running]:
   14:04:05 github.com/apache/beam/sdks/v2/go/pkg/beam/log.Fatalf(0x1143648, 0xc000120000, 0x10323eb, 0x19, 0xc0003cde00, 0x1, 0x1)
   14:04:05 	/var/jenkins_real_home/workspace/beam_LoadTests_Go_ParDo_Flink_Batch/src/sdks/go/pkg/beam/log/log.go:153 ****0xec
   14:04:05 main.main()
   14:04:05 	/var/jenkins_real_home/workspace/beam_LoadTests_Go_ParDo_Flink_Batch/src/sdks/go/test/load/pardo/pardo.go:105 ****0x3ca
   14:04:05 
   14:04:05 \> Task :sdks:go:test:load:run FAILED
   
   Imported from Jira [BEAM-14174](https://issues.apache.org/jira/browse/BEAM-14174). Original Jira may contain additional context.
   Reported by: andoni.guzman.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1317571530

   Moving to P2 as OOM errors are mostly a performance issue instead of blocking functionality


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1288142868

   Turns out that even specifying `flink-snapshot-url=https://archive.apache.org/dist/flink/flink-1.13.6/flink-1.13.6-bin-scala_2.12.tgz` the flink version on dataproc is still 1.12.5 (see https://cloud.google.com/dataproc/docs/concepts/versioning/dataproc-release-2.0). The latest error message is definitely due to inconsistent flink versions.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1317225676

   Well, after #24129 three ParDo loadtest (python and go) now passing, but the remaining six tests still fail with different reasons:
   
   beam_LoadTests_Go_SideInput_Flink_Batch: GRPC Error
   ```
   07:00:41 Full error:
   07:00:41 while executing Process for Plan[2-2]:
   07:00:41 2: Discard
   07:00:41 3: PCollection[n8] Out:[2]
   07:00:41 4: ParDo[load.RuntimeMonitor] Out:[3]
   07:00:41 5: PCollection[n7] Out:[4]
   07:00:41 6: ParDo[main.iterSideInputFn] Out:[5]
   07:00:41 1: DataSource[S[fn/read/n6:0@localhost:35493], local_output] Coder:W;fn/wire/n6:0<KV;c2<bytes;c0,bytes;c0>>!GWC Out:6
   07:00:41 	caused by:
   07:00:41 panic: broken stream: StateChannel[localhost:45275].Send(r1): context canceled
   07:00:41 	caused by:
   07:00:41 rpc error: code = Internal desc = unexpected EOF goroutine 51 [running]:
   ```
   
   beam_LoadTests_Go_GBK_Flink_Batch
   beam_LoadTests_Go_Combine_Flink_Batch: Heartbeat timeout
   ```
   05:58:54 Caused by: java.util.concurrent.TimeoutException: Heartbeat of TaskManager with id container_1668592968333_0001_01_000004(beam-loadtests-go-gbk-flink-batch-715-w-3.c.apache-beam-testing.internal:8026) timed out.
   05:58:54 	... 31 more
   05:58:54 2022/11/16 10:58:54  (): java.util.concurrent.TimeoutException: Heartbeat of TaskManager with id container_1668592968333_0001_01_000004(beam-loadtests-go-gbk-flink-batch-715-w-3.c.apache-beam-testing.internal:8026) timed out.
   05:58:54 2022/11/16 10:58:54 Job state: FAILED
   ```
   
   beam_LoadTests_Go_CoGBK_Flink_batch: OOM
   ```
   03:39:28 2022/11/16 08:39:28  (): org.apache.flink.client.program.ProgramInvocationException: Job failed (JobID: 30932d0296729e9591b3ea8e710c2dc3)
   03:39:28 	at org.apache.flink.client.deployment.ClusterClientJobClientAdapter.lambda$null$6(ClusterClientJobClientAdapter.java:130)
   ...
   03:39:28 Caused by: java.lang.OutOfMemoryError: Direct buffer memory. The direct out-of-memory error has occurred. This can mean two things: either job(s) require(s) a larger size of JVM direct memory or there is a direct memory leak. The direct memory can be allocated by user code or some of its dependencies. In this case 'taskmanager.memory.task.off-heap.size' configuration option should be increased. Flink framework and its dependencies also consume the direct memory, mostly for network communication. The most of network memory is managed by Flink and should not result in out-of-memory error. In certain special cases, in particular for jobs with high parallelism, the framework may require more direct memory which is not managed by Flink. In this case 'taskmanager.memory.framework.off-heap.size' configuration option should be increased. If the error persists then there is probably a direct memory leak in user code or some of its dependencies which has to be investigated and fixe
 d. The task executor has to be shutdown...
   ```
   beam_LoadTests_Python_Combine_Flink_Streaming: time out and OOM
   ```
   15:01:39 AssertionError: Job did not reach to a terminal state after waiting indefinitely. Console URL: https://console.cloud.google.com/dataflow/jobs/<RegionId>/2022-11-15_07_07_59-7787368501547085875?project
   ```
   another:
   ```
   14:11:11 RuntimeError: Pipeline load-tests-python-flink-streaming-combine-4-1115182214_25b36822-a47b-4521-a445-6c31525fd9e9 failed in state FAILED: java.lang.OutOfMemoryError: Java heap space
   ```
   
   beam_LoadTests_Python_Combine_Flink_Batch
   - still running
   
   These all sounds real issue in production environment (instead of configuration issue the test had).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1282856126

   .take-issue


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1287472367

   The original failure was due to that the job server artifact of flink 1.12 has long stopped updating (https://pantheon.corp.google.com/gcr/images/apache-beam-testing/global/beam_portability/beam_flink1.12_job_server). Switched to using Flink 1.13, now all jobs are having same error:
   
   ```
   04:43:52 2022/10/21 08:43:52 Submitted job: load0tests0go0flink0batch0combine0101021065324-root-1021084351-5f971342_60528140-7d74-4569-a5ec-99335a2f7dfe
   04:43:52 2022/10/21 08:43:52 Job state: STOPPED
   04:43:52 2022/10/21 08:43:52 Job state: STARTING
   04:43:52 2022/10/21 08:43:52 Job state: RUNNING
   04:45:01 2022/10/21 08:45:01  (): java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.RuntimeException: Error while waiting for job to be initialized
   04:45:01 	at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:316)
   04:45:01 	at org.apache.flink.api.java.ExecutionEnvironment.executeAsync(ExecutionEnvironment.java:1061)
   04:45:01 	at org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:958)
   04:45:01 	at org.apache.beam.runners.flink.FlinkBatchPortablePipelineTranslator$BatchTranslationContext.execute(FlinkBatchPortablePipelineTranslator.java:195)
   04:45:01 	at org.apache.beam.runners.flink.FlinkPipelineRunner.runPipelineWithTranslator(FlinkPipelineRunner.java:132)
   04:45:01 	at org.apache.beam.runners.flink.FlinkPipelineRunner.run(FlinkPipelineRunner.java:99)
   04:45:01 	at org.apache.beam.runners.jobsubmission.JobInvocation.runPipeline(JobInvocation.java:86)
   04:45:01 	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
   04:45:01 	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:57)
   04:45:01 	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
   04:45:01 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   04:45:01 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   04:45:01 	at java.lang.Thread.run(Thread.java:750)
   04:45:01 Caused by: java.util.concurrent.ExecutionException: java.lang.RuntimeException: Error while waiting for job to be initialized
   04:45:01 	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
   04:45:01 	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
   04:45:01 	at org.apache.flink.api.java.ExecutionEnvironment.executeAsync(ExecutionEnvironment.java:1056)
   04:45:01 	... 11 more
   04:45:01 Caused by: java.lang.RuntimeException: Error while waiting for job to be initialized
   04:45:01 	at org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(ClientUtils.java:160)
   04:45:01 	at org.apache.flink.client.deployment.executors.AbstractSessionClusterExecutor.lambda$execute$2(AbstractSessionClusterExecutor.java:82)
   04:45:01 	at org.apache.flink.util.function.FunctionUtils.lambda$uncheckedFunction$2(FunctionUtils.java:73)
   04:45:01 	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
   04:45:01 	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
   04:45:01 	at java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:457)
   04:45:01 	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
   04:45:01 	at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
   04:45:01 	at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
   04:45:01 	at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
   04:45:01 Caused by: java.util.concurrent.ExecutionException: org.apache.flink.runtime.concurrent.FutureUtils$RetryException: Could not complete the operation. Number of retries has been exhausted.
   04:45:01 	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
   04:45:01 	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
   04:45:01 	at org.apache.flink.client.deployment.executors.AbstractSessionClusterExecutor.lambda$null$0(AbstractSessionClusterExecutor.java:83)
   04:45:01 	at org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(ClientUtils.java:140)
   04:45:01 	... 9 more
   04:45:01 Caused by: org.apache.flink.runtime.concurrent.FutureUtils$RetryException: Could not complete the operation. Number of retries has been exhausted.
   04:45:01 	at org.apache.flink.runtime.concurrent.FutureUtils.lambda$retryOperationWithDelay$9(FutureUtils.java:386)
   04:45:01 	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
   04:45:01 	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
   04:45:01 	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
   04:45:01 	at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
   04:45:01 	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:943)
   04:45:01 	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
   04:45:01 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   04:45:01 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   04:45:01 	at java.lang.Thread.run(Thread.java:750)
   04:45:01 Caused by: java.util.concurrent.CompletionException: org.apache.flink.runtime.rest.util.RestClientException: Response was neither of the expected type([simple type, class org.apache.flink.runtime.rest.messages.job.JobDetailsInfo]) nor an error.
   04:45:01 	at java.util.concurrent.CompletableFuture.encodeRelay(CompletableFuture.java:326)
   04:45:01 	at java.util.concurrent.CompletableFuture.completeRelay(CompletableFuture.java:338)
   04:45:01 	at java.util.concurrent.CompletableFuture.uniRelay(CompletableFuture.java:925)
   04:45:01 	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:967)
   04:45:01 	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:940)
   04:45:01 	... 4 more
   04:45:01 Caused by: org.apache.flink.runtime.rest.util.RestClientException: Response was neither of the expected type([simple type, class org.apache.flink.runtime.rest.messages.job.JobDetailsInfo]) nor an error.
   04:45:01 	at org.apache.flink.runtime.rest.RestClient.parseResponse(RestClient.java:502)
   04:45:01 	at org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$3(RestClient.java:466)
   04:45:01 	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:966)
   04:45:01 	... 5 more
   04:45:01 Caused by: org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot map `null` into type `long` (set DeserializationConfig.DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES to 'false' to allow)
   04:45:01  at [Source: UNKNOWN; line: -1, column: -1] (through reference chain: org.apache.flink.runtime.rest.messages.job.JobDetailsInfo["maxParallelism"])
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:63)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext.reportInputMismatch(DeserializationContext.java:1575)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.NumberDeserializers$PrimitiveOrWrapperDeserializer.getNullValue(NumberDeserializers.java:176)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.impl.PropertyValueBuffer._findMissing(PropertyValueBuffer.java:204)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.impl.PropertyValueBuffer.getParameters(PropertyValueBuffer.java:160)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.ValueInstantiator.createFromObjectWith(ValueInstantiator.java:288)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.impl.PropertyBasedCreator.build(PropertyBasedCreator.java:202)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:520)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1390)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserializeFromObject(BeanDeserializer.java:362)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:195)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.DefaultDeserializationContext.readRootValue(DefaultDeserializationContext.java:322)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper._readValue(ObjectMapper.java:4569)
   04:45:01 	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:2867)
   04:45:01 	at org.apache.flink.runtime.rest.RestClient.parseResponse(RestClient.java:475)
   04:45:01 	... 7 more
   04:45:01 2022/10/21 08:45:01  (): org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot map `null` into type `long` (set DeserializationConfig.DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES to 'false' to allow)
   04:45:01  at [Source: UNKNOWN; line: -1, column: -1] (through reference chain: org.apache.flink.runtime.rest.messages.job.JobDetailsInfo["maxParallelism"])
   04:45:01 2022/10/21 08:45:01 Job state: FAILED
   04:45:02 2022/10/21 08:45:01 Failed to execute job: job load0tests0go0flink0batch0combine0101021065324-root-1021084351-5f971342_60528140-7d74-4569-a5ec-99335a2f7dfe failed
   
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn closed issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn closed issue #21696: Flink Tests failure :  java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions 
URL: https://github.com/apache/beam/issues/21696


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #21696: Flink Tests failure : java.lang.NoClassDefFoundError: Could not initialize class org.apache.beam.runners.core.construction.SerializablePipelineOptions

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #21696:
URL: https://github.com/apache/beam/issues/21696#issuecomment-1251380253

   These tests have been failed since March 24, 2022 which is one day after we dropped flink 1.11 on master (#17145) likely some leftover code still referring to flink v1.11 classes causing failure.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org