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

Elias Levy commented on FLINK-9450:
-----------------------------------

The logs show no errors.  Alas, we are using the JRE, so these nodes don't have 
jps or jstack installed. 

Re: job failure on checkpoint failure.  My understanding is that is a 1.5 
feature (env.getCheckpointConfig.setFailTasksOnCheckpointingErrors).  We are 
running 1.4.2.

The checkpoint and the job just hang.  There is no failure, at least for the 
amount of time I've waited, which has been several minutes.  I don't know if it 
will fail if I wait longer.  The job continues after I clear the firewall rules.

Looks like the checkpoint may be hanging on the synchronous portion of the 
async checkpoint.

This is from a Flink job running on a cluster with a single TM:

I run:

{noformat}
iptables -A OUTPUT -p tcp --dport 80 -j REJECT  --reject-with tcp-reset
iptables -A OUTPUT -p tcp --dport 443 -j REJECT  --reject-with tcp-reset
{noformat}

The logs show what seems like a normal checkpoint attempt:

{noformat}
2018-05-28 16:44:06.449070500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Updates 
(1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:44:06.449103500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Updates 
(1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:44:06.449388500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Queries Topic -> Filter 
(1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:44:06.449390500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:44:06.449400500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:44:06.451695500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Engine (1/1),5,Flink Task Threads] took 2 ms
{noformat}

 After I disable the firewall rules:
{noformat}
iptables -D OUTPUT -p tcp --dport 80 -j REJECT --reject-with tcp-reset
iptables -D OUTPUT -p tcp --dport 443 -j REJECT  --reject-with tcp-reset
{noformat}

{noformat}
2018-05-28 16:47:19.741581500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-58-thread-1,5,Flink Task Threads] took 
193290 ms.
2018-05-28 16:47:23.424434500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.424876500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Queries Topic -> Agent 
Queries Filter (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426263500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Queries Topic -> Agent 
Queries Filter (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426265500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Queries Topic -> Agent 
Queries Filter (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426281500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426288500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426291500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426295500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426301500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.426304500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Async calls on Source: Kafka Control Topic -> Fetch 
Updates (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.428811500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Engine (1/1),5,Flink Task Threads] took 2 ms.
2018-05-28 16:47:23.430908500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Engine (1/1),5,Flink Task Threads] took 2 ms.
2018-05-28 16:47:23.433059500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Engine (1/1),5,Flink Task Threads] took 2 ms.
2018-05-28 16:47:23.478915500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
54 ms.
2018-05-28 16:47:23.507251500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-2,5,Flink Task Threads] took 
81 ms.
2018-05-28 16:47:23.524032500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-4,5,Flink Task Threads] took 
97 ms.
2018-05-28 16:47:23.526583500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-60-thread-2,5,Flink Task Threads] took 
101 ms.
2018-05-28 16:47:23.544790500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
66 ms.
2018-05-28 16:47:23.550983500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-60-thread-4,5,Flink Task Threads] took 
125 ms.
2018-05-28 16:47:23.559251500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-3,5,Flink Task Threads] took 
133 ms.
2018-05-28 16:47:23.576058500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-2,5,Flink Task Threads] took 
69 ms.
2018-05-28 16:47:23.584631500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-4,5,Flink Task Threads] took 
60 ms.
2018-05-28 16:47:23.596741500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-60-thread-3,5,Flink Task Threads] took 
171 ms.
2018-05-28 16:47:23.608761500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.608793500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.608972500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-3,5,Flink Task Threads] took 
49 ms.
2018-05-28 16:47:23.622091500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
77 ms.
2018-05-28 16:47:23.638608500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-58-thread-3,5,Flink Task Threads] took 
205 ms.
2018-05-28 16:47:23.737491500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-58-thread-2,5,Flink Task Threads] took 
306 ms.
2018-05-28 16:47:23.905074500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 1 ms.
2018-05-28 16:47:23.905226500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.905255500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:23.926104500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-58-thread-1,5,Flink Task Threads] took 
497 ms.
2018-05-28 16:47:23.956786500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
51 ms.
2018-05-28 16:47:24.006804500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
50 ms.
2018-05-28 16:47:24.055511500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
49 ms.
2018-05-28 16:47:24.196191500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:24.196327500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:24.196353500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:24.244838500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
48 ms.
2018-05-28 16:47:24.298130500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
54 ms.
2018-05-28 16:47:24.301823500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, synchronous 
part) in thread Thread[Suppress Events -> Assign Timestamp -> (Sink: Kafka 
Event Topic, Sink: Kafka Event Topic) (1/1),5,Flink Task Threads] took 0 ms.
2018-05-28 16:47:24.352434500 INFO  
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend     - Heap backend 
snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-3,5,Flink Task Threads] took 
50 ms.
2018-05-28 16:47:24.371525500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-2,5,Flink Task Threads] took 
73 ms.
2018-05-28 16:47:24.405921500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-3,5,Flink Task Threads] took 
53 ms.
2018-05-28 16:47:24.621455500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-59-thread-3,5,Flink Task Threads] took 
216 ms.
2018-05-28 16:47:24.743632500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527525786450} from checkpoint 83
2018-05-28 16:47:24.743663500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527525846449} from checkpoint 84
2018-05-28 16:47:24.743665500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527526043608} from checkpoint 85
2018-05-28 16:47:24.743668500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527526043905} from checkpoint 86
2018-05-28 16:47:24.743675500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527525786450} from checkpoint 83
2018-05-28 16:47:24.743705500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527525846449} from checkpoint 84
2018-05-28 16:47:24.743706500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527526043608} from checkpoint 85
2018-05-28 16:47:24.743713500 INFO  
org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction  - 
FlinkKafkaProducer011 0/1 - checkpoint 86 complete, committing transaction 
TransactionHolder{handle=KafkaTransactionState [transactionalId=null, 
producerId=-1, epoch=-1], transactionStartTime=1527526043905} from checkpoint 86
2018-05-28 16:47:28.143548500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-60-thread-1,5,Flink Task Threads] took 
201694 ms.
2018-05-28 16:47:29.398640500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-1,5,Flink Task Threads] took 
202949 ms.
2018-05-28 16:47:29.451288500 INFO  
org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
DefaultOperatorStateBackend snapshot (File Stream Factory @ 
s3://bucket/flink/job/checkpoints/03a1057d5d217777dff9745eebd954c6, 
asynchronous part) in thread Thread[pool-61-thread-1,5,Flink Task Threads] took 
53 ms.
{noformat}



 

> Job hangs if S3 access it denied during checkpoints
> ---------------------------------------------------
>
>                 Key: FLINK-9450
>                 URL: https://issues.apache.org/jira/browse/FLINK-9450
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.4.2
>            Reporter: Elias Levy
>            Priority: Major
>
> We have a streaming job that consumes from and writes to Kafka.  The job is 
> configured to checkpoint to S3.  If we deny access to S3 by using iptables on 
> the TM host to deny all outgoing connections to ports 80 and 443, whether 
> using DROP or REJECT, and whether using REJECT with -reject-with tcp-reset or 
> -r reject-with imp-port-unreachable, the job soon stops publishing to Kafka.
> This happens whether or not the Kafka sources have 
> {{setCommitOffsetsOnCheckpoints}} set to true or false.
> The system is configured to use Presto for the S3 file system.  The job has a 
> small amount of state, so it is configured to use {{FsStateBackend}} with 
> asynchronous snapshots.
> If the ip tables rules are removed, the job continues the function.
> I would expect the job to either fail or continue running if a checkpoint 
> fails.



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

Reply via email to