Hi Theo!

I have not seen this error before however I have encountered many strange
things when using Kryo for serialization. From the stack trace it seems
that this might indeed be a Kryo related issue.

I am not sure what it is but what I would try is to change the state
serializers to a non Kryo variants.

When you create the state you can specify the TypeInformation object for
the state instead of the class. You have 2 states:

For LinkedList:
Use the ListTypeInfo class and change your state type to List

For HashSet:
There is no Set Type info built into Flink but you can replace it with a
Map and use the MapTypeInfo to try it out, before implementing a custom
type info

Also make sure that your MyPOJO type is a nice flink supported pojo. All
public non-final fields + empty constructor.

Let me know if this makes sense :)

Cheers,
Gyula

On Wed, Nov 27, 2019 at 2:18 PM theo.diefent...@scoop-software.de <
theo.diefent...@scoop-software.de> wrote:

> Sorry, I forgot to mention the environment.
> We use Flink 1.9.1 on a cloudera cdh6. 3.1 cluster (with Hadoop 3.0.0 but
> using Flink shaded 2.8.3-7. Might this be a problem? As it seems to arise
> from kryo, I doubt it)
> Our flink is configured as default. Our job uses FsStateBackend and
> exactly once processing with Kafka source and sink.
> Best regardsTheo
> -------- Ursprüngliche Nachricht --------
> Betreff: ArrayIndexOutOfBoundException on checkpoint creation
> Von: Theo Diefenthal
> An: user
> Cc:
>
>
> Hi,
>
> We have a pipeline with a custom ProcessFunction and state (see [1],
> implemented as suggested by Fabian with a ValueState<LinkedList<MyPOJO> and
> ValueState<HashSet<String>>)
> The behavior of that function works fine in our unittests and with low
> load in our test environment (100.000 records per minute). On the
> production environment, we observe reproduceable crashes like the attached
> one.
> Any idea on why this out of bound could be caused? Every time we read the
> state and modify it, we are certain that an .update() was called:
>
> 2019-11-26T11:26:55+01:00 host19 java.lang.Exception: Could not
> materialize checkpoint 7 for operator our_operator) (4/8).
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.handleExecutionException(StreamTask.java:1100)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:1042)
>
> 2019-11-26T11:26:55+01:00 host19     at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>
> 2019-11-26T11:26:55+01:00 host19     at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>
> 2019-11-26T11:26:55+01:00 host19     at
> java.lang.Thread.run(Thread.java:745)
> 2019-11-26T11:26:55+01:00 host19 Caused by:
> java.util.concurrent.ExecutionException:
> java.lang.ArrayIndexOutOfBoundsException: 67108864
> 2019-11-26T11:26:55+01:00 host19     at
> java.util.concurrent.FutureTask.report(FutureTask.java:122)
> 2019-11-26T11:26:55+01:00 host19     at
> java.util.concurrent.FutureTask.get(FutureTask.java:192)
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:450)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:47)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:1011)
>
> 2019-11-26T11:26:55+01:00 host19     ... 3 more
> 2019-11-26T11:26:55+01:00 host19 Caused by:
> java.lang.ArrayIndexOutOfBoundsException: 67108864
> 2019-11-26T11:26:55+01:00 host19     at
> com.esotericsoftware.kryo.util.IdentityObjectIntMap.clear(IdentityObjectIntMap.java:364)
>
> 2019-11-26T11:26:55+01:00 host19     at
> com.esotericsoftware.kryo.util.MapReferenceResolver.reset(MapReferenceResolver.java:47)
>
> 2019-11-26T11:26:55+01:00 host19     at
> com.esotericsoftware.kryo.Kryo.reset(Kryo.java:836)
> 2019-11-26T11:26:55+01:00 host19     at
> com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:601)
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.serialize(KryoSerializer.java:305)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.heap.CopyOnWriteStateMapSnapshot.writeState(CopyOnWriteStateMapSnapshot.java:116)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.heap.AbstractStateTableSnapshot.writeStateInKeyGroup(AbstractStateTableSnapshot.java:121)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.heap.CopyOnWriteStateTableSnapshot.writeStateInKeyGroup(CopyOnWriteStateTableSnapshot.java:37)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.heap.HeapSnapshotStrategy$1.callInternal(HeapSnapshotStrategy.java:191)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.heap.HeapSnapshotStrategy$1.callInternal(HeapSnapshotStrategy.java:158)
>
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.state.AsyncSnapshotCallable.call(AsyncSnapshotCallable.java:75)
>
> 2019-11-26T11:26:55+01:00 host19     at
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 2019-11-26T11:26:55+01:00 host19     at
> org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:447)
>
> 2019-11-26T11:26:55+01:00 host19     ... 5 more
> 2019-11-26T11:26:55+01:00 host18 WARN
> org.apache.hadoop.hdfs.DataStreamer                           -
> DataStreamer Exception
> 2019-11-26T11:26:55+01:00 host18 java.io.FileNotFoundException: File does
> not exist:
> /.../STATE/CHECKPOINTS/0a2e111b3a800aae0d3b49f33e0db6f3/chk-7/3da2a0a4-f5ef-4e8c-bc1a-9fe892cb0b18
> (inode 577546140) Holder DFSClient_NONMAPREDUCE_-1714419242_95 does not
> have any open files.
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2782)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.analyzeFileState(FSDirWriteFileOp.java:599)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.validateAddBlock(FSDirWriteFileOp.java:171)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2661)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:872)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:550)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815)
> 2019-11-26T11:26:55+01:00 host18     at
> java.security.AccessController.doPrivileged(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> javax.security.auth.Subject.doAs(Subject.java:422)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1875)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675)
> 2019-11-26T11:26:55+01:00 host18
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>
> 2019-11-26T11:26:55+01:00 host18     at
> java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:121)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:88)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.locateFollowingBlock(DataStreamer.java:1842)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1638)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:704)
> 2019-11-26T11:26:55+01:00 host18 Caused by:
> org.apache.hadoop.ipc.RemoteException(java.io.FileNotFoundException):
> File does not exist:
> /.../STATE/INGESTOR/CHECKPOINTS/0a2e111b3a800aae0d3b49f33e0db6f3/chk-7/3da2a0a4-f5ef-4e8c-bc1a-9fe892cb0b18
> (inode 577546140) Holder DFSClient_NONMAPREDUCE_-1714419242_95 does not
> have any open files.
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2782)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.analyzeFileState(FSDirWriteFileOp.java:599)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.validateAddBlock(FSDirWriteFileOp.java:171)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2661)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:872)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:550)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815)
> 2019-11-26T11:26:55+01:00 host18     at
> java.security.AccessController.doPrivileged(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> javax.security.auth.Subject.doAs(Subject.java:422)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1875)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675)
> 2019-11-26T11:26:55+01:00 host18
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1489)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.call(Client.java:1435)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.call(Client.java:1345)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:227)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116)
>
> 2019-11-26T11:26:55+01:00 host18     at
> com.sun.proxy.$Proxy19.addBlock(Unknown Source)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:444)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>
> 2019-11-26T11:26:55+01:00 host18     at
> java.lang.reflect.Method.invoke(Method.java:498)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:409)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:163)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:155)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:346)
>
> 2019-11-26T11:26:55+01:00 host18     at
> com.sun.proxy.$Proxy20.addBlock(Unknown Source)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.locateFollowingBlock(DataStreamer.java:1838)
>
> 2019-11-26T11:26:55+01:00 host18     ... 2 more
> 2019-11-26T11:26:55+01:00 host18 WARN
> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory  -
> Could not close the state stream for
> hdfs:/.../STATE/INGESTOR/CHECKPOINTS/0a2e111b3a800aae0d3b49f33e0db6f3/chk-7/3da2a0a4-f5ef-4e8c-bc1a-9fe892cb0b18.
>
> 2019-11-26T11:26:55+01:00 host18 java.io.FileNotFoundException: File does
> not exist:
> /.../STATE/INGESTOR/CHECKPOINTS/0a2e111b3a800aae0d3b49f33e0db6f3/chk-7/3da2a0a4-f5ef-4e8c-bc1a-9fe892cb0b18
> (inode 577546140) Holder DFSClient_NONMAPREDUCE_-1714419242_95 does not
> have any open files.
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2782)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.analyzeFileState(FSDirWriteFileOp.java:599)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.validateAddBlock(FSDirWriteFileOp.java:171)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2661)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:872)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:550)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815)
> 2019-11-26T11:26:55+01:00 host18     at
> java.security.AccessController.doPrivileged(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> javax.security.auth.Subject.doAs(Subject.java:422)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1875)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675)
> 2019-11-26T11:26:55+01:00 host18
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>
>
> 2019-11-26T11:26:55+01:00 host18     at
> java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:121)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:88)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.locateFollowingBlock(DataStreamer.java:1842)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1638)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:704)
> 2019-11-26T11:26:55+01:00 host18 Caused by:
> org.apache.hadoop.ipc.RemoteException(java.io.FileNotFoundException):
> File does not exist:
> /.../STATE/INGESTOR/CHECKPOINTS/0a2e111b3a800aae0d3b49f33e0db6f3/chk-7/3da2a0a4-f5ef-4e8c-bc1a-9fe892cb0b18
> (inode 577546140) Holder DFSClient_NONMAPREDUCE_-1714419242_95 does not
> have any open files.
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2782)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.analyzeFileState(FSDirWriteFileOp.java:599)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.validateAddBlock(FSDirWriteFileOp.java:171)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2661)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:872)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:550)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815)
> 2019-11-26T11:26:55+01:00 host18     at
> java.security.AccessController.doPrivileged(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> javax.security.auth.Subject.doAs(Subject.java:422)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1875)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675)
> 2019-11-26T11:26:55+01:00 host18
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1489)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.call(Client.java:1435)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.Client.call(Client.java:1345)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:227)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116)
>
> 2019-11-26T11:26:55+01:00 host18     at
> com.sun.proxy.$Proxy19.addBlock(Unknown Source)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:444)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>
> 2019-11-26T11:26:55+01:00 host18     at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>
> 2019-11-26T11:26:55+01:00 host18     at
> java.lang.reflect.Method.invoke(Method.java:498)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:409)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:163)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:155)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
>
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:346)
>
> 2019-11-26T11:26:55+01:00 host18     at
> com.sun.proxy.$Proxy20.addBlock(Unknown Source)
> 2019-11-26T11:26:55+01:00 host18     at
> org.apache.hadoop.hdfs.DataStreamer.locateFollowingBlock(DataStreamer.java:1838)
>
> 2019-11-26T11:26:55+01:00 host18     ... 2 more
> 2019-11-26T11:29:50+01:00 host19 INFO
> org.apache.flink.yarn.YarnResourceManager                     - The
> heartbeat of TaskManager with id
> container_e137_1574681304655_2382_01_000007 timed out.
>
> Best regards
> Theo
>
> [1]
> http://mail-archives.apache.org/mod_mbox/flink-user/201909.mbox/%3c1340607597.13576306.1568278936983.javamail.zim...@scoop-software.de%3E
>

Reply via email to