You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2018/07/03 06:39:46 UTC

hive git commit: HIVE-20051 : Skip authorization for temp tables (Zoltan Haindrich via Jason Dere)

Repository: hive
Updated Branches:
  refs/heads/master f01269123 -> 285a9b4dc


HIVE-20051 : Skip authorization for temp tables (Zoltan Haindrich via Jason Dere)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 285a9b4dc6e8e126864310d0c80426ebc5025cdf
Parents: f012691
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Mon Jul 2 23:39:05 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Mon Jul 2 23:39:05 2018 -0700

----------------------------------------------------------------------
 .../TestHiveAuthorizerCheckInvocation.java      | 94 +++++++++++++++++++-
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 10 ++-
 .../apache/hadoop/hive/ql/metadata/Table.java   |  7 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 13 +--
 4 files changed, 113 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
index e3c83d2..b9ef8b7 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization.plugin;
 
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -27,6 +28,7 @@ import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.File;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -53,7 +55,6 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
 
 /**
  * Test HiveAuthorizer api invocation
@@ -386,6 +387,97 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
+  public void testTempTable() throws Exception {
+
+    String tmpTableDir = getDefaultTmp() + File.separator + "THSAC_testTableTable";
+
+    final String tableName = "testTempTable";
+    { // create temp table
+      reset(mockedAuthorizer);
+      int status = driver.run("create temporary table " + tableName + "(i int) location '" + tmpTableDir + "'")
+          .getResponseCode();
+      assertEquals(0, status);
+
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+
+      // only the URI should be passed for authorization check
+      assertEquals("input count", 1, inputs.size());
+      assertEquals("input type", HivePrivilegeObjectType.LOCAL_URI, inputs.get(0).getType());
+
+      // only the dbname should be passed authorization check
+      assertEquals("output count", 1, outputs.size());
+      assertEquals("output type", HivePrivilegeObjectType.DATABASE, outputs.get(0).getType());
+
+      status = driver.compile("select * from " + tableName);
+      assertEquals(0, status);
+    }
+    { // select from the temp table
+      reset(mockedAuthorizer);
+      int status = driver.compile("insert into " + tableName + " values(1)");
+      assertEquals(0, status);
+
+      // temp tables should be skipped from authorization
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+      System.err.println("inputs " + inputs);
+      System.err.println("outputs " + outputs);
+
+      assertEquals("input count", 0, inputs.size());
+      assertEquals("output count", 0, outputs.size());
+    }
+    { // select from the temp table
+      reset(mockedAuthorizer);
+      int status = driver.compile("select * from " + tableName);
+      assertEquals(0, status);
+
+      // temp tables should be skipped from authorization
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+      System.err.println("inputs " + inputs);
+      System.err.println("outputs " + outputs);
+
+      assertEquals("input count", 0, inputs.size());
+      assertEquals("output count", 0, outputs.size());
+    }
+
+  }
+
+  @Test
+  public void testTempTableImplicit() throws Exception {
+    final String tableName = "testTempTableImplicit";
+    int status = driver.run("create table " + tableName + "(i int)").getResponseCode();
+    assertEquals(0, status);
+
+    reset(mockedAuthorizer);
+    status = driver.compile("insert into " + tableName + " values (1)");
+    assertEquals(0, status);
+
+    List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+    List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+
+    // only the URI should be passed for authorization check
+    assertEquals("input count", 0, inputs.size());
+
+    reset(mockedAuthorizer);
+    status = driver.compile("select * from " + tableName);
+    assertEquals(0, status);
+
+    inputs = getHivePrivilegeObjectInputs().getLeft();
+    outputs = getHivePrivilegeObjectInputs().getRight();
+
+    // temp tables should be skipped from authorization
+    assertEquals("input count", 1, inputs.size());
+    assertEquals("output count", 0, outputs.size());
+
+  }
+
+  private String getDefaultTmp() {
+    return System.getProperty("test.tmp.dir",
+        "target" + File.separator + "test" + File.separator + "tmp");
+  }
+
+  @Test
   public void testUpdateSomeColumnsUsed() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("update " + acidTableName + " set i = 5 where j = 3");

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 762e57c..bf4d29c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -41,8 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
@@ -77,8 +75,8 @@ import org.apache.hadoop.hive.ql.exec.DagUtils;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
-import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
+import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -150,6 +148,7 @@ import org.apache.hive.common.util.TxnIdUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
@@ -1334,6 +1333,11 @@ public class Driver implements IDriver {
         //do not authorize temporary uris
         continue;
       }
+      if (privObject.getTyp() == Type.TABLE
+          && (privObject.getT() == null || privObject.getT().isTemporary())) {
+        // skip temporary tables from authorization
+        continue;
+      }
       //support for authorization on partitions needs to be added
       String dbname = null;
       String objName = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index f0061c0..14e60f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -30,7 +30,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -75,6 +74,8 @@ import org.apache.hive.common.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A Hive Table: is a fundamental unit of data in Hive that shares a common schema/DDL.
  *
@@ -1010,6 +1011,10 @@ public class Table implements Serializable {
     return tTable.isTemporary();
   }
 
+  public void setTemporary(boolean isTemporary) {
+    tTable.setTemporary(isTemporary);
+  }
+
   public static boolean hasMetastoreBasedSchema(HiveConf conf, String serdeLib) {
     return StringUtils.isEmpty(serdeLib) ||
         conf.getStringCollection(ConfVars.SERDESUSINGMETASTOREFORSCHEMA.varname).contains(serdeLib);

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 2731f19..576f337 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -13176,7 +13176,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     case CREATE_TABLE: // REGULAR CREATE TABLE DDL
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
 
       CreateTableDesc crtTblDesc = new CreateTableDesc(dbDotTab, isExt, isTemporary, cols, partCols,
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
@@ -13200,7 +13200,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     case CTLT: // create table like <tbl_name>
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
 
       if (isTemporary) {
         Table likeTable = getTable(likeTableName, false);
@@ -13280,7 +13280,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
       tableDesc = new CreateTableDesc(qualifiedTabName[0], dbDotTab, isExt, isTemporary, cols,
           partCols, bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
           rowFormatParams.fieldEscape, rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim,
@@ -13304,13 +13304,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
   /** Adds entities for create table/create view. */
   private void addDbAndTabToOutputs(String[] qualifiedTabName, TableType type,
-      Map<String, String> tblProps) throws SemanticException {
+      boolean isTemporary, Map<String, String> tblProps) throws SemanticException {
     Database database  = getDatabase(qualifiedTabName[0]);
     outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_SHARED));
 
     Table t = new Table(qualifiedTabName[0], qualifiedTabName[1]);
     t.setParameters(tblProps);
     t.setTableType(type);
+    t.setTemporary(isTemporary);
     outputs.add(new WriteEntity(t, WriteEntity.WriteType.DDL_NO_LOCK));
   }
 
@@ -13411,7 +13412,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(),
           location, storageFormat.getSerde(), storageFormat.getStorageHandler(),
           storageFormat.getSerdeProps());
-      addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW, tblProps);
+      addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW, false, tblProps);
       queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
     } else {
       createVwDesc = new CreateViewDesc(
@@ -13420,7 +13421,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getOutputFormat(), storageFormat.getSerde());
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           createVwDesc)));
-      addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW, tblProps);
+      addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW, false, tblProps);
       queryState.setCommandType(HiveOperation.CREATEVIEW);
     }
     qb.setViewDesc(createVwDesc);