[GitHub] flink pull request: [FLINK-2488] Expose Attempt Number in RuntimeC...

2015-12-07 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-07 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162467645
  
Thanks for addressing the comments. 

+1, will merge this...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-07 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162463009
  
Travis passes. Should be good to merge.
This also resolves FLINK-2524, so the commit message should reflect that 
too.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46776269
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java 
---
@@ -1150,8 +1149,8 @@ else if (index < 0 || index >= 
this.driver.getNumberOfDriverComparators()) {
 * @return The string for logging.
 */
public static String constructLogString(String message, String 
taskName, AbstractInvokable parent) {
-   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getIndexInSubtaskGroup() + 1) +
-   '/' + 
parent.getEnvironment().getNumberOfSubtasks() + ')';
+   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getTaskInfo().getIndexOfThisSubtask() + 1) +
--- End diff --

Ah, I see...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162345554
  
@StephanEwen thanks for the review. I have addressed all but one of your 
comments. 


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread sachingoel0101
Github user sachingoel0101 commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46775517
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java 
---
@@ -1150,8 +1149,8 @@ else if (index < 0 || index >= 
this.driver.getNumberOfDriverComparators()) {
 * @return The string for logging.
 */
public static String constructLogString(String message, String 
taskName, AbstractInvokable parent) {
-   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getIndexInSubtaskGroup() + 1) +
-   '/' + 
parent.getEnvironment().getNumberOfSubtasks() + ')';
+   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getTaskInfo().getIndexOfThisSubtask() + 1) +
--- End diff --

That can lead to problems in handling of Chained Tasks. For those, the name 
of chained task, but the index and parallelism of parent is used.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162334713
  
All in all, pretty good change. Makes many parts of the code nicer. Also 
good java docs and comments!

Comments were added inline. Nothing major, few details.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773807
  
--- Diff: 
flink-tests/src/test/java/org/apache/flink/test/misc/AttemptNumberITCase.java 
---
@@ -0,0 +1,70 @@
+/*
--- End diff --

In order to reduce the number of redundant (but time consuming) tests: Can 
you fuse this test with the TaskFailureITCase?


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773693
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java ---
@@ -120,16 +121,9 @@
/** The execution attempt of the parallel subtask */
private final ExecutionAttemptID executionId;
 
-   /** The index of the parallel subtask, in [0, numberOfSubtasks) */
-   private final int subtaskIndex;
+   /** TaskInfo object for this task */
+   private final TaskInfo taskInfo;
 
-   /** The number of parallel subtasks for the 
JobVertex/ExecutionJobVertex that this task belongs to */
-   private final int parallelism;
-
-   /** The name of the task */
-   private final String taskName;
-
-   /** The name of the task, including the subtask index and the 
parallelism */
private final String taskNameWithSubtask;
--- End diff --

This field is also subsumed by the `TaskInfo`


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773689
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java ---
@@ -240,14 +234,13 @@ public Task(TaskDeploymentDescriptor tdd,
checkArgument(tdd.getNumberOfSubtasks() > 0);
checkArgument(tdd.getIndexInSubtaskGroup() >= 0);
checkArgument(tdd.getIndexInSubtaskGroup() < 
tdd.getNumberOfSubtasks());
+   checkArgument(tdd.getAttemptNumber() >= 0);
--- End diff --

You can probably skip these checks, given that `TaskInfo` has these checks 
already...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773676
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java 
---
@@ -1150,8 +1149,8 @@ else if (index < 0 || index >= 
this.driver.getNumberOfDriverComparators()) {
 * @return The string for logging.
 */
public static String constructLogString(String message, String 
taskName, AbstractInvokable parent) {
-   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getIndexInSubtaskGroup() + 1) +
-   '/' + 
parent.getEnvironment().getNumberOfSubtasks() + ')';
+   return message + ":  " + taskName + " (" + 
(parent.getEnvironment().getTaskInfo().getIndexOfThisSubtask() + 1) +
--- End diff --

Can replace the subtask concatenation with 
`getTaskInfo().getTaskNameWithSubtasks()`.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773630
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
 ---
@@ -178,9 +178,9 @@ private boolean decrementReferenceInternal(BatchTask referenceHolder, bool
if (errorIfNoReference) {
throw new IllegalStateException(
String.format("The task 
%s (%d/%d) did not hold a reference to the broadcast variable %s.",
-   
referenceHolder.getEnvironment().getTaskName(),
-   
referenceHolder.getEnvironment().getIndexInSubtaskGroup() + 1,
-   
referenceHolder.getEnvironment().getNumberOfSubtasks(),
+   
referenceHolder.getEnvironment().getTaskInfo().getTaskName(),
--- End diff --

Can replace this with `getTaskInfo().getTaskNameWithSubtasks()`.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773625
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
 ---
@@ -85,9 +85,9 @@ public void materializeVariable(MutableReader reader, 
TypeSerializerFactory

[GitHub] flink pull request: [FLINK-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773363
  
--- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
 ---
@@ -65,6 +65,22 @@
int getIndexOfThisSubtask();
 
/**
+* Gets the attempt number of this parallel subtask. First attempt is 
numbered 0.
--- End diff --

I would add here something like: "The attempt number corresponds to how 
many times the task has been restarted (after a failure or cancellation) since 
the job was initially started."


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request:

https://github.com/apache/flink/pull/1386#discussion_r46773342
  
--- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
@@ -0,0 +1,93 @@
+/*
+ * 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.api.common;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Encapsulates task-specific information: name, index of subtask, 
parallelism and attempt number.
+ */
+public class TaskInfo {
+
+   private final String taskName;
+   private final String taskNameWithSubtasks;
+   private final int indexOfSubtask;
+   private final int numberOfParallelSubtasks;
+   private final int attemptNumber;
+
+   public TaskInfo(String taskName, int indexOfSubtask, int 
numberOfParallelSubtasks, int attemptNumber) {
+   Preconditions.checkArgument(indexOfSubtask >= 0, "Task index 
must be a non-negative number.");
+   Preconditions.checkArgument(numberOfParallelSubtasks >= 1, 
"Parallelism must be a positive number.");
+   Preconditions.checkArgument(indexOfSubtask < 
numberOfParallelSubtasks, "Task index must be less than parallelism.");
+   Preconditions.checkArgument(attemptNumber >= 0, "Attempt number 
must be a non-negative number.");
--- End diff --

Good set of checks. Would be good to add a non-null check for `taskName` as 
well.
Optional: statically importing ´checkArgument` makes it even nicer.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162331160
  
I am adding a number of comments. When you address them, please do not 
squash commits, so we can review just the changes separately.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162330497
  
Checking this out...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-06 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-162325866
  
@StephanEwen can you take a look again? I have introduced the `TaskInfo` 
object in the second commit, and we can squash them before 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-2488] Expose Attempt Number in RuntimeC...

2015-12-04 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-161921756
  
@StephanEwen you're absolutely right. It will most certainly do that.
I will push both things in a single commit and file an additional jira for 
introducing the `TaskInfo` object.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-03 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-161608168
  
As far as I can see from these changes, introducing a `TaskInfo` will 
completely subsume this change here and change every line again.

You are right, usually different issues should go into different pull 
requests. But here, one change is a superset of the other change, so that the 
"task attempt" change without the "task info" change becomes obsolete.

I am not strictly against merging this, just thinking that it makes sense 
to put the TaskInfo into this change as well...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-02 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-161449824
  
Yes. I was planning to start working on that after this. Since it's
preferable to fix different jiras in different commits, it'd be good if I
can base my work after this is committed. What do you think?
On Dec 2, 2015 6:39 PM, "Stephan Ewen"  wrote:

> This generally looks good and pretty straight forward.
> As such it is actually good to merge.
>
> I remember that a while back, we were discussing to create a TaskInfo
> object that would contain "subtaskIndex", "parallelism", "name",
> "name-with-subtask", "attempt", "vertex id", "attempt id", etc...
> Having such an object would allow to pass all these elements simply from
> the TaskDeploymentDescriptor to the Task to the RuntimeContext. And
> whenever we add another field, we need not propagate it manually through
> all the functions calls, but simply add it to the TaskInfo.
>
> What do you think?
>
> —
> Reply to this email directly or view it on GitHub
> .
>



---
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-2488] Expose Attempt Number in RuntimeC...

2015-12-02 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-161286037
  
This generally looks good and pretty straight forward.
As such it is actually good to merge. 

I remember that a while back, we were discussing to create a `TaskInfo` 
object that would contain "subtaskIndex", "parallelism", "name", 
"name-with-subtask", "attempt", "vertex id", "attempt id", etc...
Having such an object would allow to pass all these elements simply from 
the TaskDeploymentDescriptor to the Task to the RuntimeContext. And whenever we 
add another field, we need not propagate it manually through all the functions 
calls, but simply add it to the TaskInfo.

What do you think?


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-30 Thread StephanEwen
Github user StephanEwen commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-160610157
  
I'll shepherd this pull request in...


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-26 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-159853758
  
I'll wait for Stephan to review this then.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-25 Thread rmetzger
Github user rmetzger commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-159662427
  
Hey @sachingoel0101, I think @StephanEwen is the best contact person here. 
Since this affects a very  important interface we shouldn't make any mistakes ;)


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-25 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-159656251
  
Hi @rmetzger, can you take another look? I need to base some further work 
on this [related to cleaning up passing of task name, index, etc.].


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-20 Thread sachingoel0101
Github user sachingoel0101 commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-158408532
  
@rmetzger I have added a test case to verify the functionality.


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-19 Thread rmetzger
Github user rmetzger commented on the pull request:

https://github.com/apache/flink/pull/1386#issuecomment-158118963
  
Can you maybe extend a test which is restarting a job multiple times and 
check there that the attempt number is set properly?


---
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-2488] Expose Attempt Number in RuntimeC...

2015-11-19 Thread sachingoel0101
GitHub user sachingoel0101 opened a pull request:

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

[FLINK-2488] Expose Attempt Number in RuntimeContext

Passes the attempt number all the way from `TaskDeploymentDescriptor` to 
the `RuntimeContext`. 
Small thing I want to confirm: For `RuntimeContext` in Tez, is it okay to 
use `TaskContext#getTaskAttemptNumber` provided by Tez as a proxy for the 
attempt number? 

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

$ git pull https://github.com/sachingoel0101/flink attempt_number

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

https://github.com/apache/flink/pull/1386.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 #1386


commit c7b28bbf7d40b8571ee18d1d525535ab5ed523ca
Author: Sachin Goel 
Date:   2015-11-19T16:38:15Z

[FLINK-2488] Expose Attempt Number in RuntimeContext




---
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.
---