You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2016/09/09 09:42:44 UTC

hive git commit: HIVE-14591: HS2 is shut down unexpectedly during the startup time (Tao Li reviewed by Vaibhav Gumashta)

Repository: hive
Updated Branches:
  refs/heads/master 0755348d7 -> 5edf7c833


HIVE-14591: HS2 is shut down unexpectedly during the startup time (Tao Li reviewed by Vaibhav Gumashta)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5edf7c83
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5edf7c83
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5edf7c83

Branch: refs/heads/master
Commit: 5edf7c8335279a8393300bfb11743c144d23836d
Parents: 0755348
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Fri Sep 9 02:35:37 2016 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Fri Sep 9 02:35:37 2016 -0700

----------------------------------------------------------------------
 .../hive/jdbc/miniHS2/AbstractHiveService.java  |  2 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |  4 +
 .../thrift/TestMiniHS2StateWithNoZookeeper.java | 82 ++++++++++++++++++++
 .../service/cli/session/SessionManager.java     |  2 +-
 .../apache/hive/service/server/HiveServer2.java | 17 ++--
 5 files changed, 97 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
index 2c1cd07..98c5d01 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
@@ -140,7 +140,7 @@ public abstract class AbstractHiveService {
     return startedHiveService;
   }
 
-  protected void setStarted(boolean hiveServiceStatus) {
+  public void setStarted(boolean hiveServiceStatus) {
     this.startedHiveService =  hiveServiceStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index de1ce76..abb80a2 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -502,4 +502,8 @@ public class MiniHS2 extends AbstractHiveService {
       break;
     } while (true);
   }
+  
+  public Service.STATE getState() {
+    return hiveServer2.getServiceState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
new file mode 100644
index 0000000..5114803
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestMiniHS2StateWithNoZookeeper.java
@@ -0,0 +1,82 @@
+/**
+ * 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.hive.service.cli.thrift;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.service.Service;
+import org.apache.hive.service.cli.CLIServiceClient;
+import org.apache.hive.service.cli.SessionHandle;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+*
+* TestMiniHS2StateWithNoZookeeper.
+* This tests HS2 shutdown is not triggered by CloseSession operation 
+* while HS2 has never been registered with ZooKeeper.
+*
+*/
+
+public class TestMiniHS2StateWithNoZookeeper {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(TestMiniHS2StateWithNoZookeeper.class);
+  private static MiniHS2 miniHS2 = null;
+  private static HiveConf hiveConf = null;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception   { 
+    hiveConf = new HiveConf();
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY, true);
+    hiveConf.setIntVar(ConfVars.HIVE_ZOOKEEPER_CONNECTION_MAX_RETRIES, 0);
+    hiveConf.setTimeVar(ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME, 0, TimeUnit.MILLISECONDS);
+    miniHS2 = new MiniHS2(hiveConf);
+    Map<String, String> confOverlay = new HashMap<String, String>();
+    try {
+      miniHS2.start(confOverlay);
+    } catch (Exception ex) {
+      LOG.warn("Zookeeper is not set up intentionally, so the error is expected (unless it's not related to ZK): " + ex);
+      miniHS2.setStarted(true);
+    }
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    miniHS2.stop();
+  }
+
+  @Test
+  public void openSessionAndClose() throws Exception {
+    CLIServiceClient client = miniHS2.getServiceClient();
+    SessionHandle sessionHandle = client.openSession(null, null, null);
+    client.closeSession(sessionHandle);
+    Thread.sleep(100);
+
+    Assert.assertEquals(Service.STATE.STARTED, miniHS2.getState());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index 88dee48..15bab06 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -366,7 +366,7 @@ public class SessionManager extends CompositeService {
     } finally {
       // Shutdown HiveServer2 if it has been deregistered from ZooKeeper and has no active sessions
       if (!(hiveServer2 == null) && (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY))
-          && (!hiveServer2.isRegisteredWithZooKeeper())) {
+          && (hiveServer2.isDeregisteredWithZooKeeper())) {
         // Asynchronously shutdown this instance of HiveServer2,
         // if there are no active client sessions
         if (getOpenSessionCount() == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/5edf7c83/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 0ecaa76..9822a85 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -94,7 +94,7 @@ public class HiveServer2 extends CompositeService {
   private PersistentEphemeralNode znode;
   private String znodePath;
   private CuratorFramework zooKeeperClient;
-  private boolean registeredWithZooKeeper = false;
+  private boolean deregisteredWithZooKeeper = false; // Set to true only when deregistration happens
   private HttpServer webServer; // Web UI
 
   public HiveServer2() {
@@ -324,7 +324,7 @@ public class HiveServer2 extends CompositeService {
       if (!znode.waitForInitialCreate(znodeCreationTimeout, TimeUnit.SECONDS)) {
         throw new Exception("Max znode creation wait time: " + znodeCreationTimeout + "s exhausted");
       }
-      setRegisteredWithZooKeeper(true);
+      setDeregisteredWithZooKeeper(false);
       znodePath = znode.getActualPath();
       // Set a watch on the znode
       if (zooKeeperClient.checkExists().usingWatcher(new DeRegisterWatcher()).forPath(znodePath) == null) {
@@ -415,7 +415,7 @@ public class HiveServer2 extends CompositeService {
           } catch (IOException e) {
             LOG.error("Failed to close the persistent ephemeral znode", e);
           } finally {
-            HiveServer2.this.setRegisteredWithZooKeeper(false);
+            HiveServer2.this.setDeregisteredWithZooKeeper(true);
             // If there are no more active client sessions, stop the server
             if (cliService.getSessionManager().getOpenSessionCount() == 0) {
               LOG.warn("This instance of HiveServer2 has been removed from the list of server "
@@ -430,7 +430,8 @@ public class HiveServer2 extends CompositeService {
   }
 
   private void removeServerInstanceFromZooKeeper() throws Exception {
-    setRegisteredWithZooKeeper(false);
+    setDeregisteredWithZooKeeper(true);
+    
     if (znode != null) {
       znode.close();
     }
@@ -438,12 +439,12 @@ public class HiveServer2 extends CompositeService {
     LOG.info("Server instance removed from ZooKeeper.");
   }
 
-  public boolean isRegisteredWithZooKeeper() {
-    return registeredWithZooKeeper;
+  public boolean isDeregisteredWithZooKeeper() {
+    return deregisteredWithZooKeeper;
   }
 
-  private void setRegisteredWithZooKeeper(boolean registeredWithZooKeeper) {
-    this.registeredWithZooKeeper = registeredWithZooKeeper;
+  private void setDeregisteredWithZooKeeper(boolean deregisteredWithZooKeeper) {
+    this.deregisteredWithZooKeeper = deregisteredWithZooKeeper;
   }
 
   private String getServerInstanceURI() throws Exception {