Re: Stream Task seems to be blocked after checkpoint timeout

2017-10-03 Thread Stefan Richter
Sure, I opened Jira FLINK-7757 and this PR: https://github.com/apache/flink/pull/4764 . Best, Stefan > Am 03.10.2017 um 10:25 schrieb Tony Wei : > > Hi Stefan, > > Thank you very much. I will try to investigate what's the problem on my > cluster an

Re: Stream Task seems to be blocked after checkpoint timeout

2017-10-03 Thread Tony Wei
Hi Stefan, Thank you very much. I will try to investigate what's the problem on my cluster and S3. BTW, Is there any Jira issue associated with your improvement, so that I can track it? Best Regards, Tony Wei 2017-10-03 16:01 GMT+08:00 Stefan Richter : > Hi, > > from the stack trace, it seems t

Re: Stream Task seems to be blocked after checkpoint timeout

2017-10-03 Thread Stefan Richter
Hi, from the stack trace, it seems to me like closing the checkpoint output stream to S3 is the culprit: "pool-55-thread-7" #458829 prio=5 os_prio=0 tid=0x7fda180c4000 nid=0x55a2 waiting on condition [0x7fda092d7000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsaf

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Tony Wei
Hi Stefan, That reason makes sense to me. Thanks for point me out. About my job, the database currently was never used, I disabled it for some reasons, but output to s3 was implemented by async io. I used ForkJoinPool with 50 capacity. I have tried to rebalance after count window to monitor the

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Stefan Richter
Hi, the gap between the sync and the async part does not mean too much. What happens per task is that all operators go through their sync part, and then one thread executes all the async parts, one after the other. So if an async part starts late, this is just because it started only after anot

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Tony Wei
Hi, Sorry. This is the correct one. Best Regards, Tony Wei 2017-09-28 18:55 GMT+08:00 Tony Wei : > Hi Stefan, > > Sorry for providing partial information. The attachment is the full logs > for checkpoint #1577. > > Why I would say it seems that asynchronous part was not executed > immediately i

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Tony Wei
Hi Stefan, Sorry for providing partial information. The attachment is the full logs for checkpoint #1577. Why I would say it seems that asynchronous part was not executed immediately is due to all synchronous parts were all finished at 2017-09-27 13:49. Did that mean the checkpoint barrier event

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Stefan Richter
Hi, I agree that the memory consumption looks good. If there is only one TM, it will run inside one JVM. As for the 7 minutes, you mean the reported end-to-end time? This time measurement starts when the checkpoint is triggered on the job manager, the first contributor is then the time that it

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Tony Wei
Hi Stefan, These are some telemetry information, but I don't have history information about gc. [image: 內置圖片 2] [image: 內置圖片 1] 1) Yes, my state is not large. 2) My DFS is S3, but my cluster is out of AWS. It might be a problem. Since this is a POC, we might move to AWS in the future or use HDFS

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-28 Thread Stefan Richter
Hi, when the async part takes that long I would have 3 things to look at: 1) Is your state so large? I don’t think this applies in your case, right? 2) Is something wrong with writing to DFS (network, disks, etc)? 3) Are we running low on memory on that task manager? Do you have telemetry inform

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-27 Thread Stefan Richter
Hi Tony, are your checkpoints typically close to the timeout boundary? From what I see, writing the checkpoint is relatively fast but the time from the checkpoint trigger to execution seems very long. This is typically the case if your job has a lot of backpressure and therefore the checkpoint

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-27 Thread Tony Wei
Hi Stefan, It seems that I found something strange from JM's log. It had happened more than once before, but all subtasks would finish their checkpoint attempts in the end. 2017-09-26 01:23:28,690 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1140 @ 15063

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-27 Thread Stefan Richter
Hi, thanks for the information. Unfortunately, I have no immediate idea what the reason is from the given information. I think most helpful could be a thread dump, but also metrics on the operator operator level to figure out which part of the pipeline is the culprit. Best, Stefan > Am 26.09.

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-26 Thread Tony Wei
Hi Stefan, There is no unknown exception in my full log. The Flink version is 1.3.2. My job is roughly like this. env.addSource(Kafka) .map(ParseKeyFromRecord) .keyBy() .process(CountAndTimeoutWindow) .asyncIO(UploadToS3) .addSink(UpdateDatabase) It seemed all tasks stopped like the pi

Re: Stream Task seems to be blocked after checkpoint timeout

2017-09-26 Thread Stefan Richter
Hi, that is very strange indeed. I had a look at the logs and there is no error or exception reported. I assume there is also no exception in your full logs? Which version of flink are you using and what operators were running in the task that stopped? If this happens again, would it be possibl