You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2013/08/24 06:31:21 UTC

svn commit: r1517107 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/hooks/ service/src/java/org/apache/hive/service/cli/session/ service/src/test/org/apache/hi...

Author: navis
Date: Sat Aug 24 04:31:20 2013
New Revision: 1517107

URL: http://svn.apache.org/r1517107
Log:
HIVE-4588 : Support session level hooks for HiveServer2 (Prasad Mujumdar via Navis)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHook.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java
    hive/trunk/service/src/test/org/apache/hive/service/cli/session/
    hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1517107&r1=1517106&r2=1517107&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Sat Aug 24 04:31:20 2013
@@ -749,6 +749,7 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS("hive.server2.custom.authentication.class", null),
     HIVE_SERVER2_ENABLE_DOAS("hive.server2.enable.doAs", true),
     HIVE_SERVER2_TABLE_TYPE_MAPPING("hive.server2.table.type.mapping", "HIVE"),
+    HIVE_SERVER2_SESSION_HOOK("hive.server2.session.hook", ""),
 
     HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list", null),
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1517107&r1=1517106&r2=1517107&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Sat Aug 24 04:31:20 2013
@@ -40,8 +40,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
@@ -58,6 +58,7 @@ import org.apache.hadoop.hive.ql.history
 import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
+import org.apache.hadoop.hive.ql.hooks.HookUtils;
 import org.apache.hadoop.hive.ql.hooks.PostExecute;
 import org.apache.hadoop.hive.ql.hooks.PreExecute;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
@@ -82,8 +83,8 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter;
 import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
-import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHook;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl;
 import org.apache.hadoop.hive.ql.parse.ImportSemanticAnalyzer;
@@ -423,20 +424,20 @@ public class Driver implements CommandPr
       tree = ParseUtils.findRootNonNullToken(tree);
 
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
-      List<AbstractSemanticAnalyzerHook> saHooks =
+      List<HiveSemanticAnalyzerHook> saHooks =
           getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
-                   AbstractSemanticAnalyzerHook.class);
+              HiveSemanticAnalyzerHook.class);
 
       // Do semantic analysis and plan generation
       if (saHooks != null) {
         HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
         hookCtx.setConf(conf);
-        for (AbstractSemanticAnalyzerHook hook : saHooks) {
+        for (HiveSemanticAnalyzerHook hook : saHooks) {
           tree = hook.preAnalyze(hookCtx, tree);
         }
         sem.analyze(tree, ctx);
         hookCtx.update(sem);
-        for (AbstractSemanticAnalyzerHook hook : saHooks) {
+        for (HiveSemanticAnalyzerHook hook : saHooks) {
           hook.postAnalyze(hookCtx, sem.getRootTasks());
         }
       } else {
@@ -949,7 +950,8 @@ public class Driver implements CommandPr
     // Get all the driver run hooks and pre-execute them.
     List<HiveDriverRunHook> driverRunHooks;
     try {
-      driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, HiveDriverRunHook.class);
+      driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS,
+          HiveDriverRunHook.class);
       for (HiveDriverRunHook driverRunHook : driverRunHooks) {
           driverRunHook.preDriverRun(hookContext);
       }
@@ -1064,19 +1066,14 @@ public class Driver implements CommandPr
 
   /**
    * Returns a set of hooks specified in a configuration variable.
-   *
    * See getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz)
-   * @param hookConfVar
-   * @return
-   * @throws Exception
    */
   private List<Hook> getHooks(HiveConf.ConfVars hookConfVar) throws Exception {
     return getHooks(hookConfVar, Hook.class);
   }
 
   /**
-   * Returns the hooks specified in a configuration variable.  The hooks are returned in a list in
-   * the order they were specified in the configuration variable.
+   * Returns the hooks specified in a configuration variable.
    *
    * @param hookConfVar The configuration variable specifying a comma separated list of the hook
    *                    class names.
@@ -1085,34 +1082,14 @@ public class Driver implements CommandPr
    *                    they are listed in the value of hookConfVar
    * @throws Exception
    */
-  private <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz)
-      throws Exception {
-
-    List<T> hooks = new ArrayList<T>();
-    String csHooks = conf.getVar(hookConfVar);
-    if (csHooks == null) {
-      return hooks;
-    }
-
-    csHooks = csHooks.trim();
-    if (csHooks.equals("")) {
-      return hooks;
-    }
-
-    String[] hookClasses = csHooks.split(",");
-
-    for (String hookClass : hookClasses) {
-      try {
-        T hook =
-            (T) Class.forName(hookClass.trim(), true, JavaUtils.getClassLoader()).newInstance();
-        hooks.add(hook);
-      } catch (ClassNotFoundException e) {
-        console.printError(hookConfVar.varname + " Class not found:" + e.getMessage());
-        throw e;
-      }
+  private <T extends Hook> List<T> getHooks(ConfVars hookConfVar,
+      Class<T> clazz) throws Exception {
+    try {
+      return HookUtils.getHooks(conf, hookConfVar, clazz);
+    } catch (ClassNotFoundException e) {
+      console.printError(hookConfVar.varname + " Class not found:" + e.getMessage());
+      throw e;
     }
-
-    return hooks;
   }
 
   public int execute() throws CommandNeedRetryException {

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java?rev=1517107&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java Sat Aug 24 04:31:20 2013
@@ -0,0 +1,67 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+
+public class HookUtils {
+  /**
+   * Returns the hooks specified in a configuration variable.  The hooks are returned
+   * in a list in the order they were specified in the configuration variable.
+   *
+   * @param conf        Configuration object
+   * @param hookConfVar The configuration variable specifying a comma separated list
+   *                    of the hook class names.
+   * @param clazz       The super type of the hooks.
+   * @return            A list of the hooks cast as the type specified in clazz,
+   *                    in the order they are listed in the value of hookConfVar
+   * @throws ClassNotFoundException
+   * @throws IllegalAccessException
+   * @throws InstantiationException
+   */
+  public static <T extends Hook> List<T> getHooks(HiveConf conf,
+      ConfVars hookConfVar, Class<T> clazz)
+      throws InstantiationException, IllegalAccessException, ClassNotFoundException  {
+    String csHooks = conf.getVar(hookConfVar);
+    List<T> hooks = new ArrayList<T>();
+    if (csHooks == null) {
+      return hooks;
+    }
+
+    csHooks = csHooks.trim();
+    if (csHooks.equals("")) {
+      return hooks;
+    }
+
+    String[] hookClasses = csHooks.split(",");
+    for (String hookClass : hookClasses) {
+        T hook = (T) Class.forName(hookClass.trim(), true,
+            JavaUtils.getClassLoader()).newInstance();
+        hooks.add(hook);
+    }
+
+    return hooks;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHook.java?rev=1517107&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHook.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHook.java Sat Aug 24 04:31:20 2013
@@ -0,0 +1,37 @@
+/**
+ * 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.session;
+
+import org.apache.hadoop.hive.ql.hooks.Hook;
+import org.apache.hive.service.cli.HiveSQLException;
+
+/**
+ * HiveSessionHook.
+ * HiveServer2 session level Hook interface. The run method is executed
+ *  when session manager starts a new session
+ *
+ */
+public interface HiveSessionHook extends Hook {
+
+  /**
+   * @param sessionHookContext context
+   * @throws HiveSQLException
+   */
+  public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException;
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java?rev=1517107&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java Sat Aug 24 04:31:20 2013
@@ -0,0 +1,50 @@
+/**
+ * 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.session;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+/**
+ * HiveSessionHookContext.
+ * Interface passed to the HiveServer2 session hook execution. This enables
+ * the hook implementation to accesss session config, user and session handle
+ */
+public interface HiveSessionHookContext {
+
+  enum State {OPEN, CLOSE}
+
+  public State getState();
+
+  /**
+   * Retrieve session conf
+   * @return
+   */
+  public HiveConf getSessionConf();
+
+  /**
+   * The get the username starting the session
+   * @return
+   */
+  public String getSessionUser();
+
+  /**
+   * Retrieve handle for the session
+   * @return
+   */
+  public String getSessionHandle();
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java?rev=1517107&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java Sat Aug 24 04:31:20 2013
@@ -0,0 +1,59 @@
+/**
+ * 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.session;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ *
+ * HiveSessionHookContextImpl.
+ * Session hook context implementation which is created by session  manager
+ * and passed to hook invocation.
+ */
+public class HiveSessionHookContextImpl implements HiveSessionHookContext {
+
+  private final HiveSession hiveSession;
+  private final State state;
+
+  HiveSessionHookContextImpl(HiveSession hiveSession, State state) {
+    this.hiveSession = hiveSession;
+    this.state = state;
+  }
+
+  @Override
+  public State getState() {
+    return state;
+  }
+
+  @Override
+  public HiveConf getSessionConf() {
+    return hiveSession.getHiveConf();
+  }
+
+
+  @Override
+  public String getSessionUser() {
+    return hiveSession.getUserName();
+  }
+
+  @Override
+  public String getSessionHandle() {
+    return hiveSession.getSessionHandle().toString();
+  }
+}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java?rev=1517107&r1=1517106&r2=1517107&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java Sat Aug 24 04:31:20 2013
@@ -19,9 +19,11 @@
 package org.apache.hive.service.cli.session;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.hooks.HookUtils;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
@@ -72,16 +74,15 @@ public class SessionManager extends Comp
 
   public SessionHandle openSession(String username, String password, Map<String, String> sessionConf,
           boolean withImpersonation, String delegationToken) throws HiveSQLException {
-    HiveSession session;
     if (username == null) {
       username = threadLocalUserName.get();
     }
-
+    HiveSession session;
     if (withImpersonation) {
-          HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(username, password, sessionConf,
-              delegationToken);
-          session = (HiveSession)HiveSessionProxy.getProxy(hiveSessionUgi, hiveSessionUgi.getSessionUgi());
-          hiveSessionUgi.setProxySession(session);
+      HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(username, password,
+        sessionConf, delegationToken);
+      session = HiveSessionProxy.getProxy(hiveSessionUgi, hiveSessionUgi.getSessionUgi());
+      hiveSessionUgi.setProxySession(session);
     } else {
       session = new HiveSessionImpl(username, password, sessionConf);
     }
@@ -90,6 +91,11 @@ public class SessionManager extends Comp
     synchronized(sessionMapLock) {
       handleToSession.put(session.getSessionHandle(), session);
     }
+    try {
+      executeSessionHooks(session, HiveSessionHookContext.State.OPEN);
+    } catch (Exception e) {
+      throw new HiveSQLException("Failed to execute session hooks", e);
+    }
     return session.getSessionHandle();
   }
 
@@ -101,6 +107,11 @@ public class SessionManager extends Comp
     if (session == null) {
       throw new HiveSQLException("Session does not exist!");
     }
+    try {
+      executeSessionHooks(session, HiveSessionHookContext.State.CLOSE);
+    } catch (Exception e) {
+      throw new HiveSQLException("Failed to execute session hooks", e);
+    }
     session.close();
   }
 
@@ -150,4 +161,13 @@ public class SessionManager extends Comp
     threadLocalUserName.remove();
   }
 
+  // execute session hooks
+  private void executeSessionHooks(HiveSession session, HiveSessionHookContext.State state)
+      throws Exception {
+    List<HiveSessionHook> sessionHooks = HookUtils.getHooks(hiveConf,
+        HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class);
+    for (HiveSessionHook sessionHook : sessionHooks) {
+      sessionHook.run(new HiveSessionHookContextImpl(session, state));
+    }
+  }
 }

Added: hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java?rev=1517107&view=auto
==============================================================================
--- hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java (added)
+++ hive/trunk/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java Sat Aug 24 04:31:20 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.session;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.thrift.EmbeddedThriftCLIService;
+import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSessionHooks extends TestCase {
+
+  public static final String SESSION_USER_NAME = "user1";
+  private EmbeddedThriftCLIService service;
+  private ThriftCLIServiceClient client;
+
+  public static class SessionHookTest implements HiveSessionHook {
+
+   public static AtomicInteger runCount = new AtomicInteger(0);
+
+    @Override
+    public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException {
+      Assert.assertEquals(sessionHookContext.getSessionUser(), SESSION_USER_NAME);
+      String sessionHook = sessionHookContext.getSessionConf().
+          getVar(ConfVars.HIVE_SERVER2_SESSION_HOOK);
+      Assert.assertTrue(sessionHook.contains(this.getClass().getName()));
+      Assert.assertEquals(0, runCount.getAndIncrement());
+    }
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    System.setProperty(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
+        TestSessionHooks.SessionHookTest.class.getName());
+    service = new EmbeddedThriftCLIService();
+    client = new ThriftCLIServiceClient(service);
+  }
+
+  @Test
+  public void testSessionHook () throws Exception {
+    // create session, test if the hook got fired by checking the expected property
+    SessionHandle sessionHandle = client.openSession(SESSION_USER_NAME, "foobar",
+          Collections.<String, String>emptyMap());
+    Assert.assertEquals(1, SessionHookTest.runCount.get());
+    client.closeSession(sessionHandle);
+  }
+}