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 2022/08/19 01:52:19 UTC

[hive] branch master updated: HIVE-26248: Add data connector authorization on HMS server-side (#3312)

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 82f319773cb HIVE-26248: Add data connector authorization on HMS server-side (#3312)
82f319773cb is described below

commit 82f319773cb2361a98963e861fb903ab8eecd9c4
Author: Butao Zhang <97...@users.noreply.github.com>
AuthorDate: Fri Aug 19 09:52:13 2022 +0800

    HIVE-26248: Add data connector authorization on HMS server-side (#3312)
    
    * HIVE-26248: Add data connector authorization on HMS server-side
    
    * Pass oldDC object in getInputHObjs()
    
    * Address review comment: Remove empty line and handle exception
    
    * Use Assert.fail() to handle UT exception
    
    * Do not create connector repeatedly
---
 .../metastore/HiveMetaStoreAuthorizableEvent.java  |   7 ++
 .../plugin/metastore/HiveMetaStoreAuthorizer.java  |   9 ++
 .../metastore/events/AlterDataConnectorEvent.java  | 100 +++++++++++++++++++++
 .../metastore/events/CreateDataConnectorEvent.java |  93 +++++++++++++++++++
 .../metastore/events/DropDataConnectorEvent.java   |  88 ++++++++++++++++++
 .../metastore/TestHiveMetaStoreAuthorizer.java     |  51 +++++++++++
 .../apache/hadoop/hive/metastore/HMSHandler.java   |   6 +-
 .../events/PreAlterDataConnectorEvent.java         |  47 ++++++++++
 .../events/PreDropDataConnectorEvent.java          |  44 +++++++++
 9 files changed, 442 insertions(+), 3 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizableEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizableEvent.java
index 2bae723c1d8..a801960ff57 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizableEvent.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizableEvent.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization.plugin.metastore;
 
+import org.apache.hadoop.hive.metastore.api.DataConnector;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -71,4 +72,10 @@ public abstract class HiveMetaStoreAuthorizableEvent {
     return new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.LOCAL_URI, null, uri);
   }
 
+  protected HivePrivilegeObject getHivePrivilegeObject(DataConnector connector) {
+    return new HivePrivilegeObject(HivePrivilegeObject.HivePrivilegeObjectType.DATACONNECTOR, null,
+        connector.getName(), null, null, HivePrivilegeObject.HivePrivObjectActionType.OTHER, null, null,
+        connector.getOwnerName(), connector.getOwnerType());
+  }
+
 }
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 c276e6cccb8..971e174ad21 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
@@ -434,6 +434,15 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener implement
         case DROP_FUNCTION:
           authzEvent = new DropFunctionEvent(preEventContext);
           break;
+        case CREATE_DATACONNECTOR:
+          authzEvent = new CreateDataConnectorEvent(preEventContext);
+          break;
+        case ALTER_DATACONNECTOR:
+          authzEvent = new AlterDataConnectorEvent(preEventContext);
+          break;
+        case DROP_DATACONNECTOR:
+          authzEvent = new DropDataConnectorEvent(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/AlterDataConnectorEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/AlterDataConnectorEvent.java
new file mode 100644
index 00000000000..f839645b0dd
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/AlterDataConnectorEvent.java
@@ -0,0 +1,100 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.DataConnector;
+import org.apache.hadoop.hive.metastore.events.PreAlterDataConnectorEvent;
+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;
+
+/*
+ Authorizable Event for HiveMetaStore operation AlterDataConnector
+ */
+
+public class AlterDataConnectorEvent extends HiveMetaStoreAuthorizableEvent {
+  private static final Logger LOG = LoggerFactory.getLogger(AlterDataConnectorEvent.class);
+
+  private String COMMAND_STR = "alter connector";
+
+  public AlterDataConnectorEvent(PreEventContext preEventContext) {
+    super(preEventContext);
+  }
+
+  @Override
+  public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.ALTERDATACONNECTOR, getInputHObjs(),
+            getOutputHObjs(), COMMAND_STR);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> AlterDataConnectorEvent.getInputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreAlterDataConnectorEvent event = (PreAlterDataConnectorEvent) preEventContext;
+    DataConnector connector = event.getOldDataConnector();
+
+    if (connector != null) {
+      ret.add(getHivePrivilegeObject(connector));
+      COMMAND_STR = buildCommandString(COMMAND_STR, connector);
+      LOG.debug("<== AlterDataConnectorEvent.getInputHObjs(): ret={}", ret);
+    }
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    LOG.debug("==> AlterDataConnectorEvent.getOutputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreAlterDataConnectorEvent event = (PreAlterDataConnectorEvent) preEventContext;
+    DataConnector connector = event.getNewDataConnector();
+
+    if (connector != null) {
+      ret.add(getHivePrivilegeObject(connector));
+      COMMAND_STR = buildCommandString(COMMAND_STR, connector);
+      LOG.debug("<== AlterDataConnectorEvent.getOutputHObjs(): ret={}", ret);
+    }
+
+    return ret;
+  }
+
+  private String buildCommandString(String cmdStr, DataConnector connector) {
+    String ret = cmdStr;
+
+    if (connector != null) {
+      String dcName = connector.getName();
+      ret = ret + (StringUtils.isNotEmpty(dcName) ? " " + dcName : "");
+    }
+
+    return ret;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateDataConnectorEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateDataConnectorEvent.java
new file mode 100644
index 00000000000..4298b3e1710
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/CreateDataConnectorEvent.java
@@ -0,0 +1,93 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.DataConnector;
+import org.apache.hadoop.hive.metastore.events.PreCreateDataConnectorEvent;
+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;
+
+/*
+ Authorizable Event for HiveMetaStore operation CreateDataConnector
+ */
+
+public class CreateDataConnectorEvent extends HiveMetaStoreAuthorizableEvent {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateDataConnectorEvent.class);
+
+  private String COMMAND_STR = "create connector";
+
+  public CreateDataConnectorEvent(PreEventContext preEventContext) {
+    super(preEventContext);
+  }
+
+  @Override
+  public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.CREATEDATACONNECTOR, getInputHObjs(),
+            getOutputHObjs(), COMMAND_STR);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    return Collections.emptyList();
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    LOG.debug("==> CreateDataConnectorEvent.getOutputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreCreateDataConnectorEvent event = (PreCreateDataConnectorEvent) preEventContext;
+    DataConnector connector = event.getDataConnector();
+
+    if (connector != null) {
+      ret.add(getHivePrivilegeObject(connector));
+
+      COMMAND_STR = buildCommandString(COMMAND_STR, connector);
+
+      LOG.debug("<== CreateDataConnectorEvent.getOutputHObjs(): ret={}", ret);
+    }
+
+    return ret;
+
+  }
+
+  private String buildCommandString(String cmdStr, DataConnector connector) {
+    String ret = cmdStr;
+
+    if (connector != null) {
+      String dcName = connector.getName();
+      ret = ret + (StringUtils.isNotEmpty(dcName) ? " " + dcName : "");
+    }
+
+    return ret;
+
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropDataConnectorEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropDataConnectorEvent.java
new file mode 100644
index 00000000000..189a86393f4
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/DropDataConnectorEvent.java
@@ -0,0 +1,88 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.DataConnector;
+import org.apache.hadoop.hive.metastore.events.PreDropDataConnectorEvent;
+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;
+
+/*
+ Authorizable Event for HiveMetaStore operation DropDataConnector
+ */
+
+public class DropDataConnectorEvent extends HiveMetaStoreAuthorizableEvent {
+  private static final Logger LOG = LoggerFactory.getLogger(DropDataConnectorEvent.class);
+
+  private String COMMAND_STR = "drop connector";
+
+  public DropDataConnectorEvent(PreEventContext preEventContext) {
+    super(preEventContext);
+  }
+
+  @Override
+  public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.DROPDATACONNECTOR, getInputHObjs(),
+            getOutputHObjs(), COMMAND_STR);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> DropDataConnectorEvent.getInputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreDropDataConnectorEvent event = (PreDropDataConnectorEvent) preEventContext;
+    DataConnector connector = event.getDataConnector();
+
+    ret.add(getHivePrivilegeObject(connector));
+
+    COMMAND_STR = buildCommandString(COMMAND_STR, connector);
+
+    LOG.debug("<== DropDataConnectorEvent.getInputHObjs(): ret={}", ret);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    return getInputHObjs();
+  }
+
+  private String buildCommandString(String cmdStr, DataConnector connector) {
+    String ret = cmdStr;
+
+    if (connector != null) {
+      String dcName = connector.getName();
+      ret = ret + (StringUtils.isNotEmpty(dcName) ? " " + dcName : "");
+    }
+
+    return ret;
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
index e16300267b1..bb191ba8605 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
@@ -42,6 +42,7 @@ import java.io.File;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /*
 Test whether HiveAuthorizer for MetaStore operation is trigger and HiveMetaStoreAuthzInfo is created by HiveMetaStoreAuthorizer
@@ -53,6 +54,7 @@ public class TestHiveMetaStoreAuthorizer {
   private static final String viewName         = "tmpview";
   private static final String roleName         = "tmpRole";
   private static final String catalogName      = "testCatalog";
+  private static final String dcName           = "testDC";
   private static final String unAuthorizedUser = "bob";
   private static final String authorizedUser   = "sam";
   private static final String superUser        = "hive";
@@ -89,6 +91,7 @@ public class TestHiveMetaStoreAuthorizer {
     // Create the 'hive' catalog with new warehouse directory
     HMSHandler.createDefaultCatalog(rawStore, new Warehouse(conf));
     try {
+      hmsHandler.drop_dataconnector(dcName, true ,true);
       hmsHandler.drop_table(dbName, tblName, true);
       hmsHandler.drop_database(dbName, true, false);
       hmsHandler.drop_catalog(new DropCatalogRequest(catalogName));
@@ -360,4 +363,52 @@ public class TestHiveMetaStoreAuthorizer {
       }
     }
   }
+
+  @Test
+  public void testR_CreateDataConnector_unAuthorizedUser() {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(unAuthorizedUser));
+    try {
+      DataConnector connector = new DataConnector(dcName, "mysql", "jdbc:mysql://localhost:3306/hive");
+      hmsHandler.create_dataconnector(connector);
+    } catch (Exception e) {
+      String err = e.getMessage();
+      String expected = "Operation type " + HiveOperationType.CREATEDATACONNECTOR+ " not allowed for user:" + unAuthorizedUser;
+      assertEquals(expected, err);
+    }
+  }
+
+  @Test
+  public void testS_CreateDataConnector_authorizedUser() {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(authorizedUser));
+    try {
+      DataConnector connector = new DataConnector(dcName, "mysql", "jdbc:mysql://localhost:3306/hive");
+      hmsHandler.create_dataconnector(connector);
+    } catch (Exception e) {
+      fail("testS_CreateDataConnector_authorizedUser() failed with " + e);
+    }
+  }
+
+  @Test
+  public void testT_AlterDataConnector_AuthorizedUser() {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(authorizedUser));
+    try {
+      DataConnector connector = new DataConnector(dcName, "mysql", "jdbc:mysql://localhost:3306/hive");
+      hmsHandler.create_dataconnector(connector);
+
+      DataConnector newConnector = new DataConnector(dcName, "mysql", "jdbc:mysql://localhost:3308/hive");
+      hmsHandler.alter_dataconnector(dcName, newConnector);
+    } catch (Exception e) {
+      fail("testT_AlterDataConnector_AuthorizedUser() failed with " + e);
+    }
+  }
+
+  @Test
+  public void testU_DropDataConnector_authorizedUser() {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(authorizedUser));
+    try {
+      hmsHandler.drop_dataconnector(dcName, true, true);
+    } catch (Exception e) {
+      fail("testU_DropDataConnector_authorizedUser() failed with " + e);
+    }
+  }
 }
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 1da146674ea..fa9109e3ece 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
@@ -1926,7 +1926,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
       } catch (NoSuchObjectException e) {
         // expected
       }
-
+      firePreEvent(new PreCreateDataConnectorEvent(connector, this));
       if (testTimeoutEnabled) {
         try {
           Thread.sleep(testTimeoutValue);
@@ -1994,7 +1994,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
         throw new MetaException("Could not alter dataconnector \"" + dcName +
             "\". Could not retrieve old definition.");
       }
-      // firePreEvent(new PreAlterDatabaseEvent(oldDC, newDC, this));
+      firePreEvent(new PreAlterDataConnectorEvent(oldDC, newDC, this));
 
       ms.openTransaction();
       ms.alterDataConnector(dcName, newDC);
@@ -2068,7 +2068,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
       // TODO find DBs with references to this connector
       // if any existing references and checkReferences=true, do not drop
 
-      // firePreEvent(new PreDropTableEvent(tbl, deleteData, this));
+      firePreEvent(new PreDropDataConnectorEvent(connector, this));
 
       if (!ms.dropDataConnector(dcName)) {
         throw new MetaException("Unable to drop dataconnector " + dcName);
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDataConnectorEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDataConnectorEvent.java
new file mode 100644
index 00000000000..f9ab28d74e8
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDataConnectorEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.DataConnector;
+
+public class PreAlterDataConnectorEvent extends PreEventContext {
+
+  private final DataConnector oldDC, newDC;
+
+  public PreAlterDataConnectorEvent(DataConnector oldDC, DataConnector newDC, IHMSHandler handler) {
+    super(PreEventType.ALTER_DATACONNECTOR, handler);
+    this.oldDC = oldDC;
+    this.newDC = newDC;
+  }
+
+  /**
+   * @return the old connector
+   */
+  public DataConnector getOldDataConnector() {
+    return oldDC;
+  }
+
+  /**
+   * @return the new connector
+   */
+  public DataConnector getNewDataConnector() {
+    return newDC;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDataConnectorEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDataConnectorEvent.java
new file mode 100644
index 00000000000..9d3dfebc846
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDataConnectorEvent.java
@@ -0,0 +1,44 @@
+/*
+ * 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.DataConnector;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropDataConnectorEvent extends PreEventContext {
+
+  private final DataConnector connector;
+
+  public PreDropDataConnectorEvent(DataConnector connector, IHMSHandler handler) {
+    super(PreEventType.DROP_DATACONNECTOR, handler);
+    this.connector = connector;
+  }
+
+  /**
+   * @return the connector
+   */
+  public DataConnector getDataConnector() {
+    return connector;
+  }
+}