瀏覽代碼

[LIVY-466][RSC] Fix RSCDriver exception during RPC shutdown

## What changes were proposed in this pull request?

During RSCDriver's shutdown, it will first shutdown RPC server, and then all the RPC clients. When RPC client is closed, it will register a timeout to avoid orphaned RSCDriver, but this is not necessary during RSCDriver's shutdown, so here fixing this issue. The details can be seen in [JIRA](https://issues.apache.org/jira/browse/LIVY-466).

## How was this patch tested?

Local verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #90 from jerryshao/LIVY-466.
jerryshao 7 年之前
父節點
當前提交
e3f45a057c
共有 1 個文件被更改,包括 7 次插入1 次删除
  1. 7 1
      rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java

+ 7 - 1
rsc/src/main/java/org/apache/livy/rsc/driver/RSCDriver.java

@@ -37,6 +37,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import io.netty.channel.ChannelHandler.Sharable;
@@ -92,6 +93,7 @@ public class RSCDriver extends BaseProtocol {
   protected final RSCConf livyConf;
 
   private final AtomicReference<ScheduledFuture<?>> idleTimeout;
+  private final AtomicBoolean inShutdown;
 
   public RSCDriver(SparkConf conf, RSCConf livyConf) throws Exception {
     Set<PosixFilePermission> perms = PosixFilePermissions.fromString("rwx------");
@@ -110,6 +112,7 @@ public class RSCDriver extends BaseProtocol {
     this.activeJobs = new ConcurrentHashMap<>();
     this.bypassJobs = new ConcurrentLinkedDeque<>();
     this.idleTimeout = new AtomicReference<>();
+    this.inShutdown = new AtomicBoolean(false);
   }
 
   private synchronized void shutdown() {
@@ -217,7 +220,9 @@ public class RSCDriver extends BaseProtocol {
       @Override
       public void onSuccess(Void unused) {
         clients.remove(client);
-        setupIdleTimeout();
+        if (!inShutdown.get()) {
+          setupIdleTimeout();
+        }
       }
     });
     LOG.debug("Registered new connection from {}.", client.getChannel());
@@ -304,6 +309,7 @@ public class RSCDriver extends BaseProtocol {
   }
 
   private void shutdownServer() {
+    inShutdown.compareAndSet(false, true);
     if (server != null) {
       server.close();
     }