You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2021/09/13 18:33:58 UTC

[hive] branch master 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.

ngangam 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 004b4dc  HIVE-25468: Authorization for Create/Drop functions in HMS(Saihemanth Gantasala via Naveen Gangam)
004b4dc is described below

commit 004b4dc5e746ad8d4c1037007d88a9c6338235be
Author: saihemanth <sa...@cloudera.com>
AuthorDate: Thu Aug 19 12:36:43 2021 -0700

    HIVE-25468: Authorization for Create/Drop functions in HMS(Saihemanth Gantasala via Naveen Gangam)
---
 .../plugin/metastore/HiveMetaStoreAuthorizer.java  |  6 ++
 .../metastore/events/CreateFunctionEvent.java      | 96 ++++++++++++++++++++++
 .../plugin/metastore/events/DropFunctionEvent.java | 95 +++++++++++++++++++++
 .../apache/hadoop/hive/metastore/HMSHandler.java   | 12 ++-
 .../metastore/events/PreCreateFunctionEvent.java   | 42 ++++++++++
 .../metastore/events/PreDropFunctionEvent.java     | 42 ++++++++++
 .../hive/metastore/events/PreEventContext.java     |  4 +-
 .../hive/metastore/client/TestFunctions.java       |  3 +-
 8 files changed, 297 insertions(+), 3 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
index 834db95..c3a6ef6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
@@ -422,6 +422,12 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener implement
         case READ_DATABASE:
           authzEvent = new ReadDatabaseEvent(preEventContext);
           break;
+        case CREATE_FUNCTION:
+          authzEvent = new CreateFunctionEvent(preEventContext);
+          break;
+        case DROP_FUNCTION:
+          authzEvent = new DropFunctionEvent(preEventContext);
+          break;
         case AUTHORIZATION_API_CALL:
         case READ_ISCHEMA:
         case CREATE_ISCHEMA:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateFunctionEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateFunctionEvent.java
new file mode 100644
index 0000000..3417e06
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateFunctionEvent.java
@@ -0,0 +1,96 @@
+/*
+ * 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.security.authorization.plugin.metastore.events;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.events.PreCreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/*
+ Authorizable Event for HiveMetaStore operation CreateFunction
+ */
+public class CreateFunctionEvent extends HiveMetaStoreAuthorizableEvent {
+    private static final Logger LOG = LoggerFactory.getLogger(CreateFunctionEvent.class);
+
+    private String COMMAND_STR = "create function";
+
+    public CreateFunctionEvent(PreEventContext preEventContext) {
+        super(preEventContext);
+    }
+
+    @Override
+    public HiveMetaStoreAuthzInfo getAuthzContext() {
+        HiveMetaStoreAuthzInfo ret = new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.CREATEFUNCTION, getInputHObjs(), getOutputHObjs(), COMMAND_STR);
+
+        return ret;
+    }
+
+    private List<HivePrivilegeObject> getInputHObjs() { return Collections.emptyList(); }
+
+    private List<HivePrivilegeObject> getOutputHObjs() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> CreateFunctionEvent.getOutputHObjs()");
+        }
+
+        List<HivePrivilegeObject> ret   = new ArrayList<>();
+        PreCreateFunctionEvent event = (PreCreateFunctionEvent) preEventContext;
+        Function function = event.getFunction();
+        List<ResourceUri> uris   = function.getResourceUris();
+        ret.add(new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.DATABASE, function.getDbName(), null, null, null,
+                HivePrivilegeObject.HivePrivObjectActionType.OTHER, null, null,
+                function.getOwnerName(), function.getOwnerType()));
+        ret.add(new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.FUNCTION, function.getDbName(), function.getFunctionName(), null,
+                null, HivePrivilegeObject.HivePrivObjectActionType.OTHER, null, function.getClassName(), function.getOwnerName(), function.getOwnerType()));
+
+        if (uris != null && !uris.isEmpty()) {
+            for(ResourceUri uri: uris) {
+                ret.add(new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.DFS_URI, null, uri.getUri()));
+            }
+        }
+
+        COMMAND_STR = buildCommandString(function);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== CreateFunctionEvent.getOutputHObjs(): ret=" + ret);
+        }
+
+        return ret;
+    }
+
+    private String buildCommandString(Function function) {
+        String ret = COMMAND_STR;
+        if (function != null) {
+            String functionName = function.getFunctionName();
+            ret = ret + (StringUtils.isNotEmpty(functionName)? " " + functionName : "");
+        }
+        return ret;
+    }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropFunctionEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropFunctionEvent.java
new file mode 100644
index 0000000..d072638
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropFunctionEvent.java
@@ -0,0 +1,95 @@
+/*
+ * 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.security.authorization.plugin.metastore.events;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.events.PreCreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/*
+ Authorizable Event for HiveMetaStore operation DropFunction
+ */
+public class DropFunctionEvent extends HiveMetaStoreAuthorizableEvent {
+    private static final Logger LOG = LoggerFactory.getLogger(DropFunctionEvent.class);
+
+    private String COMMAND_STR = "drop function";
+
+    public DropFunctionEvent(PreEventContext preEventContext) {
+        super(preEventContext);
+    }
+
+    @Override
+    public HiveMetaStoreAuthzInfo getAuthzContext() {
+        HiveMetaStoreAuthzInfo ret = new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.DROPFUNCTION, getInputHObjs(), getOutputHObjs(), COMMAND_STR);
+
+        return ret;
+    }
+
+    private List<HivePrivilegeObject> getInputHObjs() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> DropFunctionEvent.getInputHObjs()");
+        }
+        List<HivePrivilegeObject> ret   = new ArrayList<>();
+        PreDropFunctionEvent event = (PreDropFunctionEvent) preEventContext;
+        Function function = event.getFunction();
+        List<ResourceUri> uris   = function.getResourceUris();
+        ret.add(new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.FUNCTION, function.getDbName(), function.getFunctionName(), null,
+                null, HivePrivilegeObject.HivePrivObjectActionType.OTHER, null, function.getClassName(), function.getOwnerName(), function.getOwnerType()));
+
+        if (uris != null && !uris.isEmpty()) {
+            for(ResourceUri uri: uris) {
+                ret.add(new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.DFS_URI, null, uri.getUri()));
+            }
+        }
+
+        COMMAND_STR = buildCommandString(function);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== DropFunctionEvent.getInputHObjs(): ret=" + ret);
+        }
+
+        return ret;
+    }
+
+    private List<HivePrivilegeObject> getOutputHObjs() {
+        return Collections.emptyList();
+    }
+
+    private String buildCommandString(Function function) {
+        String ret = COMMAND_STR;
+        if (function != null) {
+            String functionName = function.getFunctionName();
+            ret = ret + (StringUtils.isNotEmpty(functionName)? " " + functionName : "");
+        }
+        return ret;
+    }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index c3046bd..4d99066 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -8328,6 +8328,14 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
     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) {
@@ -8343,7 +8351,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
         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);
@@ -8404,6 +8412,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
           }
         }
       }
+      firePreEvent(new PreDropFunctionEvent(func, this));
 
       // if the operation on metastore fails, we don't do anything in change management, but fail
       // the metastore transaction, as having a copy of the jar in change management is not going
@@ -8438,6 +8447,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
     boolean success = false;
     RawStore ms = getMS();
     try {
+      firePreEvent(new PreCreateFunctionEvent(newFunc, this));
       ms.openTransaction();
       ms.alterFunction(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], funcName, newFunc);
       success = ms.commitTransaction();
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateFunctionEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateFunctionEvent.java
new file mode 100644
index 0000000..104a19d
--- /dev/null
+++ b/standalone-metastore/metastore-server/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/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropFunctionEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropFunctionEvent.java
new file mode 100644
index 0000000..683ad38
--- /dev/null
+++ b/standalone-metastore/metastore-server/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/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
index 648a445..f0fb749 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
@@ -57,7 +57,9 @@ public abstract class PreEventContext {
     ALTER_CATALOG,
     CREATE_DATACONNECTOR,
     DROP_DATACONNECTOR,
-    ALTER_DATACONNECTOR
+    ALTER_DATACONNECTOR,
+    CREATE_FUNCTION,
+    DROP_FUNCTION
   }
 
   private final PreEventType eventType;
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
index 58f7363..aafbdd1 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
@@ -173,6 +173,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);
 
@@ -180,7 +181,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());