You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Marek Maj <ma...@gmail.com> on 2019/09/12 07:28:08 UTC

SIGSEGV error

Hi everyone,

Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an upgrade
our task managers started to fail with SIGSEGV error from time to time.

In process of adjusting the code to 1.8.1, we noticed that there were some
changes around TypeSerializerSnapshot interface and its implementations. At
that time we had a few custom serializers which we decided to throw out
during migration and then leverage flink default serializers. We don't mind
clearing the state in the process of migration, an effort to migrate with
state seems to be not worth it.

Unfortunately after running new version we see SIGSEGV errors from time to
time. It may be that serialization is not the real cause, but at the moment
it seems to be the most probable reason. We have not performed any
significant code changes besides serialization area.

We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
Checkpoint configuration: RocksDB backend, not incremental, 50s min
processing time

You can find parts of JobManager log and ErrorFile log of failed container
included below.

Any suggestions are welcome

Best regards
Marek Maj

jobmanager.log

019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   - Completed
checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965 (18532488122 bytes
in 60871 ms).

2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
Triggering checkpoint 48 @ 1568111478177 for job
c8a9ae03785ade86348c3189cf7dd965.

2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   - Completed
checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965 (19049515705 bytes
in 61083 ms).

2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
Triggering checkpoint 49 @ 1568111589279 for job
c8a9ae03785ade86348c3189cf7dd965.

2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
Requesting TaskManager's path for query services failed.

java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException:
Ask timed out on [Actor[akka://flink/user/dispatcher#374570759]] after
[10000 ms]. Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)

at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)

at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)

at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)

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:816)

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:74)

at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)

at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)

at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)

at
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)

at
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)

at
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)

at
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)

at java.lang.Thread.run(Thread.java:745)

Caused by: akka.pattern.AskTimeoutException: Ask timed out on
[Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)

... 9 common frames omitted

2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
Requesting TaskManager's path for query services failed.

java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException:
Ask timed out on [Actor[akka://flink/user/dispatcher#374570759]] after
[10000 ms]. Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)

at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)

at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)

at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)

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:816)

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:74)

at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)

at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)

at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)

at
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)

at
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)

at
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)

at
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)

at java.lang.Thread.run(Thread.java:745)

Caused by: akka.pattern.AskTimeoutException: Ask timed out on
[Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)

... 9 common frames omitted

2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
Requesting TaskManager's path for query services failed.

java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException:
Ask timed out on [Actor[akka://flink/user/dispatcher#374570759]] after
[10000 ms]. Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)

at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)

at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)

at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)

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:816)

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:74)

at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)

at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)

at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)

at
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)

at
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)

at
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)

at
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)

at
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)

at java.lang.Thread.run(Thread.java:745)

Caused by: akka.pattern.AskTimeoutException: Ask timed out on
[Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
Sender[null] sent message of type
"org.apache.flink.runtime.rpc.messages.LocalFencedMessage".

at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)

... 9 common frames omitted

2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   - The
heartbeat of TaskManager with id container_e67_1568017536744_0044_01_000023
timed out.

2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
because: The heartbeat of TaskManager with id
container_e67_1568017536744_0044_01_000023  timed out.

2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
(1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.

org.apache.flink.util.FlinkException: The assigned slot
container_e67_1568017536744_0044_01_000023_0 was removed.

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)

at
org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)

at
org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)

at
org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)

at java.util.concurrent.FutureTask.run(FutureTask.java:266)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)

at
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)

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)

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)

2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.

org.apache.flink.util.FlinkException: The assigned slot
container_e67_1568017536744_0044_01_000023_0 was removed.

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)

at
org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)

at
org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)

at
org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)

at java.util.concurrent.FutureTask.run(FutureTask.java:266)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)

at
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)

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)

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)

2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
RUNNING to FAILING.

org.apache.flink.util.FlinkException: The assigned slot
container_e67_1568017536744_0044_01_000023_0 was removed.

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)

at
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)

at
org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)

at
org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)

at
org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)

at java.util.concurrent.FutureTask.run(FutureTask.java:266)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)

at
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

at
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)

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)

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)



hs_err_pid_262348.log for failed container

#
# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
tid=0x00007f2916833700
#
# JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
1.8.0_112-b15)
# Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
linux-amd64 compressed oops)
# Problematic frame:
# C  [libzip.so+0xb2c2]  inflateEnd+0x32
#
# Core dump written. Default location:
/data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
or core.262348
#
# If you would like to submit a bug report, please visit:
#   http://bugreport.java.com/bugreport/crash.jsp
# The crash happened outside the Java Virtual Machine in native code.
# See problematic frame for where to report the bug.
#

---------------  T H R E A D  ---------------

Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
[_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]

siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
0x0000000000001080

Registers:
RAX=0x00007f0100000001, RBX=0x00007f2945e52770, RCX=0x0000000000000180,
RDX=0x00007f2945e52770
RSP=0x00007f29168323d0, RBP=0x00007f29168323e0, RSI=0x0000000000001040,
RDI=0x00007f2945e52770
R8 =0x00000007bff0f170, R9 =0x0000000000000006, R10=0x00007f2935017a08,
R11=0x00007f294b583d50
R12=0x00007f29440e81f8, R13=0x00007f293135cc58, R14=0x00007f2916832490,
R15=0x00007f29440e8000
RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
CSGSFS=0x0000000000000033, ERR=0x0000000000000004
  TRAPNO=0x000000000000000e

Top of Stack: (sp=0x00007f29168323d0)
0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
0x00007f29168323e0:   00007f2916832400 00007f294944338e
0x00007f29168323f0:   00007f293135cc58 0000000000000000
0x00007f2916832400:   00007f2916832468 00007f2935017a34
0x00007f2916832410:   00007f2916832540 00007f293501306d
0x00007f2916832420:   00007f29350055d0 00007f2916832428
0x00007f2916832430:   0000000000000000 00007f2916832490
0x00007f2916832440:   00007f293135cd70 0000000000000000
0x00007f2916832450:   00007f293135cc58 0000000000000000
0x00007f2916832460:   00007f2916832488 00007f29168324e8
0x00007f2916832470:   00007f29350082bd 00000006ab616900
0x00007f2916832480:   00007f2935011538 00007f2945e52770
0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
0x00007f29168324d0:   00007f293135c618 00007f2916832488
0x00007f29168324e0:   00007f2916832518 00007f2916832580
0x00007f29168324f0:   00007f29350082bd 0000000000000000
0x00007f2916832500:   00007f2945e52770 0000000000000000
0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
0x00007f2916832520:   0000000000000009 00000007bff0f158
0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
0x00007f2916832540:   00007f2916832520 00007f293132f631
0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
0x00007f2916832560:   0000000000000000 00007f293132f6c0
0x00007f2916832570:   00007f2916832518 00007f29168325d8
0x00007f2916832580:   00007f2916832620 00007f29350082bd
0x00007f2916832590:   0000000000000000 0000000000000000
0x00007f29168325a0:   0000000000000000 0000000000000000
0x00007f29168325b0:   0000000000000000 0000000000000000
0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38

Instructions: (pc=0x00007f294944b2c2)
0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0

Register to memory mapping:

RAX=0x00007f0100000001 is an unknown value
RBX=0x00007f2945e52770 is an unknown value
RCX=0x0000000000000180 is an unknown value
RDX=0x00007f2945e52770 is an unknown value
RSP=0x00007f29168323d0 is pointing into the stack for thread:
0x00007f29440e8000
RBP=0x00007f29168323e0 is pointing into the stack for thread:
0x00007f29440e8000
RSI=0x0000000000001040 is an unknown value
RDI=0x00007f2945e52770 is an unknown value
R8 =0x00000007bff0f170 is an oop
[Ljava.lang.Object;
 - klass: 'java/lang/Object'[]
 - length: 16
R9 =0x0000000000000006 is an unknown value
R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
method entry point (kind = native)  [0x00007f29350176e0,
0x00007f2935017fe0]  2304 bytes
R11=0x00007f294b583d50: <offset 0x9c3d50> in
/usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
R12=0x00007f29440e81f8 is an unknown value
R13={method} {0x00007f293135cc58} 'end' '(J)V' in 'java/util/zip/Inflater'
R14=0x00007f2916832490 is pointing into the stack for thread:
0x00007f29440e8000
R15=0x00007f29440e8000 is a thread


Stack: [0x00007f2916733000,0x00007f2916834000],  sp=0x00007f29168323d0,
 free space=1020k
Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
code)
C  [libzip.so+0xb2c2]  inflateEnd+0x32
C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
j  java.util.zip.Inflater.end(J)V+0
j  java.util.zip.Inflater.end()V+29
j  java.util.zip.ZipFile.close()V+169
j  sun.net.www.protocol.jar.URLJarFile.close()V+18
j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
v  ~StubRoutines::call_stub
V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*, methodHandle*,
JavaCallArguments*, Thread*)+0x1056
V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*, KlassHandle,
Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
KlassHandle, Symbol*, Symbol*, Thread*)+0x47
V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
C  [libpthread.so.0+0x7e25]  start_thread+0xc5

Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
j  java.util.zip.Inflater.end(J)V+0
j  java.util.zip.Inflater.end()V+29
j  java.util.zip.ZipFile.close()V+169
j  sun.net.www.protocol.jar.URLJarFile.close()V+18
j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
v  ~StubRoutines::call_stub

Re: SIGSEGV error

Posted by Till Rohrmann <tr...@apache.org>.
Great to hear that you fixed the problem by specifying an explicit
serializer for the state.

Cheers,
Till

On Tue, May 18, 2021 at 9:43 AM Joshua Fan <jo...@gmail.com> wrote:

> Hi Till,
> I also tried the job without gzip, it came into the same error.
> But the problem is solved now. I was about to give up to solve it, I found
> the mail at
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/JVM-crash-SIGSEGV-in-ZIP-GetEntry-td17326.html.
> So I think maybe it was something about the serialize staff.
> What I have done is :
> before:
>
> OperatorStateStore stateStore = context.getOperatorStateStore();
> ListStateDescriptor lsd = new ListStateDescriptor("bucket-states",State.class);
>
> after:
>
> OperatorStateStore stateStore = context.getOperatorStateStore();
> ListStateDescriptor lsd = new ListStateDescriptor("bucket-states",new JavaSerializer());
>
> Hope this is helpful.
>
> Yours sincerely
> Josh
>
>
>
> Till Rohrmann <tr...@apache.org> 于2021年5月18日周二 下午2:54写道:
>
>> Hi Joshua,
>>
>> could you try whether the job also fails when not using the gzip format?
>> This could help us narrow down the culprit. Moreover, you could try to run
>> your job and Flink with Java 11 now.
>>
>> Cheers,
>> Till
>>
>> On Tue, May 18, 2021 at 5:10 AM Joshua Fan <jo...@gmail.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> Most of the posts says that "Most of the times, the crashes in
>>> ZIP_GetEntry occur when the jar file being accessed has been
>>> modified/overwritten while the JVM instance was running. ", but do not
>>> know when and which jar file was modified according to the job running in
>>> flink.
>>>
>>> for your information.
>>>
>>> Yours sincerely
>>> Josh
>>>
>>> Joshua Fan <jo...@gmail.com> 于2021年5月18日周二 上午10:15写道:
>>>
>>>> Hi Stephan, Till
>>>>
>>>> Recently, I tried to upgrade a flink job from 1.7 to 1.11,
>>>> unfortunately, the weird problem appeared, " SIGSEGV (0xb) at
>>>> pc=0x0000000000000025, pid=135306, tid=140439001388800".  The pid log is
>>>> attached.
>>>> Actually, it is a simple job that consumes messages from kafka and
>>>> writes into hdfs with a gzip format. It can run in 1.11 for about 2
>>>> minutes, then the JVM will crash, then job restart and jvm crash again
>>>> until the application fails.
>>>> I also tried to set -Dsun.zip.disableMemoryMapping=true,but it turns
>>>> out helpless, the same crash keeps happening. Google suggests to upgrade
>>>> jdk to jdk1.9, but it is not feasible.
>>>> Any suggestions? Thanks a lot.
>>>>
>>>> Yours sincerely
>>>> Josh
>>>>
>>>> Stephan Ewen <se...@apache.org> 于2019年9月13日周五 下午11:11写道:
>>>>
>>>>> Given that the segfault happens in the JVM's ZIP stream code, I am
>>>>> curious is this is a bug in Flink or in the JVM core libs, that happens to
>>>>> be triggered now by newer versions of FLink.
>>>>>
>>>>> I found this on StackOverflow, which looks like it could be related:
>>>>> https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
>>>>> Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"
>>>>> ?
>>>>>
>>>>>
>>>>> On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> Hi Marek,
>>>>>>
>>>>>> could you share the logs statements which happened before the SIGSEGV
>>>>>> with us? They might be helpful to understand what happened before.
>>>>>> Moreover, it would be helpful to get access to your custom serializer
>>>>>> implementations. I'm also pulling in Gordon who worked on
>>>>>> the TypeSerializerSnapshot improvements.
>>>>>>
>>>>>> Cheers,
>>>>>> Till
>>>>>>
>>>>>> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>>>>>>> upgrade our task managers started to fail with SIGSEGV error from time to
>>>>>>> time.
>>>>>>>
>>>>>>> In process of adjusting the code to 1.8.1, we noticed that there
>>>>>>> were some changes around TypeSerializerSnapshot interface and its
>>>>>>> implementations. At that time we had a few custom serializers which we
>>>>>>> decided to throw out during migration and then leverage flink default
>>>>>>> serializers. We don't mind clearing the state in the process of migration,
>>>>>>> an effort to migrate with state seems to be not worth it.
>>>>>>>
>>>>>>> Unfortunately after running new version we see SIGSEGV errors from
>>>>>>> time to time. It may be that serialization is not the real cause, but at
>>>>>>> the moment it seems to be the most probable reason. We have not performed
>>>>>>> any significant code changes besides serialization area.
>>>>>>>
>>>>>>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>>>>>>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>>>>>>> processing time
>>>>>>>
>>>>>>> You can find parts of JobManager log and ErrorFile log of failed
>>>>>>> container included below.
>>>>>>>
>>>>>>> Any suggestions are welcome
>>>>>>>
>>>>>>> Best regards
>>>>>>> Marek Maj
>>>>>>>
>>>>>>> jobmanager.log
>>>>>>>
>>>>>>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>>>>>>> (18532488122 bytes in 60871 ms).
>>>>>>>
>>>>>>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>>> Triggering checkpoint 48 @ 1568111478177 for job
>>>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>>>
>>>>>>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>>>>>>> (19049515705 bytes in 61083 ms).
>>>>>>>
>>>>>>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>>> Triggering checkpoint 49 @ 1568111589279 for job
>>>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>>>
>>>>>>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>>
>>>>>>> java.util.concurrent.CompletionException:
>>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>>
>>>>>>> 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:816)
>>>>>>>
>>>>>>> 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:74)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>>
>>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>>
>>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>>
>>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>>
>>>>>>> ... 9 common frames omitted
>>>>>>>
>>>>>>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>>
>>>>>>> java.util.concurrent.CompletionException:
>>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>>
>>>>>>> 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:816)
>>>>>>>
>>>>>>> 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:74)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>>
>>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>>
>>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>>
>>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>>
>>>>>>> ... 9 common frames omitted
>>>>>>>
>>>>>>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>>
>>>>>>> java.util.concurrent.CompletionException:
>>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>>
>>>>>>> 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:816)
>>>>>>>
>>>>>>> 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:74)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>>
>>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>>
>>>>>>> at
>>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>>
>>>>>>> at
>>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>>
>>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>>
>>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>>> Sender[null] sent message of type
>>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>>
>>>>>>> at
>>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>>
>>>>>>> ... 9 common frames omitted
>>>>>>>
>>>>>>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>>>> The heartbeat of TaskManager with id
>>>>>>> container_e67_1568017536744_0044_01_000023 timed out.
>>>>>>>
>>>>>>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>>>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>>>>>>> because: The heartbeat of TaskManager with id
>>>>>>> container_e67_1568017536744_0044_01_000023  timed out.
>>>>>>>
>>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   -
>>>>>>> my-function (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING
>>>>>>> to FAILED.
>>>>>>>
>>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>>
>>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>>>>>>
>>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>>
>>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>>>>>>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>>>>>>> RUNNING to FAILING.
>>>>>>>
>>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>>
>>>>>>> at
>>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>>
>>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>>
>>>>>>> at
>>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>> 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)
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> hs_err_pid_262348.log for failed container
>>>>>>>
>>>>>>> #
>>>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>>>> #
>>>>>>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>>>>>>> tid=0x00007f2916833700
>>>>>>> #
>>>>>>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>>>>>>> 1.8.0_112-b15)
>>>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>>>>>>> linux-amd64 compressed oops)
>>>>>>> # Problematic frame:
>>>>>>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>>>> #
>>>>>>> # Core dump written. Default location:
>>>>>>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>>>>>>> or core.262348
>>>>>>> #
>>>>>>> # If you would like to submit a bug report, please visit:
>>>>>>> #   http://bugreport.java.com/bugreport/crash.jsp
>>>>>>> # The crash happened outside the Java Virtual Machine in native code.
>>>>>>> # See problematic frame for where to report the bug.
>>>>>>> #
>>>>>>>
>>>>>>> ---------------  T H R E A D  ---------------
>>>>>>>
>>>>>>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>>>>>>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>>>>>>
>>>>>>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>>>>>>> 0x0000000000001080
>>>>>>>
>>>>>>> Registers:
>>>>>>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770,
>>>>>>> RCX=0x0000000000000180, RDX=0x00007f2945e52770
>>>>>>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0,
>>>>>>> RSI=0x0000000000001040, RDI=0x00007f2945e52770
>>>>>>> R8 =0x00000007bff0f170, R9 =0x0000000000000006,
>>>>>>> R10=0x00007f2935017a08, R11=0x00007f294b583d50
>>>>>>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58,
>>>>>>> R14=0x00007f2916832490, R15=0x00007f29440e8000
>>>>>>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>>>>>>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>>>>>>   TRAPNO=0x000000000000000e
>>>>>>>
>>>>>>> Top of Stack: (sp=0x00007f29168323d0)
>>>>>>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>>>>>>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>>>>>>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>>>>>>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>>>>>>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>>>>>>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>>>>>>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>>>>>>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>>>>>>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>>>>>>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>>>>>>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>>>>>>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>>>>>>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>>>>>>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>>>>>>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>>>>>>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>>>>>>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>>>>>>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>>>>>>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>>>>>>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>>>>>>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>>>>>>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>>>>>>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>>>>>>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>>>>>>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>>>>>>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>>>>>>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>>>>>>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>>>>>>> 0x00007f2916832590:   0000000000000000 0000000000000000
>>>>>>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>>>>>>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>>>>>>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>>>>>>
>>>>>>> Instructions: (pc=0x00007f294944b2c2)
>>>>>>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>>>>>>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>>>>>>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>>>>>>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff
>>>>>>> d0
>>>>>>>
>>>>>>> Register to memory mapping:
>>>>>>>
>>>>>>> RAX=0x00007f0100000001 is an unknown value
>>>>>>> RBX=0x00007f2945e52770 is an unknown value
>>>>>>> RCX=0x0000000000000180 is an unknown value
>>>>>>> RDX=0x00007f2945e52770 is an unknown value
>>>>>>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>>>>>>> 0x00007f29440e8000
>>>>>>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>>>>>>> 0x00007f29440e8000
>>>>>>> RSI=0x0000000000001040 is an unknown value
>>>>>>> RDI=0x00007f2945e52770 is an unknown value
>>>>>>> R8 =0x00000007bff0f170 is an oop
>>>>>>> [Ljava.lang.Object;
>>>>>>>  - klass: 'java/lang/Object'[]
>>>>>>>  - length: 16
>>>>>>> R9 =0x0000000000000006 is an unknown value
>>>>>>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>>>>>>> method entry point (kind = native)  [0x00007f29350176e0,
>>>>>>> 0x00007f2935017fe0]  2304 bytes
>>>>>>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>>>>>>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>>>>>>> R12=0x00007f29440e81f8 is an unknown value
>>>>>>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in
>>>>>>> 'java/util/zip/Inflater'
>>>>>>> R14=0x00007f2916832490 is pointing into the stack for thread:
>>>>>>> 0x00007f29440e8000
>>>>>>> R15=0x00007f29440e8000 is a thread
>>>>>>>
>>>>>>>
>>>>>>> Stack: [0x00007f2916733000,0x00007f2916834000],
>>>>>>>  sp=0x00007f29168323d0,  free space=1020k
>>>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>>>>> C=native code)
>>>>>>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>>>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>>>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>>>> j  java.util.zip.Inflater.end()V+29
>>>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes)
>>>>>>> @ 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>>>> v  ~StubRoutines::call_stub
>>>>>>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>>>>>>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>>>>>>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*,
>>>>>>> KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>>>>>>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>>>>>>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>>>>>>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>>>>>>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>>>>>>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>>>>>>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>>>>>>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>>>>>>
>>>>>>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>>>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>>>> j  java.util.zip.Inflater.end()V+29
>>>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes)
>>>>>>> @ 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>>>> v  ~StubRoutines::call_stub
>>>>>>>
>>>>>>

Re: SIGSEGV error

Posted by Joshua Fan <jo...@gmail.com>.
Hi Till,
I also tried the job without gzip, it came into the same error.
But the problem is solved now. I was about to give up to solve it, I found
the mail at
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/JVM-crash-SIGSEGV-in-ZIP-GetEntry-td17326.html.
So I think maybe it was something about the serialize staff.
What I have done is :
before:

OperatorStateStore stateStore = context.getOperatorStateStore();
ListStateDescriptor lsd = new ListStateDescriptor("bucket-states",State.class);

after:

OperatorStateStore stateStore = context.getOperatorStateStore();
ListStateDescriptor lsd = new ListStateDescriptor("bucket-states",new
JavaSerializer());

Hope this is helpful.

Yours sincerely
Josh



Till Rohrmann <tr...@apache.org> 于2021年5月18日周二 下午2:54写道:

> Hi Joshua,
>
> could you try whether the job also fails when not using the gzip format?
> This could help us narrow down the culprit. Moreover, you could try to run
> your job and Flink with Java 11 now.
>
> Cheers,
> Till
>
> On Tue, May 18, 2021 at 5:10 AM Joshua Fan <jo...@gmail.com> wrote:
>
>> Hi all,
>>
>> Most of the posts says that "Most of the times, the crashes in
>> ZIP_GetEntry occur when the jar file being accessed has been
>> modified/overwritten while the JVM instance was running. ", but do not
>> know when and which jar file was modified according to the job running in
>> flink.
>>
>> for your information.
>>
>> Yours sincerely
>> Josh
>>
>> Joshua Fan <jo...@gmail.com> 于2021年5月18日周二 上午10:15写道:
>>
>>> Hi Stephan, Till
>>>
>>> Recently, I tried to upgrade a flink job from 1.7 to 1.11,
>>> unfortunately, the weird problem appeared, " SIGSEGV (0xb) at
>>> pc=0x0000000000000025, pid=135306, tid=140439001388800".  The pid log is
>>> attached.
>>> Actually, it is a simple job that consumes messages from kafka and
>>> writes into hdfs with a gzip format. It can run in 1.11 for about 2
>>> minutes, then the JVM will crash, then job restart and jvm crash again
>>> until the application fails.
>>> I also tried to set -Dsun.zip.disableMemoryMapping=true,but it turns
>>> out helpless, the same crash keeps happening. Google suggests to upgrade
>>> jdk to jdk1.9, but it is not feasible.
>>> Any suggestions? Thanks a lot.
>>>
>>> Yours sincerely
>>> Josh
>>>
>>> Stephan Ewen <se...@apache.org> 于2019年9月13日周五 下午11:11写道:
>>>
>>>> Given that the segfault happens in the JVM's ZIP stream code, I am
>>>> curious is this is a bug in Flink or in the JVM core libs, that happens to
>>>> be triggered now by newer versions of FLink.
>>>>
>>>> I found this on StackOverflow, which looks like it could be related:
>>>> https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
>>>> Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"?
>>>>
>>>>
>>>> On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org>
>>>> wrote:
>>>>
>>>>> Hi Marek,
>>>>>
>>>>> could you share the logs statements which happened before the SIGSEGV
>>>>> with us? They might be helpful to understand what happened before.
>>>>> Moreover, it would be helpful to get access to your custom serializer
>>>>> implementations. I'm also pulling in Gordon who worked on
>>>>> the TypeSerializerSnapshot improvements.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>>
>>>>> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:
>>>>>
>>>>>> Hi everyone,
>>>>>>
>>>>>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>>>>>> upgrade our task managers started to fail with SIGSEGV error from time to
>>>>>> time.
>>>>>>
>>>>>> In process of adjusting the code to 1.8.1, we noticed that there were
>>>>>> some changes around TypeSerializerSnapshot interface and its
>>>>>> implementations. At that time we had a few custom serializers which we
>>>>>> decided to throw out during migration and then leverage flink default
>>>>>> serializers. We don't mind clearing the state in the process of migration,
>>>>>> an effort to migrate with state seems to be not worth it.
>>>>>>
>>>>>> Unfortunately after running new version we see SIGSEGV errors from
>>>>>> time to time. It may be that serialization is not the real cause, but at
>>>>>> the moment it seems to be the most probable reason. We have not performed
>>>>>> any significant code changes besides serialization area.
>>>>>>
>>>>>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>>>>>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>>>>>> processing time
>>>>>>
>>>>>> You can find parts of JobManager log and ErrorFile log of failed
>>>>>> container included below.
>>>>>>
>>>>>> Any suggestions are welcome
>>>>>>
>>>>>> Best regards
>>>>>> Marek Maj
>>>>>>
>>>>>> jobmanager.log
>>>>>>
>>>>>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>>>>>> (18532488122 bytes in 60871 ms).
>>>>>>
>>>>>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>> Triggering checkpoint 48 @ 1568111478177 for job
>>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>>
>>>>>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>>>>>> (19049515705 bytes in 61083 ms).
>>>>>>
>>>>>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>> Triggering checkpoint 49 @ 1568111589279 for job
>>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>>
>>>>>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>
>>>>>> java.util.concurrent.CompletionException:
>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>
>>>>>> 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:816)
>>>>>>
>>>>>> 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:74)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>
>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>
>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>
>>>>>> ... 9 common frames omitted
>>>>>>
>>>>>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>
>>>>>> java.util.concurrent.CompletionException:
>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>
>>>>>> 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:816)
>>>>>>
>>>>>> 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:74)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>
>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>
>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>
>>>>>> ... 9 common frames omitted
>>>>>>
>>>>>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>>> Requesting TaskManager's path for query services failed.
>>>>>>
>>>>>> java.util.concurrent.CompletionException:
>>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>>
>>>>>> 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:816)
>>>>>>
>>>>>> 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:74)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>>
>>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>>
>>>>>> at
>>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>>
>>>>>> at
>>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>>
>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>>> Sender[null] sent message of type
>>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>>
>>>>>> at
>>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>>
>>>>>> ... 9 common frames omitted
>>>>>>
>>>>>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>>> The heartbeat of TaskManager with id
>>>>>> container_e67_1568017536744_0044_01_000023 timed out.
>>>>>>
>>>>>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>>>>>> because: The heartbeat of TaskManager with id
>>>>>> container_e67_1568017536744_0044_01_000023  timed out.
>>>>>>
>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   -
>>>>>> my-function (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING
>>>>>> to FAILED.
>>>>>>
>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>
>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>>>>>
>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>
>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>>>>>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>>>>>> RUNNING to FAILING.
>>>>>>
>>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>>
>>>>>> at
>>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>>
>>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>>
>>>>>> at
>>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>> 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)
>>>>>>
>>>>>>
>>>>>>
>>>>>> hs_err_pid_262348.log for failed container
>>>>>>
>>>>>> #
>>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>>> #
>>>>>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>>>>>> tid=0x00007f2916833700
>>>>>> #
>>>>>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>>>>>> 1.8.0_112-b15)
>>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>>>>>> linux-amd64 compressed oops)
>>>>>> # Problematic frame:
>>>>>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>>> #
>>>>>> # Core dump written. Default location:
>>>>>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>>>>>> or core.262348
>>>>>> #
>>>>>> # If you would like to submit a bug report, please visit:
>>>>>> #   http://bugreport.java.com/bugreport/crash.jsp
>>>>>> # The crash happened outside the Java Virtual Machine in native code.
>>>>>> # See problematic frame for where to report the bug.
>>>>>> #
>>>>>>
>>>>>> ---------------  T H R E A D  ---------------
>>>>>>
>>>>>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>>>>>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>>>>>
>>>>>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>>>>>> 0x0000000000001080
>>>>>>
>>>>>> Registers:
>>>>>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770,
>>>>>> RCX=0x0000000000000180, RDX=0x00007f2945e52770
>>>>>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0,
>>>>>> RSI=0x0000000000001040, RDI=0x00007f2945e52770
>>>>>> R8 =0x00000007bff0f170, R9 =0x0000000000000006,
>>>>>> R10=0x00007f2935017a08, R11=0x00007f294b583d50
>>>>>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58,
>>>>>> R14=0x00007f2916832490, R15=0x00007f29440e8000
>>>>>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>>>>>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>>>>>   TRAPNO=0x000000000000000e
>>>>>>
>>>>>> Top of Stack: (sp=0x00007f29168323d0)
>>>>>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>>>>>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>>>>>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>>>>>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>>>>>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>>>>>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>>>>>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>>>>>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>>>>>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>>>>>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>>>>>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>>>>>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>>>>>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>>>>>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>>>>>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>>>>>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>>>>>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>>>>>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>>>>>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>>>>>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>>>>>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>>>>>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>>>>>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>>>>>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>>>>>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>>>>>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>>>>>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>>>>>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>>>>>> 0x00007f2916832590:   0000000000000000 0000000000000000
>>>>>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>>>>>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>>>>>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>>>>>
>>>>>> Instructions: (pc=0x00007f294944b2c2)
>>>>>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>>>>>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>>>>>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>>>>>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>>>>>>
>>>>>> Register to memory mapping:
>>>>>>
>>>>>> RAX=0x00007f0100000001 is an unknown value
>>>>>> RBX=0x00007f2945e52770 is an unknown value
>>>>>> RCX=0x0000000000000180 is an unknown value
>>>>>> RDX=0x00007f2945e52770 is an unknown value
>>>>>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>>>>>> 0x00007f29440e8000
>>>>>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>>>>>> 0x00007f29440e8000
>>>>>> RSI=0x0000000000001040 is an unknown value
>>>>>> RDI=0x00007f2945e52770 is an unknown value
>>>>>> R8 =0x00000007bff0f170 is an oop
>>>>>> [Ljava.lang.Object;
>>>>>>  - klass: 'java/lang/Object'[]
>>>>>>  - length: 16
>>>>>> R9 =0x0000000000000006 is an unknown value
>>>>>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>>>>>> method entry point (kind = native)  [0x00007f29350176e0,
>>>>>> 0x00007f2935017fe0]  2304 bytes
>>>>>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>>>>>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>>>>>> R12=0x00007f29440e81f8 is an unknown value
>>>>>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in
>>>>>> 'java/util/zip/Inflater'
>>>>>> R14=0x00007f2916832490 is pointing into the stack for thread:
>>>>>> 0x00007f29440e8000
>>>>>> R15=0x00007f29440e8000 is a thread
>>>>>>
>>>>>>
>>>>>> Stack: [0x00007f2916733000,0x00007f2916834000],
>>>>>>  sp=0x00007f29168323d0,  free space=1020k
>>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>>>> C=native code)
>>>>>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>>> j  java.util.zip.Inflater.end()V+29
>>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes)
>>>>>> @ 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>>> v  ~StubRoutines::call_stub
>>>>>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>>>>>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>>>>>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*,
>>>>>> KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>>>>>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>>>>>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>>>>>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>>>>>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>>>>>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>>>>>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>>>>>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>>>>>
>>>>>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>>> j  java.util.zip.Inflater.end()V+29
>>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes)
>>>>>> @ 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>>> v  ~StubRoutines::call_stub
>>>>>>
>>>>>

Re: SIGSEGV error

Posted by Till Rohrmann <tr...@apache.org>.
Hi Joshua,

could you try whether the job also fails when not using the gzip format?
This could help us narrow down the culprit. Moreover, you could try to run
your job and Flink with Java 11 now.

Cheers,
Till

On Tue, May 18, 2021 at 5:10 AM Joshua Fan <jo...@gmail.com> wrote:

> Hi all,
>
> Most of the posts says that "Most of the times, the crashes in
> ZIP_GetEntry occur when the jar file being accessed has been
> modified/overwritten while the JVM instance was running. ", but do not
> know when and which jar file was modified according to the job running in
> flink.
>
> for your information.
>
> Yours sincerely
> Josh
>
> Joshua Fan <jo...@gmail.com> 于2021年5月18日周二 上午10:15写道:
>
>> Hi Stephan, Till
>>
>> Recently, I tried to upgrade a flink job from 1.7 to 1.11, unfortunately,
>> the weird problem appeared, " SIGSEGV (0xb) at pc=0x0000000000000025,
>> pid=135306, tid=140439001388800".  The pid log is attached.
>> Actually, it is a simple job that consumes messages from kafka and writes
>> into hdfs with a gzip format. It can run in 1.11 for about 2 minutes, then
>> the JVM will crash, then job restart and jvm crash again until the
>> application fails.
>> I also tried to set -Dsun.zip.disableMemoryMapping=true,but it turns out
>> helpless, the same crash keeps happening. Google suggests to upgrade jdk to
>> jdk1.9, but it is not feasible.
>> Any suggestions? Thanks a lot.
>>
>> Yours sincerely
>> Josh
>>
>> Stephan Ewen <se...@apache.org> 于2019年9月13日周五 下午11:11写道:
>>
>>> Given that the segfault happens in the JVM's ZIP stream code, I am
>>> curious is this is a bug in Flink or in the JVM core libs, that happens to
>>> be triggered now by newer versions of FLink.
>>>
>>> I found this on StackOverflow, which looks like it could be related:
>>> https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
>>> Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"?
>>>
>>>
>>> On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org>
>>> wrote:
>>>
>>>> Hi Marek,
>>>>
>>>> could you share the logs statements which happened before the SIGSEGV
>>>> with us? They might be helpful to understand what happened before.
>>>> Moreover, it would be helpful to get access to your custom serializer
>>>> implementations. I'm also pulling in Gordon who worked on
>>>> the TypeSerializerSnapshot improvements.
>>>>
>>>> Cheers,
>>>> Till
>>>>
>>>> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:
>>>>
>>>>> Hi everyone,
>>>>>
>>>>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>>>>> upgrade our task managers started to fail with SIGSEGV error from time to
>>>>> time.
>>>>>
>>>>> In process of adjusting the code to 1.8.1, we noticed that there were
>>>>> some changes around TypeSerializerSnapshot interface and its
>>>>> implementations. At that time we had a few custom serializers which we
>>>>> decided to throw out during migration and then leverage flink default
>>>>> serializers. We don't mind clearing the state in the process of migration,
>>>>> an effort to migrate with state seems to be not worth it.
>>>>>
>>>>> Unfortunately after running new version we see SIGSEGV errors from
>>>>> time to time. It may be that serialization is not the real cause, but at
>>>>> the moment it seems to be the most probable reason. We have not performed
>>>>> any significant code changes besides serialization area.
>>>>>
>>>>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>>>>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>>>>> processing time
>>>>>
>>>>> You can find parts of JobManager log and ErrorFile log of failed
>>>>> container included below.
>>>>>
>>>>> Any suggestions are welcome
>>>>>
>>>>> Best regards
>>>>> Marek Maj
>>>>>
>>>>> jobmanager.log
>>>>>
>>>>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>>>>> (18532488122 bytes in 60871 ms).
>>>>>
>>>>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>> Triggering checkpoint 48 @ 1568111478177 for job
>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>
>>>>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>>>>> (19049515705 bytes in 61083 ms).
>>>>>
>>>>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>> Triggering checkpoint 49 @ 1568111589279 for job
>>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>>
>>>>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>> Requesting TaskManager's path for query services failed.
>>>>>
>>>>> java.util.concurrent.CompletionException:
>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>
>>>>> 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:816)
>>>>>
>>>>> 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:74)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>
>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>
>>>>> at
>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>
>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>
>>>>> ... 9 common frames omitted
>>>>>
>>>>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>> Requesting TaskManager's path for query services failed.
>>>>>
>>>>> java.util.concurrent.CompletionException:
>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>
>>>>> 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:816)
>>>>>
>>>>> 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:74)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>
>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>
>>>>> at
>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>
>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>
>>>>> ... 9 common frames omitted
>>>>>
>>>>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>>> Requesting TaskManager's path for query services failed.
>>>>>
>>>>> java.util.concurrent.CompletionException:
>>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>>
>>>>> at
>>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>>
>>>>> 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:816)
>>>>>
>>>>> 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:74)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>>
>>>>> at
>>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>>
>>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>>
>>>>> at
>>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>>
>>>>> at
>>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>>
>>>>> at
>>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>>
>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>>> Sender[null] sent message of type
>>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>>
>>>>> at
>>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>>
>>>>> ... 9 common frames omitted
>>>>>
>>>>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>> The heartbeat of TaskManager with id
>>>>> container_e67_1568017536744_0044_01_000023 timed out.
>>>>>
>>>>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>>>>> because: The heartbeat of TaskManager with id
>>>>> container_e67_1568017536744_0044_01_000023  timed out.
>>>>>
>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
>>>>> (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.
>>>>>
>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>
>>>>> at
>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>
>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>
>>>>> 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)
>>>>>
>>>>> 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)
>>>>>
>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>>>>
>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>
>>>>> at
>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>
>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>
>>>>> 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)
>>>>>
>>>>> 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)
>>>>>
>>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>>>>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>>>>> RUNNING to FAILING.
>>>>>
>>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>>
>>>>> at
>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>>
>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>>
>>>>> 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)
>>>>>
>>>>> 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)
>>>>>
>>>>>
>>>>>
>>>>> hs_err_pid_262348.log for failed container
>>>>>
>>>>> #
>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>> #
>>>>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>>>>> tid=0x00007f2916833700
>>>>> #
>>>>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>>>>> 1.8.0_112-b15)
>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>>>>> linux-amd64 compressed oops)
>>>>> # Problematic frame:
>>>>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>> #
>>>>> # Core dump written. Default location:
>>>>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>>>>> or core.262348
>>>>> #
>>>>> # If you would like to submit a bug report, please visit:
>>>>> #   http://bugreport.java.com/bugreport/crash.jsp
>>>>> # The crash happened outside the Java Virtual Machine in native code.
>>>>> # See problematic frame for where to report the bug.
>>>>> #
>>>>>
>>>>> ---------------  T H R E A D  ---------------
>>>>>
>>>>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>>>>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>>>>
>>>>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>>>>> 0x0000000000001080
>>>>>
>>>>> Registers:
>>>>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770,
>>>>> RCX=0x0000000000000180, RDX=0x00007f2945e52770
>>>>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0,
>>>>> RSI=0x0000000000001040, RDI=0x00007f2945e52770
>>>>> R8 =0x00000007bff0f170, R9 =0x0000000000000006,
>>>>> R10=0x00007f2935017a08, R11=0x00007f294b583d50
>>>>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58,
>>>>> R14=0x00007f2916832490, R15=0x00007f29440e8000
>>>>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>>>>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>>>>   TRAPNO=0x000000000000000e
>>>>>
>>>>> Top of Stack: (sp=0x00007f29168323d0)
>>>>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>>>>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>>>>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>>>>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>>>>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>>>>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>>>>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>>>>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>>>>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>>>>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>>>>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>>>>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>>>>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>>>>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>>>>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>>>>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>>>>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>>>>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>>>>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>>>>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>>>>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>>>>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>>>>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>>>>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>>>>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>>>>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>>>>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>>>>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>>>>> 0x00007f2916832590:   0000000000000000 0000000000000000
>>>>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>>>>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>>>>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>>>>
>>>>> Instructions: (pc=0x00007f294944b2c2)
>>>>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>>>>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>>>>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>>>>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>>>>>
>>>>> Register to memory mapping:
>>>>>
>>>>> RAX=0x00007f0100000001 is an unknown value
>>>>> RBX=0x00007f2945e52770 is an unknown value
>>>>> RCX=0x0000000000000180 is an unknown value
>>>>> RDX=0x00007f2945e52770 is an unknown value
>>>>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>>>>> 0x00007f29440e8000
>>>>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>>>>> 0x00007f29440e8000
>>>>> RSI=0x0000000000001040 is an unknown value
>>>>> RDI=0x00007f2945e52770 is an unknown value
>>>>> R8 =0x00000007bff0f170 is an oop
>>>>> [Ljava.lang.Object;
>>>>>  - klass: 'java/lang/Object'[]
>>>>>  - length: 16
>>>>> R9 =0x0000000000000006 is an unknown value
>>>>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>>>>> method entry point (kind = native)  [0x00007f29350176e0,
>>>>> 0x00007f2935017fe0]  2304 bytes
>>>>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>>>>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>>>>> R12=0x00007f29440e81f8 is an unknown value
>>>>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in
>>>>> 'java/util/zip/Inflater'
>>>>> R14=0x00007f2916832490 is pointing into the stack for thread:
>>>>> 0x00007f29440e8000
>>>>> R15=0x00007f29440e8000 is a thread
>>>>>
>>>>>
>>>>> Stack: [0x00007f2916733000,0x00007f2916834000],
>>>>>  sp=0x00007f29168323d0,  free space=1020k
>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>>> C=native code)
>>>>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>> j  java.util.zip.Inflater.end()V+29
>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>> v  ~StubRoutines::call_stub
>>>>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>>>>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>>>>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*,
>>>>> KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>>>>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>>>>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>>>>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>>>>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>>>>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>>>>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>>>>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>>>>
>>>>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>>>>> j  java.util.zip.Inflater.end(J)V+0
>>>>> j  java.util.zip.Inflater.end()V+29
>>>>> j  java.util.zip.ZipFile.close()V+169
>>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>>> v  ~StubRoutines::call_stub
>>>>>
>>>>

Re: SIGSEGV error

Posted by Joshua Fan <jo...@gmail.com>.
Hi all,

Most of the posts says that "Most of the times, the crashes in ZIP_GetEntry
occur when the jar file being accessed has been modified/overwritten while
the JVM instance was running. ", but do not know when and which jar file
was modified according to the job running in flink.

for your information.

Yours sincerely
Josh

Joshua Fan <jo...@gmail.com> 于2021年5月18日周二 上午10:15写道:

> Hi Stephan, Till
>
> Recently, I tried to upgrade a flink job from 1.7 to 1.11, unfortunately,
> the weird problem appeared, " SIGSEGV (0xb) at pc=0x0000000000000025,
> pid=135306, tid=140439001388800".  The pid log is attached.
> Actually, it is a simple job that consumes messages from kafka and writes
> into hdfs with a gzip format. It can run in 1.11 for about 2 minutes, then
> the JVM will crash, then job restart and jvm crash again until the
> application fails.
> I also tried to set -Dsun.zip.disableMemoryMapping=true,but it turns out
> helpless, the same crash keeps happening. Google suggests to upgrade jdk to
> jdk1.9, but it is not feasible.
> Any suggestions? Thanks a lot.
>
> Yours sincerely
> Josh
>
> Stephan Ewen <se...@apache.org> 于2019年9月13日周五 下午11:11写道:
>
>> Given that the segfault happens in the JVM's ZIP stream code, I am
>> curious is this is a bug in Flink or in the JVM core libs, that happens to
>> be triggered now by newer versions of FLink.
>>
>> I found this on StackOverflow, which looks like it could be related:
>> https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
>> Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"?
>>
>>
>> On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org>
>> wrote:
>>
>>> Hi Marek,
>>>
>>> could you share the logs statements which happened before the SIGSEGV
>>> with us? They might be helpful to understand what happened before.
>>> Moreover, it would be helpful to get access to your custom serializer
>>> implementations. I'm also pulling in Gordon who worked on
>>> the TypeSerializerSnapshot improvements.
>>>
>>> Cheers,
>>> Till
>>>
>>> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:
>>>
>>>> Hi everyone,
>>>>
>>>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>>>> upgrade our task managers started to fail with SIGSEGV error from time to
>>>> time.
>>>>
>>>> In process of adjusting the code to 1.8.1, we noticed that there were
>>>> some changes around TypeSerializerSnapshot interface and its
>>>> implementations. At that time we had a few custom serializers which we
>>>> decided to throw out during migration and then leverage flink default
>>>> serializers. We don't mind clearing the state in the process of migration,
>>>> an effort to migrate with state seems to be not worth it.
>>>>
>>>> Unfortunately after running new version we see SIGSEGV errors from time
>>>> to time. It may be that serialization is not the real cause, but at the
>>>> moment it seems to be the most probable reason. We have not performed any
>>>> significant code changes besides serialization area.
>>>>
>>>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>>>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>>>> processing time
>>>>
>>>> You can find parts of JobManager log and ErrorFile log of failed
>>>> container included below.
>>>>
>>>> Any suggestions are welcome
>>>>
>>>> Best regards
>>>> Marek Maj
>>>>
>>>> jobmanager.log
>>>>
>>>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>>>> (18532488122 bytes in 60871 ms).
>>>>
>>>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>> Triggering checkpoint 48 @ 1568111478177 for job
>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>
>>>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>>>> (19049515705 bytes in 61083 ms).
>>>>
>>>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>> Triggering checkpoint 49 @ 1568111589279 for job
>>>> c8a9ae03785ade86348c3189cf7dd965.
>>>>
>>>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>> Requesting TaskManager's path for query services failed.
>>>>
>>>> java.util.concurrent.CompletionException:
>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>
>>>> 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:816)
>>>>
>>>> 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:74)
>>>>
>>>> at
>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>
>>>> at
>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>
>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>
>>>> at
>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>
>>>> at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>
>>>> ... 9 common frames omitted
>>>>
>>>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>> Requesting TaskManager's path for query services failed.
>>>>
>>>> java.util.concurrent.CompletionException:
>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>
>>>> 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:816)
>>>>
>>>> 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:74)
>>>>
>>>> at
>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>
>>>> at
>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>
>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>
>>>> at
>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>
>>>> at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>
>>>> ... 9 common frames omitted
>>>>
>>>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>>> Requesting TaskManager's path for query services failed.
>>>>
>>>> java.util.concurrent.CompletionException:
>>>> akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>>
>>>> at
>>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>>
>>>> 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:816)
>>>>
>>>> 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:74)
>>>>
>>>> at
>>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>>
>>>> at
>>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>>
>>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>>
>>>> at
>>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>>
>>>> at
>>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>>
>>>> at
>>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>>
>>>> at java.lang.Thread.run(Thread.java:745)
>>>>
>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>>> Sender[null] sent message of type
>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>>
>>>> at
>>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>>
>>>> ... 9 common frames omitted
>>>>
>>>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>> The heartbeat of TaskManager with id
>>>> container_e67_1568017536744_0044_01_000023 timed out.
>>>>
>>>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>>>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>>>> because: The heartbeat of TaskManager with id
>>>> container_e67_1568017536744_0044_01_000023  timed out.
>>>>
>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
>>>> (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.
>>>>
>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>
>>>> at
>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>
>>>> at
>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>
>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>
>>>> 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)
>>>>
>>>> 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)
>>>>
>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>>>
>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>
>>>> at
>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>
>>>> at
>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>
>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>
>>>> 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)
>>>>
>>>> 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)
>>>>
>>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>>>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>>>> RUNNING to FAILING.
>>>>
>>>> org.apache.flink.util.FlinkException: The assigned slot
>>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>>
>>>> at
>>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>>
>>>> at
>>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>>
>>>> at
>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>>
>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>>
>>>> at
>>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>>
>>>> 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)
>>>>
>>>> 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)
>>>>
>>>>
>>>>
>>>> hs_err_pid_262348.log for failed container
>>>>
>>>> #
>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>> #
>>>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>>>> tid=0x00007f2916833700
>>>> #
>>>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>>>> 1.8.0_112-b15)
>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>>>> linux-amd64 compressed oops)
>>>> # Problematic frame:
>>>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>> #
>>>> # Core dump written. Default location:
>>>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>>>> or core.262348
>>>> #
>>>> # If you would like to submit a bug report, please visit:
>>>> #   http://bugreport.java.com/bugreport/crash.jsp
>>>> # The crash happened outside the Java Virtual Machine in native code.
>>>> # See problematic frame for where to report the bug.
>>>> #
>>>>
>>>> ---------------  T H R E A D  ---------------
>>>>
>>>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>>>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>>>
>>>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>>>> 0x0000000000001080
>>>>
>>>> Registers:
>>>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770, RCX=0x0000000000000180,
>>>> RDX=0x00007f2945e52770
>>>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0, RSI=0x0000000000001040,
>>>> RDI=0x00007f2945e52770
>>>> R8 =0x00000007bff0f170, R9 =0x0000000000000006, R10=0x00007f2935017a08,
>>>> R11=0x00007f294b583d50
>>>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58, R14=0x00007f2916832490,
>>>> R15=0x00007f29440e8000
>>>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>>>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>>>   TRAPNO=0x000000000000000e
>>>>
>>>> Top of Stack: (sp=0x00007f29168323d0)
>>>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>>>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>>>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>>>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>>>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>>>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>>>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>>>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>>>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>>>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>>>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>>>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>>>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>>>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>>>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>>>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>>>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>>>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>>>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>>>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>>>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>>>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>>>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>>>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>>>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>>>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>>>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>>>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>>>> 0x00007f2916832590:   0000000000000000 0000000000000000
>>>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>>>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>>>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>>>
>>>> Instructions: (pc=0x00007f294944b2c2)
>>>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>>>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>>>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>>>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>>>>
>>>> Register to memory mapping:
>>>>
>>>> RAX=0x00007f0100000001 is an unknown value
>>>> RBX=0x00007f2945e52770 is an unknown value
>>>> RCX=0x0000000000000180 is an unknown value
>>>> RDX=0x00007f2945e52770 is an unknown value
>>>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>>>> 0x00007f29440e8000
>>>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>>>> 0x00007f29440e8000
>>>> RSI=0x0000000000001040 is an unknown value
>>>> RDI=0x00007f2945e52770 is an unknown value
>>>> R8 =0x00000007bff0f170 is an oop
>>>> [Ljava.lang.Object;
>>>>  - klass: 'java/lang/Object'[]
>>>>  - length: 16
>>>> R9 =0x0000000000000006 is an unknown value
>>>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>>>> method entry point (kind = native)  [0x00007f29350176e0,
>>>> 0x00007f2935017fe0]  2304 bytes
>>>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>>>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>>>> R12=0x00007f29440e81f8 is an unknown value
>>>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in
>>>> 'java/util/zip/Inflater'
>>>> R14=0x00007f2916832490 is pointing into the stack for thread:
>>>> 0x00007f29440e8000
>>>> R15=0x00007f29440e8000 is a thread
>>>>
>>>>
>>>> Stack: [0x00007f2916733000,0x00007f2916834000],  sp=0x00007f29168323d0,
>>>>  free space=1020k
>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>> C=native code)
>>>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>>>> j  java.util.zip.Inflater.end(J)V+0
>>>> j  java.util.zip.Inflater.end()V+29
>>>> j  java.util.zip.ZipFile.close()V+169
>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>> v  ~StubRoutines::call_stub
>>>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>>>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>>>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*,
>>>> KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>>>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>>>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>>>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>>>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>>>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>>>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>>>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>>>
>>>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>>>> j  java.util.zip.Inflater.end(J)V+0
>>>> j  java.util.zip.Inflater.end()V+29
>>>> j  java.util.zip.ZipFile.close()V+169
>>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>>> v  ~StubRoutines::call_stub
>>>>
>>>

Re: SIGSEGV error

Posted by Joshua Fan <jo...@gmail.com>.
Hi Stephan, Till

Recently, I tried to upgrade a flink job from 1.7 to 1.11, unfortunately,
the weird problem appeared, " SIGSEGV (0xb) at pc=0x0000000000000025,
pid=135306, tid=140439001388800".  The pid log is attached.
Actually, it is a simple job that consumes messages from kafka and writes
into hdfs with a gzip format. It can run in 1.11 for about 2 minutes, then
the JVM will crash, then job restart and jvm crash again until the
application fails.
I also tried to set -Dsun.zip.disableMemoryMapping=true,but it turns out
helpless, the same crash keeps happening. Google suggests to upgrade jdk to
jdk1.9, but it is not feasible.
Any suggestions? Thanks a lot.

Yours sincerely
Josh

Stephan Ewen <se...@apache.org> 于2019年9月13日周五 下午11:11写道:

> Given that the segfault happens in the JVM's ZIP stream code, I am curious
> is this is a bug in Flink or in the JVM core libs, that happens to be
> triggered now by newer versions of FLink.
>
> I found this on StackOverflow, which looks like it could be related:
> https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
> Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"?
>
>
> On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org>
> wrote:
>
>> Hi Marek,
>>
>> could you share the logs statements which happened before the SIGSEGV
>> with us? They might be helpful to understand what happened before.
>> Moreover, it would be helpful to get access to your custom serializer
>> implementations. I'm also pulling in Gordon who worked on
>> the TypeSerializerSnapshot improvements.
>>
>> Cheers,
>> Till
>>
>> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:
>>
>>> Hi everyone,
>>>
>>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>>> upgrade our task managers started to fail with SIGSEGV error from time to
>>> time.
>>>
>>> In process of adjusting the code to 1.8.1, we noticed that there were
>>> some changes around TypeSerializerSnapshot interface and its
>>> implementations. At that time we had a few custom serializers which we
>>> decided to throw out during migration and then leverage flink default
>>> serializers. We don't mind clearing the state in the process of migration,
>>> an effort to migrate with state seems to be not worth it.
>>>
>>> Unfortunately after running new version we see SIGSEGV errors from time
>>> to time. It may be that serialization is not the real cause, but at the
>>> moment it seems to be the most probable reason. We have not performed any
>>> significant code changes besides serialization area.
>>>
>>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>>> processing time
>>>
>>> You can find parts of JobManager log and ErrorFile log of failed
>>> container included below.
>>>
>>> Any suggestions are welcome
>>>
>>> Best regards
>>> Marek Maj
>>>
>>> jobmanager.log
>>>
>>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>>> (18532488122 bytes in 60871 ms).
>>>
>>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>> Triggering checkpoint 48 @ 1568111478177 for job
>>> c8a9ae03785ade86348c3189cf7dd965.
>>>
>>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>>> (19049515705 bytes in 61083 ms).
>>>
>>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>> Triggering checkpoint 49 @ 1568111589279 for job
>>> c8a9ae03785ade86348c3189cf7dd965.
>>>
>>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>> Requesting TaskManager's path for query services failed.
>>>
>>> java.util.concurrent.CompletionException:
>>> akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>
>>> 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:816)
>>>
>>> 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:74)
>>>
>>> at
>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>
>>> at
>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>
>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>
>>> at
>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>
>>> ... 9 common frames omitted
>>>
>>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>> Requesting TaskManager's path for query services failed.
>>>
>>> java.util.concurrent.CompletionException:
>>> akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>
>>> 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:816)
>>>
>>> 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:74)
>>>
>>> at
>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>
>>> at
>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>
>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>
>>> at
>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>
>>> ... 9 common frames omitted
>>>
>>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>>> Requesting TaskManager's path for query services failed.
>>>
>>> java.util.concurrent.CompletionException:
>>> akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>>
>>> at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>>
>>> 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:816)
>>>
>>> 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:74)
>>>
>>> at
>>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>>
>>> at
>>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>>
>>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>>
>>> at
>>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>>
>>> at
>>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>>
>>> at
>>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>>
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>>> Sender[null] sent message of type
>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>>
>>> at
>>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>>
>>> ... 9 common frames omitted
>>>
>>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   - The
>>> heartbeat of TaskManager with id container_e67_1568017536744_0044_01_000023
>>> timed out.
>>>
>>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>>> because: The heartbeat of TaskManager with id
>>> container_e67_1568017536744_0044_01_000023  timed out.
>>>
>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
>>> (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.
>>>
>>> org.apache.flink.util.FlinkException: The assigned slot
>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>
>>> at
>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>
>>> at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>
>>> 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)
>>>
>>> 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)
>>>
>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>>
>>> org.apache.flink.util.FlinkException: The assigned slot
>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>
>>> at
>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>
>>> at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>
>>> 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)
>>>
>>> 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)
>>>
>>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>>> RUNNING to FAILING.
>>>
>>> org.apache.flink.util.FlinkException: The assigned slot
>>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>>
>>> at
>>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>>
>>> at
>>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>>
>>> at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>>
>>> at
>>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>>
>>> 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)
>>>
>>> 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)
>>>
>>>
>>>
>>> hs_err_pid_262348.log for failed container
>>>
>>> #
>>> # A fatal error has been detected by the Java Runtime Environment:
>>> #
>>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>>> tid=0x00007f2916833700
>>> #
>>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>>> 1.8.0_112-b15)
>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>>> linux-amd64 compressed oops)
>>> # Problematic frame:
>>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>> #
>>> # Core dump written. Default location:
>>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>>> or core.262348
>>> #
>>> # If you would like to submit a bug report, please visit:
>>> #   http://bugreport.java.com/bugreport/crash.jsp
>>> # The crash happened outside the Java Virtual Machine in native code.
>>> # See problematic frame for where to report the bug.
>>> #
>>>
>>> ---------------  T H R E A D  ---------------
>>>
>>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>>
>>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>>> 0x0000000000001080
>>>
>>> Registers:
>>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770, RCX=0x0000000000000180,
>>> RDX=0x00007f2945e52770
>>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0, RSI=0x0000000000001040,
>>> RDI=0x00007f2945e52770
>>> R8 =0x00000007bff0f170, R9 =0x0000000000000006, R10=0x00007f2935017a08,
>>> R11=0x00007f294b583d50
>>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58, R14=0x00007f2916832490,
>>> R15=0x00007f29440e8000
>>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>>   TRAPNO=0x000000000000000e
>>>
>>> Top of Stack: (sp=0x00007f29168323d0)
>>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>>> 0x00007f2916832590:   0000000000000000 0000000000000000
>>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>>
>>> Instructions: (pc=0x00007f294944b2c2)
>>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>>>
>>> Register to memory mapping:
>>>
>>> RAX=0x00007f0100000001 is an unknown value
>>> RBX=0x00007f2945e52770 is an unknown value
>>> RCX=0x0000000000000180 is an unknown value
>>> RDX=0x00007f2945e52770 is an unknown value
>>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>>> 0x00007f29440e8000
>>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>>> 0x00007f29440e8000
>>> RSI=0x0000000000001040 is an unknown value
>>> RDI=0x00007f2945e52770 is an unknown value
>>> R8 =0x00000007bff0f170 is an oop
>>> [Ljava.lang.Object;
>>>  - klass: 'java/lang/Object'[]
>>>  - length: 16
>>> R9 =0x0000000000000006 is an unknown value
>>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>>> method entry point (kind = native)  [0x00007f29350176e0,
>>> 0x00007f2935017fe0]  2304 bytes
>>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>>> R12=0x00007f29440e81f8 is an unknown value
>>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in
>>> 'java/util/zip/Inflater'
>>> R14=0x00007f2916832490 is pointing into the stack for thread:
>>> 0x00007f29440e8000
>>> R15=0x00007f29440e8000 is a thread
>>>
>>>
>>> Stack: [0x00007f2916733000,0x00007f2916834000],  sp=0x00007f29168323d0,
>>>  free space=1020k
>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>> C=native code)
>>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>>> j  java.util.zip.Inflater.end(J)V+0
>>> j  java.util.zip.Inflater.end()V+29
>>> j  java.util.zip.ZipFile.close()V+169
>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>> v  ~StubRoutines::call_stub
>>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*,
>>> KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>>
>>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>>> j  java.util.zip.Inflater.end(J)V+0
>>> j  java.util.zip.Inflater.end()V+29
>>> j  java.util.zip.ZipFile.close()V+169
>>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>>> v  ~StubRoutines::call_stub
>>>
>>

Re: SIGSEGV error

Posted by Stephan Ewen <se...@apache.org>.
Given that the segfault happens in the JVM's ZIP stream code, I am curious
is this is a bug in Flink or in the JVM core libs, that happens to be
triggered now by newer versions of FLink.

I found this on StackOverflow, which looks like it could be related:
https://stackoverflow.com/questions/38326183/jvm-crashed-in-java-util-zip-zipfile-getentry
Can you try the suggested option "-Dsun.zip.disableMemoryMapping=true"?


On Fri, Sep 13, 2019 at 11:36 AM Till Rohrmann <tr...@apache.org> wrote:

> Hi Marek,
>
> could you share the logs statements which happened before the SIGSEGV with
> us? They might be helpful to understand what happened before. Moreover, it
> would be helpful to get access to your custom serializer implementations.
> I'm also pulling in Gordon who worked on the TypeSerializerSnapshot
> improvements.
>
> Cheers,
> Till
>
> On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:
>
>> Hi everyone,
>>
>> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an
>> upgrade our task managers started to fail with SIGSEGV error from time to
>> time.
>>
>> In process of adjusting the code to 1.8.1, we noticed that there were
>> some changes around TypeSerializerSnapshot interface and its
>> implementations. At that time we had a few custom serializers which we
>> decided to throw out during migration and then leverage flink default
>> serializers. We don't mind clearing the state in the process of migration,
>> an effort to migrate with state seems to be not worth it.
>>
>> Unfortunately after running new version we see SIGSEGV errors from time
>> to time. It may be that serialization is not the real cause, but at the
>> moment it seems to be the most probable reason. We have not performed any
>> significant code changes besides serialization area.
>>
>> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
>> Checkpoint configuration: RocksDB backend, not incremental, 50s min
>> processing time
>>
>> You can find parts of JobManager log and ErrorFile log of failed
>> container included below.
>>
>> Any suggestions are welcome
>>
>> Best regards
>> Marek Maj
>>
>> jobmanager.log
>>
>> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   -
>> Completed checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965
>> (18532488122 bytes in 60871 ms).
>>
>> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
>> Triggering checkpoint 48 @ 1568111478177 for job
>> c8a9ae03785ade86348c3189cf7dd965.
>>
>> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
>> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
>> (19049515705 bytes in 61083 ms).
>>
>> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
>> Triggering checkpoint 49 @ 1568111589279 for job
>> c8a9ae03785ade86348c3189cf7dd965.
>>
>> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>> Requesting TaskManager's path for query services failed.
>>
>> java.util.concurrent.CompletionException:
>> akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>
>> at
>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>
>> at
>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>
>> at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>
>> 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:816)
>>
>> 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:74)
>>
>> at
>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>
>> at
>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>
>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>
>> at
>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>
>> at java.lang.Thread.run(Thread.java:745)
>>
>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>
>> ... 9 common frames omitted
>>
>> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>> Requesting TaskManager's path for query services failed.
>>
>> java.util.concurrent.CompletionException:
>> akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>
>> at
>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>
>> at
>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>
>> at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>
>> 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:816)
>>
>> 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:74)
>>
>> at
>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>
>> at
>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>
>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>
>> at
>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>
>> at java.lang.Thread.run(Thread.java:745)
>>
>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>
>> ... 9 common frames omitted
>>
>> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
>> Requesting TaskManager's path for query services failed.
>>
>> java.util.concurrent.CompletionException:
>> akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>>
>> at
>> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>>
>> at
>> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>>
>> at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>>
>> 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:816)
>>
>> 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:74)
>>
>> at
>> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>>
>> at
>> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>>
>> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>>
>> at
>> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>>
>> at
>> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>>
>> at
>> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>>
>> at java.lang.Thread.run(Thread.java:745)
>>
>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
>> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
>> Sender[null] sent message of type
>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>>
>> at
>> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>>
>> ... 9 common frames omitted
>>
>> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   - The
>> heartbeat of TaskManager with id container_e67_1568017536744_0044_01_000023
>> timed out.
>>
>> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
>> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
>> because: The heartbeat of TaskManager with id
>> container_e67_1568017536744_0044_01_000023  timed out.
>>
>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
>> (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.
>>
>> org.apache.flink.util.FlinkException: The assigned slot
>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>
>> at
>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>
>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>
>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>
>> 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)
>>
>> 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)
>>
>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
>> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>>
>> org.apache.flink.util.FlinkException: The assigned slot
>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>
>> at
>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>
>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>
>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>
>> 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)
>>
>> 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)
>>
>> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
>> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
>> RUNNING to FAILING.
>>
>> org.apache.flink.util.FlinkException: The assigned slot
>> container_e67_1568017536744_0044_01_000023_0 was removed.
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>>
>> at
>> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>>
>> at
>> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>>
>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>
>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>>
>> at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>>
>> 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)
>>
>> 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)
>>
>>
>>
>> hs_err_pid_262348.log for failed container
>>
>> #
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
>> tid=0x00007f2916833700
>> #
>> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
>> 1.8.0_112-b15)
>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
>> linux-amd64 compressed oops)
>> # Problematic frame:
>> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
>> #
>> # Core dump written. Default location:
>> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
>> or core.262348
>> #
>> # If you would like to submit a bug report, please visit:
>> #   http://bugreport.java.com/bugreport/crash.jsp
>> # The crash happened outside the Java Virtual Machine in native code.
>> # See problematic frame for where to report the bug.
>> #
>>
>> ---------------  T H R E A D  ---------------
>>
>> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
>> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>>
>> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
>> 0x0000000000001080
>>
>> Registers:
>> RAX=0x00007f0100000001, RBX=0x00007f2945e52770, RCX=0x0000000000000180,
>> RDX=0x00007f2945e52770
>> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0, RSI=0x0000000000001040,
>> RDI=0x00007f2945e52770
>> R8 =0x00000007bff0f170, R9 =0x0000000000000006, R10=0x00007f2935017a08,
>> R11=0x00007f294b583d50
>> R12=0x00007f29440e81f8, R13=0x00007f293135cc58, R14=0x00007f2916832490,
>> R15=0x00007f29440e8000
>> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
>> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>>   TRAPNO=0x000000000000000e
>>
>> Top of Stack: (sp=0x00007f29168323d0)
>> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
>> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
>> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
>> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
>> 0x00007f2916832410:   00007f2916832540 00007f293501306d
>> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
>> 0x00007f2916832430:   0000000000000000 00007f2916832490
>> 0x00007f2916832440:   00007f293135cd70 0000000000000000
>> 0x00007f2916832450:   00007f293135cc58 0000000000000000
>> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
>> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
>> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
>> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
>> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
>> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
>> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
>> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
>> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
>> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
>> 0x00007f2916832500:   00007f2945e52770 0000000000000000
>> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
>> 0x00007f2916832520:   0000000000000009 00000007bff0f158
>> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
>> 0x00007f2916832540:   00007f2916832520 00007f293132f631
>> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
>> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
>> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
>> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
>> 0x00007f2916832590:   0000000000000000 0000000000000000
>> 0x00007f29168325a0:   0000000000000000 0000000000000000
>> 0x00007f29168325b0:   0000000000000000 0000000000000000
>> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>>
>> Instructions: (pc=0x00007f294944b2c2)
>> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
>> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
>> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
>> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>>
>> Register to memory mapping:
>>
>> RAX=0x00007f0100000001 is an unknown value
>> RBX=0x00007f2945e52770 is an unknown value
>> RCX=0x0000000000000180 is an unknown value
>> RDX=0x00007f2945e52770 is an unknown value
>> RSP=0x00007f29168323d0 is pointing into the stack for thread:
>> 0x00007f29440e8000
>> RBP=0x00007f29168323e0 is pointing into the stack for thread:
>> 0x00007f29440e8000
>> RSI=0x0000000000001040 is an unknown value
>> RDI=0x00007f2945e52770 is an unknown value
>> R8 =0x00000007bff0f170 is an oop
>> [Ljava.lang.Object;
>>  - klass: 'java/lang/Object'[]
>>  - length: 16
>> R9 =0x0000000000000006 is an unknown value
>> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
>> method entry point (kind = native)  [0x00007f29350176e0,
>> 0x00007f2935017fe0]  2304 bytes
>> R11=0x00007f294b583d50: <offset 0x9c3d50> in
>> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
>> R12=0x00007f29440e81f8 is an unknown value
>> R13={method} {0x00007f293135cc58} 'end' '(J)V' in 'java/util/zip/Inflater'
>> R14=0x00007f2916832490 is pointing into the stack for thread:
>> 0x00007f29440e8000
>> R15=0x00007f29440e8000 is a thread
>>
>>
>> Stack: [0x00007f2916733000,0x00007f2916834000],  sp=0x00007f29168323d0,
>>  free space=1020k
>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
>> code)
>> C  [libzip.so+0xb2c2]  inflateEnd+0x32
>> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
>> j  java.util.zip.Inflater.end(J)V+0
>> j  java.util.zip.Inflater.end()V+29
>> j  java.util.zip.ZipFile.close()V+169
>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>> v  ~StubRoutines::call_stub
>> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*,
>> methodHandle*, JavaCallArguments*, Thread*)+0x1056
>> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*, KlassHandle,
>> Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
>> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
>> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
>> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
>> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
>> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
>> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
>> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>>
>> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
>> j  java.util.zip.Inflater.end(J)V+0
>> j  java.util.zip.Inflater.end()V+29
>> j  java.util.zip.ZipFile.close()V+169
>> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
>> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
>> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
>> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
>> v  ~StubRoutines::call_stub
>>
>

Re: SIGSEGV error

Posted by Till Rohrmann <tr...@apache.org>.
Hi Marek,

could you share the logs statements which happened before the SIGSEGV with
us? They might be helpful to understand what happened before. Moreover, it
would be helpful to get access to your custom serializer implementations.
I'm also pulling in Gordon who worked on the TypeSerializerSnapshot
improvements.

Cheers,
Till

On Thu, Sep 12, 2019 at 9:28 AM Marek Maj <ma...@gmail.com> wrote:

> Hi everyone,
>
> Recently we decided to upgrade from flink 1.7.2 to 1.8.1. After an upgrade
> our task managers started to fail with SIGSEGV error from time to time.
>
> In process of adjusting the code to 1.8.1, we noticed that there were some
> changes around TypeSerializerSnapshot interface and its implementations.
> At that time we had a few custom serializers which we decided to throw out
> during migration and then leverage flink default serializers. We don't mind
> clearing the state in the process of migration, an effort to migrate with
> state seems to be not worth it.
>
> Unfortunately after running new version we see SIGSEGV errors from time to
> time. It may be that serialization is not the real cause, but at the moment
> it seems to be the most probable reason. We have not performed any
> significant code changes besides serialization area.
>
> We run job on yarn, hdp version 2.7.3.2.6.2.0-205.
> Checkpoint configuration: RocksDB backend, not incremental, 50s min
> processing time
>
> You can find parts of JobManager log and ErrorFile log of failed container
> included below.
>
> Any suggestions are welcome
>
> Best regards
> Marek Maj
>
> jobmanager.log
>
> 019-09-10 16:30:28.177 INFO  o.a.f.r.c.CheckpointCoordinator   - Completed
> checkpoint 47 for job c8a9ae03785ade86348c3189cf7dd965 (18532488122 bytes
> in 60871 ms).
>
> 2019-09-10 16:31:19.223 INFO  o.a.f.r.c.CheckpointCoordinator   -
> Triggering checkpoint 48 @ 1568111478177 for job
> c8a9ae03785ade86348c3189cf7dd965.
>
> 2019-09-10 16:32:19.280 INFO  o.a.f.r.c.CheckpointCoordinator   -
> Completed checkpoint 48 for job c8a9ae03785ade86348c3189cf7dd965
> (19049515705 bytes in 61083 ms).
>
> 2019-09-10 16:33:10.480 INFO  o.a.f.r.c.CheckpointCoordinator   -
> Triggering checkpoint 49 @ 1568111589279 for job
> c8a9ae03785ade86348c3189cf7dd965.
>
> 2019-09-10 16:33:36.773 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
> Requesting TaskManager's path for query services failed.
>
> java.util.concurrent.CompletionException:
> akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>
> at
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>
> at
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>
> at
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>
> 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:816)
>
> 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:74)
>
> at
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>
> at
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>
> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>
> at
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>
> at
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>
> at java.lang.Thread.run(Thread.java:745)
>
> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>
> ... 9 common frames omitted
>
> 2019-09-10 16:33:48.782 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
> Requesting TaskManager's path for query services failed.
>
> java.util.concurrent.CompletionException:
> akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>
> at
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>
> at
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>
> at
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>
> 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:816)
>
> 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:74)
>
> at
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>
> at
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>
> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>
> at
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>
> at
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>
> at java.lang.Thread.run(Thread.java:745)
>
> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>
> ... 9 common frames omitted
>
> 2019-09-10 16:34:00.802 WARN  o.a.f.r.r.h.l.m.MetricFetcherImpl   -
> Requesting TaskManager's path for query services failed.
>
> java.util.concurrent.CompletionException:
> akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>
> at
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>
> at
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
>
> at
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
>
> 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:816)
>
> 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:74)
>
> at
> scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
>
> at
> scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
>
> at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>
> at
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>
> at
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>
> at
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>
> at
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>
> at java.lang.Thread.run(Thread.java:745)
>
> Caused by: akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#374570759]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
> at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>
> ... 9 common frames omitted
>
> 2019-09-10 16:34:03.800 INFO  o.a.flink.yarn.YarnResourceManager   - The
> heartbeat of TaskManager with id container_e67_1568017536744_0044_01_000023
> timed out.
>
> 2019-09-10 16:34:03.801 INFO  o.a.flink.yarn.YarnResourceManager   -
> Closing TaskExecutor connection container_e67_1568017536744_0044_01_000023
> because: The heartbeat of TaskManager with id
> container_e67_1568017536744_0044_01_000023  timed out.
>
> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - my-function
> (1/32) (ae416d03ddc94a3633673c4050b8f2ae) switched from RUNNING to FAILED.
>
> org.apache.flink.util.FlinkException: The assigned slot
> container_e67_1568017536744_0044_01_000023_0 was removed.
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>
> at
> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>
> 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)
>
> 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)
>
> 2019-09-10 16:34:03.803 INFO  o.a.f.r.c.CheckpointCoordinator   -
> Discarding checkpoint 49 of job c8a9ae03785ade86348c3189cf7dd965.
>
> org.apache.flink.util.FlinkException: The assigned slot
> container_e67_1568017536744_0044_01_000023_0 was removed.
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>
> at
> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>
> 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)
>
> 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)
>
> 2019-09-10 16:34:03.803 INFO  o.a.f.r.e.ExecutionGraph   - Job
> ProcessingJob (c8a9ae03785ade86348c3189cf7dd965) switched from state
> RUNNING to FAILING.
>
> org.apache.flink.util.FlinkException: The assigned slot
> container_e67_1568017536744_0044_01_000023_0 was removed.
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlot(SlotManager.java:899)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.removeSlots(SlotManager.java:869)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.internalUnregisterTaskManager(SlotManager.java:1080)
>
> at
> org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager.unregisterTaskManager(SlotManager.java:391)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager.closeTaskManagerConnection(ResourceManager.java:845)
>
> at
> org.apache.flink.runtime.resourcemanager.ResourceManager$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(ResourceManager.java:1187)
>
> at
> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:392)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:185)
>
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:147)
>
> 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)
>
> 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)
>
>
>
> hs_err_pid_262348.log for failed container
>
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007f294944b2c2, pid=262348,
> tid=0x00007f2916833700
> #
> # JRE version: Java(TM) SE Runtime Environment (8.0_112-b15) (build
> 1.8.0_112-b15)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.112-b15 mixed mode
> linux-amd64 compressed oops)
> # Problematic frame:
> # C  [libzip.so+0xb2c2]  inflateEnd+0x32
> #
> # Core dump written. Default location:
> /data/hadoop/yarn/local/usercache/flink/appcache/application_1568017536744_0044/container_e67_1568017536744_0044_01_000023/core
> or core.262348
> #
> # If you would like to submit a bug report, please visit:
> #   http://bugreport.java.com/bugreport/crash.jsp
> # The crash happened outside the Java Virtual Machine in native code.
> # See problematic frame for where to report the bug.
> #
>
> ---------------  T H R E A D  ---------------
>
> Current thread (0x00007f29440e8000):  JavaThread "Finalizer" daemon
> [_thread_in_native, id=262401, stack(0x00007f2916733000,0x00007f2916834000)]
>
> siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr:
> 0x0000000000001080
>
> Registers:
> RAX=0x00007f0100000001, RBX=0x00007f2945e52770, RCX=0x0000000000000180,
> RDX=0x00007f2945e52770
> RSP=0x00007f29168323d0, RBP=0x00007f29168323e0, RSI=0x0000000000001040,
> RDI=0x00007f2945e52770
> R8 =0x00000007bff0f170, R9 =0x0000000000000006, R10=0x00007f2935017a08,
> R11=0x00007f294b583d50
> R12=0x00007f29440e81f8, R13=0x00007f293135cc58, R14=0x00007f2916832490,
> R15=0x00007f29440e8000
> RIP=0x00007f294944b2c2, EFLAGS=0x0000000000010202,
> CSGSFS=0x0000000000000033, ERR=0x0000000000000004
>   TRAPNO=0x000000000000000e
>
> Top of Stack: (sp=0x00007f29168323d0)
> 0x00007f29168323d0:   ffffffff440e8000 00007f2945e52770
> 0x00007f29168323e0:   00007f2916832400 00007f294944338e
> 0x00007f29168323f0:   00007f293135cc58 0000000000000000
> 0x00007f2916832400:   00007f2916832468 00007f2935017a34
> 0x00007f2916832410:   00007f2916832540 00007f293501306d
> 0x00007f2916832420:   00007f29350055d0 00007f2916832428
> 0x00007f2916832430:   0000000000000000 00007f2916832490
> 0x00007f2916832440:   00007f293135cd70 0000000000000000
> 0x00007f2916832450:   00007f293135cc58 0000000000000000
> 0x00007f2916832460:   00007f2916832488 00007f29168324e8
> 0x00007f2916832470:   00007f29350082bd 00000006ab616900
> 0x00007f2916832480:   00007f2935011538 00007f2945e52770
> 0x00007f2916832490:   00000007bff0f1e8 00000007bff0f1e8
> 0x00007f29168324a0:   00000007bff0f1e8 00007f2916832498
> 0x00007f29168324b0:   00007f293135c5e5 00007f2916832518
> 0x00007f29168324c0:   00007f293135cd70 00007f29313f9840
> 0x00007f29168324d0:   00007f293135c618 00007f2916832488
> 0x00007f29168324e0:   00007f2916832518 00007f2916832580
> 0x00007f29168324f0:   00007f29350082bd 0000000000000000
> 0x00007f2916832500:   00007f2945e52770 0000000000000000
> 0x00007f2916832510:   00000007bff0f1e8 00000007bff0cd38
> 0x00007f2916832520:   0000000000000009 00000007bff0f158
> 0x00007f2916832530:   0000006ce4720709 00000007bff0cd98
> 0x00007f2916832540:   00007f2916832520 00007f293132f631
> 0x00007f2916832550:   00007f29168325d8 00007f2931330ce0
> 0x00007f2916832560:   0000000000000000 00007f293132f6c0
> 0x00007f2916832570:   00007f2916832518 00007f29168325d8
> 0x00007f2916832580:   00007f2916832620 00007f29350082bd
> 0x00007f2916832590:   0000000000000000 0000000000000000
> 0x00007f29168325a0:   0000000000000000 0000000000000000
> 0x00007f29168325b0:   0000000000000000 0000000000000000
> 0x00007f29168325c0:   00000007bff0f158 00000007bff0cd38
>
> Instructions: (pc=0x00007f294944b2c2)
> 0x00007f294944b2a2:   fe ff ff ff 48 83 c4 08 5b c9 c3 0f 1f 00 48 8b
> 0x00007f294944b2b2:   77 28 48 85 f6 74 e8 48 8b 47 38 48 85 c0 74 df
> 0x00007f294944b2c2:   48 8b 56 40 48 85 d2 74 11 48 89 d6 48 8b 7f 40
> 0x00007f294944b2d2:   ff d0 48 8b 43 38 48 8b 73 28 48 8b 7b 40 ff d0
>
> Register to memory mapping:
>
> RAX=0x00007f0100000001 is an unknown value
> RBX=0x00007f2945e52770 is an unknown value
> RCX=0x0000000000000180 is an unknown value
> RDX=0x00007f2945e52770 is an unknown value
> RSP=0x00007f29168323d0 is pointing into the stack for thread:
> 0x00007f29440e8000
> RBP=0x00007f29168323e0 is pointing into the stack for thread:
> 0x00007f29440e8000
> RSI=0x0000000000001040 is an unknown value
> RDI=0x00007f2945e52770 is an unknown value
> R8 =0x00000007bff0f170 is an oop
> [Ljava.lang.Object;
>  - klass: 'java/lang/Object'[]
>  - length: 16
> R9 =0x0000000000000006 is an unknown value
> R10=0x00007f2935017a08 is at code_begin+808 in an Interpreter codelet
> method entry point (kind = native)  [0x00007f29350176e0,
> 0x00007f2935017fe0]  2304 bytes
> R11=0x00007f294b583d50: <offset 0x9c3d50> in
> /usr/jdk64/jdk1.8.0_112/jre/lib/amd64/server/libjvm.so at 0x00007f294abc0000
> R12=0x00007f29440e81f8 is an unknown value
> R13={method} {0x00007f293135cc58} 'end' '(J)V' in 'java/util/zip/Inflater'
> R14=0x00007f2916832490 is pointing into the stack for thread:
> 0x00007f29440e8000
> R15=0x00007f29440e8000 is a thread
>
>
> Stack: [0x00007f2916733000,0x00007f2916834000],  sp=0x00007f29168323d0,
>  free space=1020k
> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
> code)
> C  [libzip.so+0xb2c2]  inflateEnd+0x32
> C  [libzip.so+0x338e]  Java_java_util_zip_Inflater_end+0x1e
> j  java.util.zip.Inflater.end(J)V+0
> j  java.util.zip.Inflater.end()V+29
> j  java.util.zip.ZipFile.close()V+169
> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
> v  ~StubRoutines::call_stub
> V  [libjvm.so+0x690c66]  JavaCalls::call_helper(JavaValue*, methodHandle*,
> JavaCallArguments*, Thread*)+0x1056
> V  [libjvm.so+0x691171]  JavaCalls::call_virtual(JavaValue*, KlassHandle,
> Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321
> V  [libjvm.so+0x691617]  JavaCalls::call_virtual(JavaValue*, Handle,
> KlassHandle, Symbol*, Symbol*, Thread*)+0x47
> V  [libjvm.so+0x72c990]  thread_entry(JavaThread*, Thread*)+0xa0
> V  [libjvm.so+0xa755f3]  JavaThread::thread_main_inner()+0x103
> V  [libjvm.so+0xa7573c]  JavaThread::run()+0x11c
> V  [libjvm.so+0x926138]  java_start(Thread*)+0x108
> C  [libpthread.so.0+0x7e25]  start_thread+0xc5
>
> Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
> j  java.util.zip.Inflater.end(J)V+0
> j  java.util.zip.Inflater.end()V+29
> j  java.util.zip.ZipFile.close()V+169
> j  sun.net.www.protocol.jar.URLJarFile.close()V+18
> j  sun.net.www.protocol.jar.URLJarFile.finalize()V+1
> J 9535% C2 java.lang.ref.Finalizer$FinalizerThread.run()V (55 bytes) @
> 0x00007f293674cec0 [0x00007f293674cc00+0x2c0]
> v  ~StubRoutines::call_stub
>