This is an automated email from the ASF dual-hosted git repository. jshao pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/incubator-livy.git
The following commit(s) were added to refs/heads/master by this push: new 6c53d2b [LIVY-658] RSCDriver should catch exception if cancel job failed during shutdown 6c53d2b is described below commit 6c53d2b41975f5b0171ce45320deb4b69f8ddea7 Author: Jeffrey(Xilang) Yan <7855100+yan...@users.noreply.github.com> AuthorDate: Sun Sep 29 10:09:10 2019 +0800 [LIVY-658] RSCDriver should catch exception if cancel job failed during shutdown ## What changes were proposed in this pull request? Currently, if startup meet exception, exception will trigger spark to shutdown, then trigger cancel job, but cancel job will throw another exception due to spark is not initialized. The new exception will swallow the old exception. https://issues.apache.org/jira/browse/LIVY-658 Before changes: ![cancel job exception](https://user-images.githubusercontent.com/7855100/64118287-f0961900-cdc9-11e9-9b72-d051fb4bdbdf.jpg) After changes: ![cancel job exception after fix](https://user-images.githubusercontent.com/7855100/64118295-f4c23680-cdc9-11e9-9a2d-38efa0770a99.jpg) ## How was this patch tested? Tested manually, and add unit test. Please review https://livy.incubator.apache.org/community/ before opening a pull request. Author: Jeffrey(Xilang) Yan <7855100+yan...@users.noreply.github.com> Closes #223 from yantzu/initialize_exception_swallow_by_shutdown_exception. --- .../java/org/apache/livy/rsc/driver/RSCDriver.java | 6 +++- .../org/apache/livy/rsc/driver/TestRSCDriver.java | 36 ++++++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java b/rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java index eeba300..0d8eec5 100644 --- a/rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java +++ b/rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java @@ -124,7 +124,11 @@ public class RSCDriver extends BaseProtocol { // Cancel any pending jobs. for (JobWrapper<?> job : activeJobs.values()) { - job.cancel(); + try { + job.cancel(); + } catch (Exception e) { + LOG.warn("Error during cancel job.", e); + } } try { diff --git a/rsc/src/test/java/org/apache/livy/rsc/driver/TestRSCDriver.java b/rsc/src/test/java/org/apache/livy/rsc/driver/TestRSCDriver.java new file mode 100644 index 0000000..df6ccea --- /dev/null +++ b/rsc/src/test/java/org/apache/livy/rsc/driver/TestRSCDriver.java @@ -0,0 +1,36 @@ +/* + * 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.livy.rsc.driver; + +import org.apache.spark.SparkConf; +import org.junit.Test; + +import org.apache.livy.rsc.BaseProtocol; +import org.apache.livy.rsc.RSCConf; + +public class TestRSCDriver { + @Test(expected = IllegalArgumentException.class) + public void testCancelJobAfterInitializeFailed() + throws Exception { + //use empty Conf to trigger initialize throw IllegalArgumentException + RSCDriver rscDriver = new RSCDriver(new SparkConf(), new RSCConf()); + //add asynchronous dummy job request to trigger cancel job failure + rscDriver.handle(null, new BaseProtocol.BypassJobRequest("RequestId", null, null, false)); + rscDriver.run(); + } +}