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

tarun razdan edited comment on FLINK-7756 at 2/21/18 10:21 AM:
---------------------------------------------------------------

[~aljoscha]

I tried building your branch and the build was successful. Flink distribution 
on YARN was a success. Submitted the distribution using the below command.
{code:java}
yarn-session.sh -n 2 -s 4 -nm 'Flink1.4.1' -d -tm 10000 -st
{code}

In the flink UI, there were no task managers available, though the job was 
submitted succesfully. 

Now when I tried to submit my flink application using the command,
{code:java}
flink run -yst -p 1 -d  -m 
ip-000-00-00-36.ap-southeast-1.compute.internal:43713 
/home/flink/<path>/target/scala-2.11/Appname-assembly-0.2.jar 
{code}
Following error occurred:
{code:java}
org.apache.flink.client.program.ProgramInvocationException: Could not submit 
job 4805abcc5b394afa5d2f05a615204fd1.
        at 
org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:207)
        at 
org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:451)
        at 
org.apache.flink.client.program.DetachedEnvironment.finalizeExecute(DetachedEnvironment.java:77)
        at 
org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:397)
        at 
org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:729)
        at 
org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:275)
        at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:210)
        at 
org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:968)
        at 
org.apache.flink.client.cli.CliFrontend.lambda$main$8(CliFrontend.java:1041)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1556)
        at 
org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
        at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1041)
Caused by: org.apache.flink.runtime.client.JobSubmissionException: Failed to 
submit JobGraph.
        at 
org.apache.flink.client.program.rest.RestClusterClient.lambda$submitJob$4(RestClusterClient.java:312)
        at 
java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
        at 
java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
        at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
        at 
org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$1(RestClient.java:181)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:680)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:603)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:563)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:424)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.fulfillConnectPromise(AbstractNioChannel.java:268)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:284)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:528)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.util.concurrent.CompletionException: java.net.ConnectException: 
Connection refused: localhost/127.0.0.1:9065
        at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
        at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
        at 
java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943)
        at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926)
        ... 16 more
Caused by: java.net.ConnectException: Connection refused: 
localhost/127.0.0.1:9065
        at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
        at 
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:224)
        at 
org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:281)
        ... 7 more
{code}



was (Author: trazdan):
[~aljoscha]

I tried building your branch and the build was successful. Now when i run the 
flink distribution on YARN, I could not find any task manager running. I tried 
to run with debug log on, but there were no errors.

Can you suggest some ways to debug this?

> RocksDB state backend Checkpointing (Async and Incremental)  is not working 
> with CEP.
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-7756
>                 URL: https://issues.apache.org/jira/browse/FLINK-7756
>             Project: Flink
>          Issue Type: Sub-task
>          Components: CEP, State Backends, Checkpointing, Streaming
>    Affects Versions: 1.4.0, 1.3.2
>         Environment: Flink 1.3.2, Yarn, HDFS, RocksDB backend
>            Reporter: Shashank Agarwal
>            Assignee: Aljoscha Krettek
>            Priority: Blocker
>             Fix For: 1.5.0, 1.4.1
>
>         Attachments: jobmanager.log, jobmanager_without_cassandra.log, 
> taskmanager.log, taskmanager_without_cassandra.log
>
>
> When i try to use RocksDBStateBackend on my staging cluster (which is using 
> HDFS as file system) it crashes. But When i use FsStateBackend on staging 
> (which is using HDFS as file system) it is working fine.
> On local with local file system it's working fine in both cases.
> Please check attached logs. I have around 20-25 tasks in my app.
> {code:java}
> 2017-09-29 14:21:31,639 INFO  
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink  - No state 
> to restore for the BucketingSink (taskIdx=0).
> 2017-09-29 14:21:31,640 INFO  
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend  - 
> Initializing RocksDB keyed state backend from snapshot.
> 2017-09-29 14:21:32,020 INFO  
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink  - No state 
> to restore for the BucketingSink (taskIdx=1).
> 2017-09-29 14:21:32,022 INFO  
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend  - 
> Initializing RocksDB keyed state backend from snapshot.
> 2017-09-29 14:21:32,078 INFO  com.datastax.driver.core.NettyUtil              
>               - Found Netty's native epoll transport in the classpath, using 
> it
> 2017-09-29 14:21:34,177 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to fail task externally Co-Flat Map (1/2) 
> (b879f192c4e8aae6671cdafb3a24c00a).
> 2017-09-29 14:21:34,177 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to fail task externally Map (2/2) 
> (1ea5aef6ccc7031edc6b37da2912d90b).
> 2017-09-29 14:21:34,178 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to fail task externally Co-Flat Map (2/2) 
> (4bac8e764c67520d418a4c755be23d4d).
> 2017-09-29 14:21:34,178 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Co-Flat Map (1/2) (b879f192c4e8aae6671cdafb3a24c00a) switched 
> from RUNNING to FAILED.
> AsynchronousException{java.lang.Exception: Could not materialize checkpoint 2 
> for operator Co-Flat Map (1/2).}
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:970)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.Exception: Could not materialize checkpoint 2 for 
> operator Co-Flat Map (1/2).
>       ... 6 more
> Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException
>       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>       ... 5 more
>       Suppressed: java.lang.Exception: Could not properly cancel managed 
> keyed state future.
>               at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:90)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:1023)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:961)
>               ... 5 more
>       Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException
>               at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>               at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>               at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>               at 
> org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:85)
>               at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:88)
>               ... 7 more
>       Caused by: java.lang.IllegalStateException
>               at 
> org.apache.flink.util.Preconditions.checkState(Preconditions.java:179)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeSnapshot(RocksDBKeyedStateBackend.java:878)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:353)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:350)
>               at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>               at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>               ... 5 more
>       [CIRCULAR REFERENCE:java.lang.IllegalStateException]
> 2017-09-29 14:21:34,177 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to fail task externally Map (1/2) 
> (a06925261e74b4efdf50a30089e2b778).
> 2017-09-29 14:21:34,177 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to fail task externally Map (1/2) 
> (1747902c96e63fefd977ac4d4a01d2fa).
> 2017-09-29 14:21:34,180 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Map (1/2) (a06925261e74b4efdf50a30089e2b778) switched from 
> RUNNING to FAILED.
> AsynchronousException{java.lang.Exception: Could not materialize checkpoint 2 
> for operator Map (1/2).}
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:970)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.Exception: Could not materialize checkpoint 2 for 
> operator Map (1/2).
>       ... 6 more
> Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException
>       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>       ... 5 more
>       Suppressed: java.lang.Exception: Could not properly cancel managed 
> keyed state future.
>               at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:90)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:1023)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:961)
>               ... 5 more
>       Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException
>               at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>               at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>               at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>               at 
> org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:85)
>               at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:88)
>               ... 7 more
>       Caused by: java.lang.IllegalStateException
>               at 
> org.apache.flink.util.Preconditions.checkState(Preconditions.java:179)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeSnapshot(RocksDBKeyedStateBackend.java:878)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:353)
>               at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:350)
>               at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>               at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
>               at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>               ... 5 more
>       [CIRCULAR REFERENCE:java.lang.IllegalStateException]
> {code}
> That same printed for around 12-13 tasks. Than following logs printed :
> {code:java}
> 2017-09-29 14:21:35,039 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Ensuring all FileSystem streams are closed for task Source: 
> Custom Source (2/2) (77c896e2a2063e98f399244cae21c260) [CANCELED]
> 2017-09-29 14:21:35,041 WARN  org.apache.hadoop.ipc.Client                    
>               - interrupted waiting to send rpc request to server
> java.lang.InterruptedException
>       at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:404)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:191)
>       at 
> org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1059)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1454)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1412)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
>       at com.sun.proxy.$Proxy12.delete(Unknown Source)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.delete(ClientNamenodeProtocolTranslatorPB.java:540)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:497)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>       at com.sun.proxy.$Proxy13.delete(Unknown Source)
>       at org.apache.hadoop.hdfs.DFSClient.delete(DFSClient.java:2044)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:707)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:703)
>       at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:714)
>       at 
> org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.delete(HadoopFileSystem.java:435)
>       at 
> org.apache.flink.core.fs.SafetyNetWrapperFileSystem.delete(SafetyNetWrapperFileSystem.java:106)
>       at 
> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.closeAndGetHandle(FsCheckpointStreamFactory.java:324)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeMetaData(RocksDBKeyedStateBackend.java:826)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeSnapshot(RocksDBKeyedStateBackend.java:875)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:353)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:350)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 2017-09-29 14:21:35,042 WARN  
> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory  - Could 
> not delete the checkpoint stream file 
> hdfs://static.175.87.9.5.clients.your-server.de:8020/flink/flink-checkpoints/rocksDB/events/e10dbe09aa2ecccb22737ddce8b4dc9f/chk-2/a28796de-978a-4f1a-8ff5-5f5c654b0ffc.
> java.io.IOException: java.lang.InterruptedException
>       at org.apache.hadoop.ipc.Client.call(Client.java:1460)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1412)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
>       at com.sun.proxy.$Proxy12.delete(Unknown Source)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.delete(ClientNamenodeProtocolTranslatorPB.java:540)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:497)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>       at com.sun.proxy.$Proxy13.delete(Unknown Source)
>       at org.apache.hadoop.hdfs.DFSClient.delete(DFSClient.java:2044)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:707)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:703)
>       at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:714)
>       at 
> org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.delete(HadoopFileSystem.java:435)
>       at 
> org.apache.flink.core.fs.SafetyNetWrapperFileSystem.delete(SafetyNetWrapperFileSystem.java:106)
>       at 
> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.closeAndGetHandle(FsCheckpointStreamFactory.java:324)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeMetaData(RocksDBKeyedStateBackend.java:826)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeSnapshot(RocksDBKeyedStateBackend.java:875)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:353)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:350)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.InterruptedException
>       at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:404)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:191)
>       at 
> org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1059)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1454)
>       ... 31 more
> 2017-09-29 14:21:35,054 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Attempting to cancel task KeyedCEPPatternOperator -> Flat Map 
> -> (Flat Map, Flat Map) (1/2) (8c6eff62d47c4a624a7554065bac36ee).
> 2017-09-29 14:21:35,055 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - KeyedCEPPatternOperator -> Flat Map -> (Flat Map, Flat Map) 
> (1/2) (8c6eff62d47c4a624a7554065bac36ee) switched from RUNNING to CANCELING.
> {code}
> Than same printed for 12-13 tasks.



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

Reply via email to