You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2019/11/25 20:43:29 UTC

[hive] branch master updated: HIVE-22516: TestScheduledQueryIntegration fails occasionally (Zoltan Haindrich reviewed by Miklos Gergely)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new a111af9  HIVE-22516: TestScheduledQueryIntegration fails occasionally (Zoltan Haindrich reviewed by Miklos Gergely)
a111af9 is described below

commit a111af999e8da73e54e34e9c5abd7cefa1e9a54c
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Mon Nov 25 16:55:45 2019 +0000

    HIVE-22516: TestScheduledQueryIntegration fails occasionally (Zoltan Haindrich reviewed by Miklos Gergely)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../hive/schq/TestScheduledQueryIntegration.java   | 50 +++++++++++++---------
 .../hive/ql/parse/ScheduledQueryAnalyzer.java      | 16 ++++++-
 .../scheduled/ScheduledQueryExecutionService.java  |  9 +++-
 .../results/clientpositive/llap/sysdb_schq.q.out   |  2 +-
 4 files changed, 51 insertions(+), 26 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/schq/TestScheduledQueryIntegration.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/schq/TestScheduledQueryIntegration.java
index 0927609..9d1ca5b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/schq/TestScheduledQueryIntegration.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/schq/TestScheduledQueryIntegration.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.schq;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.scheduled.ScheduledQueryExecutionService;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.testutils.HiveTestEnvSetup;
+import org.hamcrest.Matchers;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -45,10 +47,10 @@ import org.junit.rules.TestRule;
 public class TestScheduledQueryIntegration {
 
   @ClassRule
-  public static HiveTestEnvSetup env_setup = new HiveTestEnvSetup();
+  public static HiveTestEnvSetup envSetup = new HiveTestEnvSetup();
 
   @Rule
-  public TestRule methodRule = env_setup.getMethodRule();
+  public TestRule methodRule = envSetup.getMethodRule();
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -65,14 +67,12 @@ public class TestScheduledQueryIntegration {
       driver.run(cmd);
     }
 
-    ScheduledQueryExecutionService.startScheduledQueryExecutorService(env_setup.getTestCtx().hiveConf);
-
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
-    env_setup.getTestCtx().hiveConf.set("hive.test.authz.sstd.hs2.mode", "false");
-    env_setup.getTestCtx().hiveConf.set("hive.security.authorization.enabled", "false");
+    envSetup.getTestCtx().hiveConf.set("hive.test.authz.sstd.hs2.mode", "false");
+    envSetup.getTestCtx().hiveConf.set("hive.security.authorization.enabled", "false");
 
     IDriver driver = createDriver();
     dropTables(driver);
@@ -85,31 +85,37 @@ public class TestScheduledQueryIntegration {
     }
   }
 
-  @Test(expected = CommandProcessorException.class)
+  @Test
   public void testBasicImpersonation() throws ParseException, Exception {
-    CommandProcessorResponse ret;
 
     setupAuthorization();
 
-    ret = runAsUser("user1", "create table t1 (a integer)");
-
-    ret = runAsUser("user2", "drop table t1");
+    runAsUser("user1", "create table t1 (a integer)");
+    try {
+      runAsUser("user2", "drop table t1");
+      fail("Exception expected");
+    } catch (CommandProcessorException cpe) {
+      assertThat(cpe.getErrorMessage(), Matchers.containsString("HiveAccessControlException Permission denied"));
+    }
+    runAsUser("user1", "drop table t1");
   }
 
   @Test
   public void testScheduledQueryExecutionImpersonation() throws ParseException, Exception {
-    env_setup.getTestCtx().hiveConf.setVar(HiveConf.ConfVars.HIVE_SCHEDULED_QUERIES_EXECUTOR_IDLE_SLEEP_TIME, "1s");
-    env_setup.getTestCtx().hiveConf.setVar(HiveConf.ConfVars.HIVE_SCHEDULED_QUERIES_EXECUTOR_PROGRESS_REPORT_INTERVAL,
+    envSetup.getTestCtx().hiveConf.setVar(HiveConf.ConfVars.HIVE_SCHEDULED_QUERIES_EXECUTOR_IDLE_SLEEP_TIME, "1s");
+    envSetup.getTestCtx().hiveConf.setVar(HiveConf.ConfVars.HIVE_SCHEDULED_QUERIES_EXECUTOR_PROGRESS_REPORT_INTERVAL,
         "1s");
     setupAuthorization();
 
-    ScheduledQueryExecutionService.startScheduledQueryExecutorService(env_setup.getTestCtx().hiveConf);
+    try (ScheduledQueryExecutionService schqS =
+        ScheduledQueryExecutionService.startScheduledQueryExecutorService(envSetup.getTestCtx().hiveConf)) {
 
-    runAsUser("user1",
-        "create scheduled query s1 cron '* * * * * ? *' defined as create table tx1 as select 12 as i");
+      runAsUser("user1",
+          "create scheduled query s1 cron '* * * * * ? *' defined as create table tx1 as select 12 as i");
 
+      Thread.sleep(20000);
 
-    Thread.sleep(20000);
+    }
 
     // table exists - and owner is able to select from it
     runAsUser("user1", "select * from tx1");
@@ -127,7 +133,7 @@ public class TestScheduledQueryIntegration {
   }
 
   private CommandProcessorResponse runAsUser(String userName, String sql) throws CommandProcessorException {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
+    HiveConf conf = envSetup.getTestCtx().hiveConf;
     conf.set("user.name", userName);
     try(IDriver driver = createDriver()) {
       return driver.run(sql);
@@ -135,7 +141,7 @@ public class TestScheduledQueryIntegration {
   }
 
   private void setupAuthorization() {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
+    HiveConf conf = envSetup.getTestCtx().hiveConf;
     conf.set("hive.test.authz.sstd.hs2.mode", "true");
     conf.set("hive.security.authorization.manager",
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest");
@@ -159,9 +165,11 @@ public class TestScheduledQueryIntegration {
   }
 
   private static IDriver createDriver() {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
+    HiveConf conf = envSetup.getTestCtx().hiveConf;
 
-    SessionState.start(conf);
+    String userName = conf.get("user.name");
+    SessionState ss = new SessionState(conf, userName);
+    SessionState.start(ss);
 
     IDriver driver = DriverFactory.newDriver(conf);
     return driver;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ScheduledQueryAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ScheduledQueryAnalyzer.java
index f08f9ed..2deb1b7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ScheduledQueryAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ScheduledQueryAnalyzer.java
@@ -87,10 +87,22 @@ public class ScheduledQueryAnalyzer extends BaseSemanticAnalyzer {
   private ScheduledQuery buildEmptySchq() {
     ScheduledQuery ret = new ScheduledQuery();
     ret.setEnabled(true);
-    ret.setUser(SessionState.get().getUserName());
+    ret.setUser(getUserName());
     return ret;
   }
 
+  private String getUserName() {
+    SessionState sessionState = SessionState.get();
+    if (sessionState.getAuthenticator() != null && sessionState.getAuthenticator().getUserName() != null) {
+      return sessionState.getAuthenticator().getUserName();
+    }
+    String userName = sessionState.getUserName();
+    if(userName == null) {
+     throw new RuntimeException("userName is unset; this is unexpected");
+    }
+    return userName;
+  }
+
   /**
    * Composes an overlay object.
    *
@@ -169,7 +181,7 @@ public class ScheduledQueryAnalyzer extends BaseSemanticAnalyzer {
 
     try {
       if (!schqAuthorization) {
-        String currentUser = SessionState.get().getUserName();
+        String currentUser = getUserName();
         if (!Objects.equal(currentUser, schq.getUser())) {
           throw new HiveAccessControlException(
               "authorization of scheduled queries is not enabled - only owners may change scheduled queries");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
index 949b7c7..551d1f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.metastore.api.ScheduledQueryProgressInfo;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorException;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,7 +47,8 @@ public class ScheduledQueryExecutionService implements Closeable {
   private ScheduledQueryExecutionContext context;
   private ScheduledQueryExecutor worker;
 
-  public static ScheduledQueryExecutionService startScheduledQueryExecutorService(HiveConf conf) {
+  public static ScheduledQueryExecutionService startScheduledQueryExecutorService(HiveConf conf0) {
+    HiveConf conf = new HiveConf(conf0);
     MetastoreBasedScheduledQueryService qService = new MetastoreBasedScheduledQueryService(conf);
     ExecutorService executor =
         Executors.newCachedThreadPool(
@@ -107,7 +109,10 @@ public class ScheduledQueryExecutionService implements Closeable {
       try {
         HiveConf conf = new HiveConf(context.conf);
         conf.set(Constants.HIVE_QUERY_EXCLUSIVE_LOCK, lockNameFor(q.getScheduleKey()));
-        state = SessionState.start(conf);
+        conf.setVar(HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER, SessionStateUserAuthenticator.class.getName());
+        conf.unset(HiveConf.ConfVars.HIVESESSIONID.varname);
+        state = new SessionState(conf, q.getUser());
+        SessionState.start(state);
         info = new ScheduledQueryProgressInfo();
         info.setScheduledExecutionId(q.getExecutionId());
         info.setState(QueryState.EXECUTING);
diff --git a/ql/src/test/results/clientpositive/llap/sysdb_schq.q.out b/ql/src/test/results/clientpositive/llap/sysdb_schq.q.out
index b76c6ee..a4f5b77 100644
--- a/ql/src/test/results/clientpositive/llap/sysdb_schq.q.out
+++ b/ql/src/test/results/clientpositive/llap/sysdb_schq.q.out
@@ -3818,7 +3818,7 @@ POSTHOOK: query: select
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@scheduled_queries
 #### A masked pattern was here ####
-1	asd	true	hive	* * * * * ? *	NULL	select 1	true
+1	asd	true	hive	* * * * * ? *	hive_test_user	select 1	true
 PREHOOK: query: select	scheduled_execution_id,
 	scheduled_query_id,
 	state,