You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by GitBox <gi...@apache.org> on 2020/03/12 05:36:53 UTC

[GitHub] [incubator-gobblin] autumnust opened a new pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

autumnust opened a new pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923
 
 
   Dear Gobblin maintainers,
   
   Please accept this PR. I understand that it will not be reviewed until I have checked off all the steps below!
   
   
   ### JIRA
   - [ ] My PR addresses the following [Gobblin JIRA](https://issues.apache.org/jira/browse/GOBBLIN/) issues and references them in the PR title. For example, "[GOBBLIN-XXX] My Gobblin PR"
       - https://issues.apache.org/jira/browse/GOBBLIN-XXX
   
   
   ### Description
   - [ ] Here are some details about my PR, including screenshots (if applicable):
   
   
   ### Tests
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   
   ### Commits
   - [ ] My commits all reference JIRA issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
       1. Subject is separated from body by a blank line
       2. Subject is limited to 50 characters
       3. Subject does not end with a period
       4. Subject uses the imperative mood ("add", not "adding")
       5. Body wraps at 72 characters
       6. Body explains "what" and "why", not "how"
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404621465
 
 

 ##########
 File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/BaseAbstractTask.java
 ##########
 @@ -91,4 +94,13 @@ public String getProgress() {
   public boolean isSpeculativeExecutionSafe() {
     return false;
   }
+
+  /**
+   * Similar to org.apache.gobblin.runtime.Task#failTask(java.lang.Throwable), we need to propagate exception thrown
+   * in workunit level by setting {@link ConfigurationKeys#TASK_FAILURE_EXCEPTION_KEY}
+   */
+  protected void failTask(Throwable t, TaskContext taskContext) {
+    taskContext.getTaskState().setWorkingState(WorkUnitState.WorkingState.FAILED);
 
 Review comment:
   Should we save taskContext in BaseAbstractTask ? In this way, signature of this method can be consistent with that in Task.java

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404606800
 
 

 ##########
 File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SleepingTask.java
 ##########
 @@ -20,27 +20,30 @@
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.gobblin.runtime.TaskContext;
+import org.apache.gobblin.runtime.task.BaseAbstractTask;
+
 import com.google.common.base.Throwables;
 import com.google.common.io.Files;
 
 import lombok.extern.slf4j.Slf4j;
 
-import org.apache.gobblin.runtime.TaskContext;
-import org.apache.gobblin.runtime.TaskState;
-import org.apache.gobblin.runtime.task.BaseAbstractTask;
-
 @Slf4j
 public class SleepingTask extends BaseAbstractTask {
   public static final String TASK_STATE_FILE_KEY = "task.state.file.path";
+  public static final String SLEEPING_TASK_SLEEP_TIME = "data.publisher.sleep.time.in.seconds";
 
   private final long sleepTime;
   private File taskStateFile;
+  private final TaskContext taskContext;
 
   public SleepingTask(TaskContext taskContext) {
     super(taskContext);
-    TaskState taskState = taskContext.getTaskState();
-    sleepTime = taskState.getPropAsLong("data.publisher.sleep.time.in.seconds", 10L);
-    taskStateFile = new File(taskState.getProp(TASK_STATE_FILE_KEY));
+    this.taskContext = taskContext;
+    sleepTime = taskContext.getTaskState().getPropAsLong(SLEEPING_TASK_SLEEP_TIME, 10L);
 
 Review comment:
   why `TaskState taskState = taskContext.getTaskState();` is removed?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] autumnust edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
autumnust edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-608950010
 
 
   @arjun4084346  sorry about missing the description. Adding some explanation on those, specifically returning FAILED in case of CANCELLED is recommended by Helix to maximize the chance of a cancelled task to be rescheduled, so that in Gobblin-Streaming we don't have a kafka partition ( part of cancelled helix) being orphaned. 
   
   @arjun4084346  Can you also take a look the PR ? Thanks. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] autumnust commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
autumnust commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-608950010
 
 
   @arjun4084346  sorry about missing the description. Adding some explanation on those, specifically returning FAILED in case of CANCELLED is recommended by Helix to maximize the chance of a cancelled task to be rescheduled, so that in Gobblin-Streaming we don't have a kafka partition ( part of cancelled helix) being orphaned. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404623597
 
 

 ##########
 File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/FailedTask.java
 ##########
 @@ -16,19 +16,30 @@
  */
 package org.apache.gobblin.runtime.task;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.publisher.DataPublisher;
 import org.apache.gobblin.publisher.NoopPublisher;
 import org.apache.gobblin.runtime.JobState;
 import org.apache.gobblin.runtime.TaskContext;
+import org.apache.gobblin.runtime.TaskState;
 import org.apache.gobblin.source.workunit.WorkUnit;
 
+import com.google.common.base.Throwables;
+
+import groovy.util.logging.Slf4j;
+
+
 /**
- * A task which raise an exception when run
+ * A task which returns "FAILED" state directly.
  */
+@Slf4j
 public class FailedTask extends BaseAbstractTask {
-  public FailedTask (TaskContext taskContext) {
+  private final TaskContext taskContext;
 
 Review comment:
   Oh yes, maybe have this in BaseAbstractTask so it can be removed from all child classes?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] codecov-io commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-608952545
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=h1) Report
   > Merging [#2923](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-gobblin/commit/9ef461ad37bb54c023bd383608a3c4024adad06f&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `37.50%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc)](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2923      +/-   ##
   ============================================
   - Coverage     45.59%   45.59%   -0.01%     
   + Complexity     9160     9159       -1     
   ============================================
     Files          1936     1936              
     Lines         73295    73300       +5     
     Branches       8088     8088              
   ============================================
     Hits          33418    33418              
   - Misses        36781    36787       +6     
   + Partials       3096     3095       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...a/org/apache/gobblin/cluster/GobblinHelixTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4VGFzay5qYXZh) | `77.61% <0.00%> (ø)` | `6.00 <0.00> (ø)` | |
   | [...e/gobblin/cluster/TaskRunnerSuiteProcessModel.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvVGFza1J1bm5lclN1aXRlUHJvY2Vzc01vZGVsLmphdmE=) | `100.00% <ø> (ø)` | `5.00 <0.00> (ø)` | |
   | [...he/gobblin/writer/FineGrainedWatermarkTracker.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jb3JlLWJhc2Uvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vd3JpdGVyL0ZpbmVHcmFpbmVkV2F0ZXJtYXJrVHJhY2tlci5qYXZh) | `82.25% <ø> (ø)` | `28.00 <0.00> (ø)` | |
   | [...src/main/java/org/apache/gobblin/runtime/Task.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvVGFzay5qYXZh) | `67.26% <ø> (ø)` | `84.00 <0.00> (ø)` | |
   | [.../apache/gobblin/runtime/task/BaseAbstractTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvdGFzay9CYXNlQWJzdHJhY3RUYXNrLmphdmE=) | `52.38% <0.00%> (-8.74%)` | `6.00 <0.00> (ø)` | |
   | [...va/org/apache/gobblin/runtime/task/FailedTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvdGFzay9GYWlsZWRUYXNrLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...in/java/org/apache/gobblin/cluster/HelixUtils.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhVdGlscy5qYXZh) | `37.30% <50.00%> (-3.18%)` | `15.00 <0.00> (-1.00)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `48.48% <80.00%> (+9.09%)` | `3.00 <0.00> (ø)` | |
   | [.../org/apache/gobblin/metrics/RootMetricContext.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1tZXRyaWNzLWxpYnMvZ29iYmxpbi1tZXRyaWNzLWJhc2Uvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vbWV0cmljcy9Sb290TWV0cmljQ29udGV4dC5qYXZh) | `78.12% <0.00%> (-1.57%)` | `15.00% <0.00%> (-1.00%)` | |
   | ... and [2 more](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=footer). Last update [9ef461a...cc02f49](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-606877502
 
 
   Can you add a 'Description' in the PR. I did not understand why you are trying to return a HelixTask return Failed when it is cancelled?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] sv2000 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404329650
 
 

 ##########
 File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixUtils.java
 ##########
 @@ -188,10 +192,10 @@ static void waitJobCompletion(HelixManager helixManager, String workFlowName, St
             // user requested cancellation, which is executed by executeCancellation()
             log.info("Job {} is cancelled, it will be deleted now.", jobName);
             HelixUtils.deleteStoppedHelixJob(helixManager, workFlowName, jobName);
-            return;
+            return false;
           case FAILED:
 
 Review comment:
   Based on the javadoc, shouldn't we return false for FAILED?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-606877502
 
 
   Can you add a 'Description' in the PR. I did not understand why you are trying to make a HelixTask return Failed when it is cancelled?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404622306
 
 

 ##########
 File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/BaseAbstractTask.java
 ##########
 @@ -91,4 +94,13 @@ public String getProgress() {
   public boolean isSpeculativeExecutionSafe() {
     return false;
   }
+
+  /**
+   * Similar to org.apache.gobblin.runtime.Task#failTask(java.lang.Throwable), we need to propagate exception thrown
+   * in workunit level by setting {@link ConfigurationKeys#TASK_FAILURE_EXCEPTION_KEY}
+   */
+  protected void failTask(Throwable t, TaskContext taskContext) {
+    taskContext.getTaskState().setWorkingState(WorkUnitState.WorkingState.FAILED);
+    taskContext.getTaskState().setProp(ConfigurationKeys.TASK_FAILURE_EXCEPTION_KEY, Throwables.getStackTraceAsString(t));
+  }
 
 Review comment:
   No failureEvent here?
   Is there a reasonable way to not have duplicate code?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-608950990
 
 
   I see. Just keep that in mind that sometimes, we do not want to reschedule it, e.g. when user cancelled the job in GaaS. Is there a way to do that?
   Will take a deeper look on Monday. Happy Friday!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404608709
 
 

 ##########
 File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SleepingTask.java
 ##########
 @@ -73,10 +76,10 @@ public void run() {
     } catch (InterruptedException e) {
       log.error("Sleep interrupted.");
       Thread.currentThread().interrupt();
-      Throwables.propagate(e);
+      failTask(e, taskContext);
 
 Review comment:
   SleepingTask was written to test cancellation code or hanging issues. It is meant to sleep for some amount of time to simulate as a hanging task in production. It is supposed propagate exceptions like any normal task would do.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404623597
 
 

 ##########
 File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/FailedTask.java
 ##########
 @@ -16,19 +16,30 @@
  */
 package org.apache.gobblin.runtime.task;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.publisher.DataPublisher;
 import org.apache.gobblin.publisher.NoopPublisher;
 import org.apache.gobblin.runtime.JobState;
 import org.apache.gobblin.runtime.TaskContext;
+import org.apache.gobblin.runtime.TaskState;
 import org.apache.gobblin.source.workunit.WorkUnit;
 
+import com.google.common.base.Throwables;
+
+import groovy.util.logging.Slf4j;
+
+
 /**
- * A task which raise an exception when run
+ * A task which returns "FAILED" state directly.
  */
+@Slf4j
 public class FailedTask extends BaseAbstractTask {
-  public FailedTask (TaskContext taskContext) {
+  private final TaskContext taskContext;
 
 Review comment:
   Oh yes, maybe have this in BaseAbstractTask so it can be removed from all child classes like I said in previous comment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] codecov-io edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#issuecomment-608952545
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=h1) Report
   > Merging [#2923](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-gobblin/commit/9ef461ad37bb54c023bd383608a3c4024adad06f&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `37.50%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc)](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2923      +/-   ##
   ============================================
   - Coverage     45.59%   45.59%   -0.01%     
   + Complexity     9160     9159       -1     
   ============================================
     Files          1936     1936              
     Lines         73295    73300       +5     
     Branches       8088     8088              
   ============================================
     Hits          33418    33418              
   - Misses        36781    36787       +6     
   + Partials       3096     3095       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...a/org/apache/gobblin/cluster/GobblinHelixTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4VGFzay5qYXZh) | `77.61% <0.00%> (ø)` | `6.00 <0.00> (ø)` | |
   | [...e/gobblin/cluster/TaskRunnerSuiteProcessModel.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvVGFza1J1bm5lclN1aXRlUHJvY2Vzc01vZGVsLmphdmE=) | `100.00% <ø> (ø)` | `5.00 <0.00> (ø)` | |
   | [...he/gobblin/writer/FineGrainedWatermarkTracker.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jb3JlLWJhc2Uvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vd3JpdGVyL0ZpbmVHcmFpbmVkV2F0ZXJtYXJrVHJhY2tlci5qYXZh) | `82.25% <ø> (ø)` | `28.00 <0.00> (ø)` | |
   | [...src/main/java/org/apache/gobblin/runtime/Task.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvVGFzay5qYXZh) | `67.26% <ø> (ø)` | `84.00 <0.00> (ø)` | |
   | [.../apache/gobblin/runtime/task/BaseAbstractTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvdGFzay9CYXNlQWJzdHJhY3RUYXNrLmphdmE=) | `52.38% <0.00%> (-8.74%)` | `6.00 <0.00> (ø)` | |
   | [...va/org/apache/gobblin/runtime/task/FailedTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1ydW50aW1lL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3J1bnRpbWUvdGFzay9GYWlsZWRUYXNrLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...in/java/org/apache/gobblin/cluster/HelixUtils.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhVdGlscy5qYXZh) | `37.30% <50.00%> (-3.18%)` | `15.00 <0.00> (-1.00)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `48.48% <80.00%> (+9.09%)` | `3.00 <0.00> (ø)` | |
   | [.../org/apache/gobblin/metrics/RootMetricContext.java](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree#diff-Z29iYmxpbi1tZXRyaWNzLWxpYnMvZ29iYmxpbi1tZXRyaWNzLWJhc2Uvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vbWV0cmljcy9Sb290TWV0cmljQ29udGV4dC5qYXZh) | `78.12% <0.00%> (-1.57%)` | `15.00% <0.00%> (-1.00%)` | |
   | ... and [2 more](https://codecov.io/gh/apache/incubator-gobblin/pull/2923/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=footer). Last update [9ef461a...cc02f49](https://codecov.io/gh/apache/incubator-gobblin/pull/2923?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404626735
 
 

 ##########
 File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/FailedTask.java
 ##########
 @@ -16,19 +16,30 @@
  */
 package org.apache.gobblin.runtime.task;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.publisher.DataPublisher;
 import org.apache.gobblin.publisher.NoopPublisher;
 import org.apache.gobblin.runtime.JobState;
 import org.apache.gobblin.runtime.TaskContext;
+import org.apache.gobblin.runtime.TaskState;
 import org.apache.gobblin.source.workunit.WorkUnit;
 
+import com.google.common.base.Throwables;
+
+import groovy.util.logging.Slf4j;
+
+
 /**
- * A task which raise an exception when run
+ * A task which returns "FAILED" state directly.
 
 Review comment:
   If the code on the left raises an exception, why the code on the right would not raise an exception? javadoc on the left is incorrect?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404608709
 
 

 ##########
 File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SleepingTask.java
 ##########
 @@ -73,10 +76,10 @@ public void run() {
     } catch (InterruptedException e) {
       log.error("Sleep interrupted.");
       Thread.currentThread().interrupt();
-      Throwables.propagate(e);
+      failTask(e, taskContext);
 
 Review comment:
   SleepingTask was written to test cancellation code or hanging issues. It is meant to sleep for some amount to simulate as a hanging task in production. It is supposed propagate exceptions like any normal task would do.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-gobblin] sv2000 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #2923: [GOBBLIN-1083] Unit test improving & return failed when helix task cancelled
URL: https://github.com/apache/incubator-gobblin/pull/2923#discussion_r404331133
 
 

 ##########
 File path: gobblin-core-base/src/main/java/org/apache/gobblin/writer/FineGrainedWatermarkTracker.java
 ##########
 @@ -29,6 +29,15 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.gobblin.configuration.State;
 
 Review comment:
   Should we leave this class unchanged, since it is unrelated to the PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services