[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174491828
  
Good fix.

I think if we want to be on the super safe side, actually all tasks should 
always report back to the JobManager that they received the "TriggerCheckpoint" 
message (either Ack or Decline).

The CheckpointCoordinator would then "ask" the TaskManagers and would 
cancel the checkpoint if some of the asks time out (3-5 secs or so). That way, 
lost messages are properly accounted for.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread aljoscha
Github user aljoscha commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174505573
  
@senorcarbone Yes, it works correctly with the timeouts, the job can just 
hang for a long time, especially in tests this occurred several times.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174521106
  
The default timeout is 10 minutes. That is too long time to recognize that 
a checkpoint was started pre-maturely ;-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread senorcarbone
Github user senorcarbone commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174500162
  
Looks cool. 
Just so I understand exactly, what is wrong again if the Coordinator simply 
aborts expired checkpoint attempts? Wouldn't the protocol be the same, with 
less messages? If a task is not ready it can simply discard the checkpoint 
request which will eventually time out at the Coordinator. The Coordinator 
attempts might potentially keep timing out but there will be a complete 
snapshot eventually when all tasks are ready. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174523012
  
Normally (except in this kind of situation), checkpoints progress and need 
no aggressive timeout as a guard. On the other hand, full checkpoints with huge 
state (for example when re-running a streaming job on historic data) may once 
in a while take long (when the network is congested or so) and we did not want 
to overaggressively kill the pending checkpoints in that case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/1537#discussion_r50669903
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 ---
@@ -505,6 +506,88 @@ else if (pendingCheckpoints.size() >= 
maxConcurrentCheckpointAttempts) {
}
 
/**
+* Receives a {@link DeclineCheckpoint} message and returns whether the
+* message was associated with a pending checkpoint.
+*
+* @param message Checkpoint decline from the task manager
+*
+* @return Flag indicating whether the declined checkpoint was 
associated
+* with a pending checkpoint.
+*/
+
+   public boolean receiveDeclineMessage(DeclineCheckpoint message) throws 
Exception {
+   if (shutdown || message == null) {
+   return false;
+   }
+   if (!job.equals(message.getJob())) {
+   LOG.error("Received DeclineCheckpoint message for wrong 
job: {}", message);
+   return false;
+   }
+
+   final long checkpointId = message.getCheckpointId();
+
+   CompletedCheckpoint completed = null;
+   PendingCheckpoint checkpoint;
+
+   // Flag indicating whether the ack message was for a known 
pending
+   // checkpoint.
+   boolean isPendingCheckpoint;
+
+   synchronized (lock) {
+   // we need to check inside the lock for being shutdown 
as well, otherwise we
+   // get races and invalid error log messages
+   if (shutdown) {
+   return false;
+   }
+
+   checkpoint = pendingCheckpoints.get(checkpointId);
+
+   if (checkpoint != null && !checkpoint.isDiscarded()) {
+   isPendingCheckpoint = true;
+
+   LOG.info("Discarding checkpoint " + checkpointId
+   + " because of checkpoint decline from 
task " + message.getTaskExecutionId());
+
+   pendingCheckpoints.remove(checkpointId);
+   checkpoint.discard(userClassLoader);
+   rememberRecentCheckpointId(checkpointId);
+
+   boolean haveMoreRecentPending = false;
+   Iterator> 
entries = pendingCheckpoints.entrySet().iterator();
+   while (entries.hasNext()) {
+   PendingCheckpoint p = 
entries.next().getValue();
+   if (!p.isDiscarded() && 
p.getCheckpointTimestamp() >= checkpoint.getCheckpointTimestamp()) {
+   haveMoreRecentPending = true;
+   break;
+   }
+   }
+   if (!haveMoreRecentPending && 
!triggerRequestQueued) {
+   LOG.info("Triggering new checkpoint 
because of discarded checkpoint " + checkpointId);
+   
triggerCheckpoint(System.currentTimeMillis());
+   } else if (!haveMoreRecentPending) {
+   LOG.info("Promoting queued checkpoint 
request because of discarded checkpoint " + checkpointId);
+   triggerQueuedRequests();
+   }
+
--- End diff --

empty line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/1537#discussion_r50670074
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.messages.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+/**
+ * This message is sent from the {@link 
org.apache.flink.runtime.taskmanager.TaskManager} to the
+ * {@link org.apache.flink.runtime.jobmanager.JobManager} to tell the 
checkpoint coordinator
+ * that a checkpoint request could not be heeded. This can happen if a 
Task is already in
--- End diff --

I had to look up `heeded` :stuck_out_tongue_winking_eye: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread aljoscha
Github user aljoscha commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174448455
  
Ok, I'm addressing the comments about code-style and merging.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread gyfora
Github user gyfora commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174438711
  
Looks good to me!
This is a critical fix :+1: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread rmetzger
Github user rmetzger commented on the pull request:

https://github.com/apache/flink/pull/1537#issuecomment-174446152
  
I tested this as well. A Kafka test was previously failing, now its passing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/1537#discussion_r50670120
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java ---
@@ -854,7 +855,8 @@ private void notifyObservers(ExecutionState newState, 
Throwable error) {
// 

 
/**
-* Calls the invokable to trigger a checkpoint, if the invokable 
implements the interface
+* Calls the invokable to trigger a checkpoint, i
+* f the invokable implements the interface
--- End diff --

line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-25 Thread aljoscha
Github user aljoscha closed the pull request at:

https://github.com/apache/flink/pull/1537


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-3261] Allow Task to decline checkpoint ...

2016-01-21 Thread aljoscha
GitHub user aljoscha opened a pull request:

https://github.com/apache/flink/pull/1537

[FLINK-3261] Allow Task to decline checkpoint request if not ready

Before, it could happen that a StreamingTask receives a Checkpoint
Trigger message while internally not being ready. The checkpoint
coordinator would then wait the specified timeout interval before
continuing. Now, tasks can signal that they are not ready and the
checkpoint coordinator will dicard a checkpoint for which is this the
case and trigger new checkpoints if necessary.

The newly triggered checkpoints will also release alignment locks in
streaming tasks that are still waiting for barriers from failed
checkpoints.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/aljoscha/flink checkpoint-coordinator-decline

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/1537.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1537


commit c759e2a0a2a1310467c25d84912544abaf5ab29e
Author: Aljoscha Krettek 
Date:   2016-01-21T16:21:09Z

[FLINK-3261] Allow Task to decline checkpoint request if not ready

Before, it could happen that a StreamingTask receives a Checkpoint
Trigger message while internally not being ready. The checkpoint
coordinator would then wait the specified timeout interval before
continuing. Now, tasks can signal that they are not ready and the
checkpoint coordinator will dicard a checkpoint for which is this the
case and trigger new checkpoints if necessary.

The newly triggered checkpoints will also release alignment locks in
streaming tasks that are still waiting for barriers from failed
checkpoints.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---