You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sa...@apache.org on 2023/05/23 14:31:28 UTC

[hive] branch branch-3 updated: HIVE-25468: Authorization for Create/Drop functions in HMS(Saihemanth Gantasala via Naveen Gangam)

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

sankarh pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/branch-3 by this push:
     new f906e0246db HIVE-25468: Authorization for Create/Drop functions in HMS(Saihemanth Gantasala via Naveen Gangam)
f906e0246db is described below

commit f906e0246db8ae7eb573c801a02412f529fcbd50
Author: Diksha628 <43...@users.noreply.github.com>
AuthorDate: Tue May 23 20:00:41 2023 +0530

    HIVE-25468: Authorization for Create/Drop functions in HMS(Saihemanth Gantasala via Naveen Gangam)
    
    Co-authored-by: saihemanth <sa...@cloudera.com>
    
    Signed-off-by: Sankar Hariappan <sa...@apache.org>
    Closes (#4342)
---
 .../AuthorizationPreEventListener.java             | 38 ++++++++++++++++++++
 .../hadoop/hive/metastore/HiveMetaStore.java       | 15 ++++++--
 .../metastore/events/PreCreateFunctionEvent.java   | 42 ++++++++++++++++++++++
 .../metastore/events/PreDropFunctionEvent.java     | 42 ++++++++++++++++++++++
 .../hive/metastore/events/PreEventContext.java     |  4 ++-
 .../hive/metastore/client/TestFunctions.java       |  3 +-
 6 files changed, 140 insertions(+), 4 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
index 2cc057ee6e8..fef9fee1afe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
@@ -42,8 +42,10 @@ import org.apache.hadoop.hive.metastore.events.PreAlterDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
@@ -170,6 +172,12 @@ public class AuthorizationPreEventListener extends MetaStorePreEventListener {
     case DROP_DATABASE:
       authorizeDropDatabase((PreDropDatabaseEvent)context);
       break;
+    case CREATE_FUNCTION:
+      authorizeCreateFunction((PreCreateFunctionEvent)context);
+      break;
+    case DROP_FUNCTION:
+      authorizeDropFunction((PreDropFunctionEvent)context);
+      break;
     case LOAD_PARTITION_DONE:
       // noop for now
       break;
@@ -402,6 +410,36 @@ public class AuthorizationPreEventListener extends MetaStorePreEventListener {
     }
   }
 
+  private void authorizeCreateFunction(PreCreateFunctionEvent context)
+          throws InvalidOperationException, MetaException {
+    try {
+      for (HiveMetastoreAuthorizationProvider authorizer : tAuthorizers.get()) {
+        authorizer.authorize(
+                HiveOperation.CREATEFUNCTION.getInputRequiredPrivileges(),
+                HiveOperation.CREATEFUNCTION.getOutputRequiredPrivileges());
+      }
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeDropFunction(PreDropFunctionEvent context)
+          throws InvalidOperationException, MetaException {
+    try {
+      for (HiveMetastoreAuthorizationProvider authorizer : tAuthorizers.get()) {
+        authorizer.authorize(
+                HiveOperation.DROPFUNCTION.getInputRequiredPrivileges(),
+                HiveOperation.DROPFUNCTION.getOutputRequiredPrivileges());
+      }
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
   private void authorizeAlterPartition(PreAlterPartitionEvent context)
       throws InvalidOperationException, MetaException {
     try {
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index f5d5c5a41a5..8270d8bf282 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -133,11 +133,13 @@ import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreAuthorizationCallEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreAddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropSchemaVersionEvent;
@@ -7004,6 +7006,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
       try {
         String catName = func.isSetCatName() ? func.getCatName() : getDefaultCatalog(conf);
+        if (!func.isSetOwnerName()) {
+          try {
+            func.setOwnerName(SecurityUtils.getUGI().getShortUserName());
+          } catch (Exception ex) {
+            LOG.error("Cannot obtain username from the session to create a function", ex);
+            throw new TException(ex);
+          }
+        }
         ms.openTransaction();
         Database db = ms.getDatabase(catName, func.getDbName());
         if (db == null) {
@@ -7015,7 +7025,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           throw new AlreadyExistsException(
               "Function " + func.getFunctionName() + " already exists");
         }
-
+        firePreEvent(new PreCreateFunctionEvent(func, this));
         long time = System.currentTimeMillis() / 1000;
         func.setCreateTime((int) time);
         ms.createFunction(func);
@@ -7059,7 +7069,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
         Boolean isSourceOfReplication =
               ReplChangeManager.isSourceOfReplication(get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]));
-
+        firePreEvent(new PreDropFunctionEvent(func, this));
         // if copy of jar to change management fails we fail the metastore transaction, since the
         // user might delete the jars on HDFS externally after dropping the function, hence having
         // a copy is required to allow incremental replication to work correctly.
@@ -7105,6 +7115,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       String[] parsedDbName = parseDbName(dbName, conf);
       try {
         ms.openTransaction();
+        firePreEvent(new PreCreateFunctionEvent(newFunc, this));
         ms.alterFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName, newFunc);
         success = ms.commitTransaction();
       } finally {
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateFunctionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateFunctionEvent.java
new file mode 100644
index 00000000000..104a19d1521
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateFunctionEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateFunctionEvent extends PreEventContext {
+    private final Function function;
+
+    public PreCreateFunctionEvent(Function function, IHMSHandler handler){
+        super(PreEventType.CREATE_FUNCTION, handler);
+        this.function = function;
+    }
+
+    /**
+     * @return the function
+     */
+    public Function getFunction() {
+        return function;
+    }
+}
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropFunctionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropFunctionEvent.java
new file mode 100644
index 00000000000..683ad387058
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropFunctionEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropFunctionEvent extends PreEventContext {
+    private final Function function;
+
+    public PreDropFunctionEvent(Function function, IHMSHandler handler){
+        super(PreEventType.DROP_FUNCTION, handler);
+        this.function = function;
+    }
+
+    /**
+     * @return the function
+     */
+    public Function getFunction() {
+        return function;
+    }
+}
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
index b93675fd5dd..b703e7b16d0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
@@ -54,7 +54,9 @@ public abstract class PreEventContext {
     CREATE_CATALOG,
     DROP_CATALOG,
     READ_CATALOG,
-    ALTER_CATALOG
+    ALTER_CATALOG,
+    CREATE_FUNCTION,
+    DROP_FUNCTION
   }
 
   private final PreEventType eventType;
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
index 1989caac42a..fd75d6febaf 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
@@ -171,6 +171,7 @@ public class TestFunctions extends MetaStoreClientTest {
     function.setDbName(OTHER_DATABASE);
     function.setFunctionName("test_function");
     function.setClassName(TEST_FUNCTION_CLASS);
+    function.setOwnerName("owner3");
     function.setOwnerType(PrincipalType.USER);
     function.setFunctionType(FunctionType.JAVA);
 
@@ -178,7 +179,7 @@ public class TestFunctions extends MetaStoreClientTest {
 
     Function createdFunction = client.getFunction(function.getDbName(),
         function.getFunctionName());
-    Assert.assertNull("Comparing OwnerName", createdFunction.getOwnerName());
+    Assert.assertEquals("Comparing OwnerName", createdFunction.getOwnerName(), "owner3");
     Assert.assertEquals("Comparing ResourceUris", 0, createdFunction.getResourceUris().size());
     // The create time is set
     Assert.assertNotEquals("Comparing CreateTime", 0, createdFunction.getCreateTime());