You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by hu...@apache.org on 2018/03/27 01:21:54 UTC

incubator-gobblin git commit: [GOBBLIN-447] If a custom task throws an exception in the run() method, it was not getting marked as complete which would result in the job hanging. Change the task wrapper to mark tasks complete in a finally block.

Repository: incubator-gobblin
Updated Branches:
  refs/heads/master d969f937d -> 54bda2736


[GOBBLIN-447] If a custom task throws an exception in the run() method, it was not getting marked as complete which would result in the job hanging. Change the task wrapper to mark tasks complete in a finally block.

Closes #2313 from eogren/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/54bda273
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/54bda273
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/54bda273

Branch: refs/heads/master
Commit: 54bda27362e203ef099aeb443e3556a8d6efc0a7
Parents: d969f93
Author: Eric Ogren <eo...@linkedin.com>
Authored: Mon Mar 26 18:20:47 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Mon Mar 26 18:21:13 2018 -0700

----------------------------------------------------------------------
 .../gobblin/runtime/task/TaskIFaceWrapper.java  |   7 +-
 .../org/apache/gobblin/task/CustomTaskTest.java |  11 ++
 .../task/FailsWithExceptionTaskFactory.java     | 113 +++++++++++++++++++
 3 files changed, 129 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/54bda273/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/TaskIFaceWrapper.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/TaskIFaceWrapper.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/TaskIFaceWrapper.java
index 6a93f88..bf55c42 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/TaskIFaceWrapper.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/task/TaskIFaceWrapper.java
@@ -73,8 +73,11 @@ public class TaskIFaceWrapper extends Task {
 
   @Override
   public void run() {
-    this.underlyingTask.run();
-    this.taskStateTracker.onTaskRunCompletion(this);
+    try {
+      this.underlyingTask.run();
+    } finally {
+      this.taskStateTracker.onTaskRunCompletion(this);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/54bda273/gobblin-runtime/src/test/java/org/apache/gobblin/task/CustomTaskTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/task/CustomTaskTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/task/CustomTaskTest.java
index 7056b12..2bc02e6 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/task/CustomTaskTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/task/CustomTaskTest.java
@@ -29,12 +29,23 @@ import org.apache.gobblin.writer.test.TestingEventBuses;
 import java.io.File;
 import java.util.Set;
 import java.util.UUID;
+
+import org.junit.internal.runners.statements.Fail;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
 public class CustomTaskTest {
 
+  @Test(timeOut = 30000)
+  public void testTaskFailsWithException() throws Exception {
+    // Test that the job runner fails with a reasonable amount of time if a custom task throws an exception
+    JobExecutionResult result =
+        new EmbeddedGobblin("alwaysThrowsJob").setConfiguration(ConfigurationKeys.SOURCE_CLASS_KEY, FailsWithExceptionTaskFactory.Source.class.getName())
+        .run();
+    Assert.assertFalse(result.isSuccessful());
+  }
+
   @Test
   public void testCustomTask() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/54bda273/gobblin-runtime/src/test/java/org/apache/gobblin/task/FailsWithExceptionTaskFactory.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/task/FailsWithExceptionTaskFactory.java b/gobblin-runtime/src/test/java/org/apache/gobblin/task/FailsWithExceptionTaskFactory.java
new file mode 100644
index 0000000..e71e18a
--- /dev/null
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/task/FailsWithExceptionTaskFactory.java
@@ -0,0 +1,113 @@
+/*
+ * 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.gobblin.task;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.gobblin.configuration.SourceState;
+import org.apache.gobblin.configuration.State;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.publisher.DataPublisher;
+import org.apache.gobblin.runtime.JobState;
+import org.apache.gobblin.runtime.TaskContext;
+import org.apache.gobblin.runtime.task.TaskFactory;
+import org.apache.gobblin.runtime.task.TaskIFace;
+import org.apache.gobblin.runtime.task.TaskUtils;
+import org.apache.gobblin.source.extractor.Extractor;
+import org.apache.gobblin.source.workunit.WorkUnit;
+
+
+public class FailsWithExceptionTaskFactory implements TaskFactory {
+  @Override
+  public TaskIFace createTask(TaskContext taskContext) {
+    return new AlwaysThrowsTask();
+  }
+
+  @Override
+  public DataPublisher createDataPublisher(JobState.DatasetState datasetState) {
+    return null;
+  }
+
+  public static class Source implements org.apache.gobblin.source.Source<String, String> {
+    @Override
+    public List<WorkUnit> getWorkunits(SourceState state) {
+      WorkUnit wu = new WorkUnit();
+      TaskUtils.setTaskFactoryClass(wu, FailsWithExceptionTaskFactory.class);
+      return Collections.singletonList(wu);
+    }
+
+    @Override
+    public Extractor<String, String> getExtractor(WorkUnitState state)
+        throws IOException {
+      return null;
+    }
+
+    @Override
+    public void shutdown(SourceState state) {
+
+    }
+  }
+  private static class AlwaysThrowsTask implements TaskIFace {
+    @Override
+    public void run() {
+      throw new IllegalArgumentException("I always fail with an exception!");
+    }
+
+    @Override
+    public void commit() {
+
+    }
+
+    @Override
+    public State getPersistentState() {
+      return null;
+    }
+
+    @Override
+    public State getExecutionMetadata() {
+      return null;
+    }
+
+    @Override
+    public WorkUnitState.WorkingState getWorkingState() {
+      return null;
+    }
+
+    @Override
+    public void shutdown() {
+
+    }
+
+    @Override
+    public boolean awaitShutdown(long timeoutMillis)
+        throws InterruptedException {
+      return false;
+    }
+
+    @Override
+    public String getProgress() {
+      return null;
+    }
+
+    @Override
+    public boolean isSpeculativeExecutionSafe() {
+      return false;
+    }
+  }
+}