You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sa...@apache.org on 2019/04/27 17:24:54 UTC

[atlas] branch master updated: ATLAS-3157: Add Integration tests for Hive metastore hook

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c37656c  ATLAS-3157: Add Integration tests for Hive metastore hook
c37656c is described below

commit c37656cc64b1c41c8188ec92c24907e84ae05a06
Author: Sarath Subramanian <ss...@hortonworks.com>
AuthorDate: Sat Apr 27 10:24:33 2019 -0700

    ATLAS-3157: Add Integration tests for Hive metastore hook
---
 .../java/org/apache/atlas/hive/HiveITBase.java     |  63 +++-
 .../org/apache/atlas/hive/hook/HiveHookIT.java     |  72 ++--
 .../atlas/hive/hook/HiveMetastoreHookIT.java       | 364 +++++++++++++++++++++
 .../hive-bridge/src/test/resources/hive-site.xml   |   5 +
 4 files changed, 453 insertions(+), 51 deletions(-)

diff --git a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/HiveITBase.java b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/HiveITBase.java
index 6793d8a..cbee7bf 100644
--- a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/HiveITBase.java
+++ b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/HiveITBase.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasClientV2;
+import org.apache.atlas.AtlasServiceException;
 import org.apache.atlas.hive.bridge.ColumnLineageUtils;
 import org.apache.atlas.hive.bridge.HiveMetaStoreBridge;
 import org.apache.atlas.hive.hook.HiveHookIT;
@@ -67,7 +68,10 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
 
+import static com.sun.jersey.api.client.ClientResponse.Status.NOT_FOUND;
 import static org.apache.atlas.hive.bridge.HiveMetaStoreBridge.HDFS_PATH;
+import static org.apache.atlas.hive.hook.events.BaseHiveEvent.ATTRIBUTE_QUALIFIED_NAME;
+import static org.apache.atlas.hive.model.HiveDataTypes.HIVE_DB;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertTrue;
@@ -103,35 +107,46 @@ public class HiveITBase {
         //Set-up hive session
         conf = new HiveConf();
         conf.setClassLoader(Thread.currentThread().getContextClassLoader());
+        conf.set("hive.metastore.event.listeners", "");
+
+        // 'driver' using this configuration will be used for tests in HiveHookIT
+        //  HiveHookIT will use this driver to test post-execution hooks in HiveServer2.
+        //  initialize 'driver' with HMS hook disabled.
         driver = new Driver(conf);
-        ss = new SessionState(conf);
-        ss = SessionState.start(ss);
+        ss     = new SessionState(conf);
+        ss     = SessionState.start(ss);
 
         SessionState.setCurrentSessionState(ss);
 
         Configuration configuration = ApplicationProperties.get();
 
         String[] atlasEndPoint = configuration.getStringArray(HiveMetaStoreBridge.ATLAS_ENDPOINT);
+
         if (atlasEndPoint == null || atlasEndPoint.length == 0) {
             atlasEndPoint = new String[] { DGI_URL };
         }
 
         if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
             atlasClientV2 = new AtlasClientV2(atlasEndPoint, new String[]{"admin", "admin"});
-            atlasClient = new AtlasClient(atlasEndPoint, new String[]{"admin", "admin"});
+            atlasClient   = new AtlasClient(atlasEndPoint, new String[]{"admin", "admin"});
         } else {
             atlasClientV2 = new AtlasClientV2(atlasEndPoint);
-            atlasClient = new AtlasClient(atlasEndPoint);
-
+            atlasClient   = new AtlasClient(atlasEndPoint);
         }
 
         hiveMetaStoreBridge = new HiveMetaStoreBridge(configuration, conf, atlasClientV2);
 
         HiveConf conf = new HiveConf();
+
         conf.set("hive.exec.post.hooks", "");
+
         SessionState ss = new SessionState(conf);
         ss = SessionState.start(ss);
         SessionState.setCurrentSessionState(ss);
+
+        // 'driverWithoutContext' using this configuration will be used for tests in HiveMetastoreHookIT
+        //  HiveMetastoreHookIT will use this driver to test event listeners in HiveMetastore.
+        //  initialize 'driverWithoutContext' with HiveServer2 post execution hook disabled.
         driverWithoutContext = new Driver(conf);
     }
 
@@ -149,8 +164,11 @@ public class HiveITBase {
 
     protected void runCommandWithDelay(Driver driver, String cmd, int sleepMs) throws Exception {
         LOG.debug("Running command '{}'", cmd);
+
         CommandProcessorResponse response = driver.run(cmd);
+
         assertEquals(response.getResponseCode(), 0);
+
         if (sleepMs != 0) {
             Thread.sleep(sleepMs);
         }
@@ -182,11 +200,15 @@ public class HiveITBase {
     }
 
     protected String random() {
-        return RandomStringUtils.randomAlphanumeric(10);
+        return RandomStringUtils.randomAlphanumeric(10).toLowerCase();
     }
 
     protected String tableName() {
-        return "table" + random();
+        return "table_" + random();
+    }
+
+    protected String dbName() {
+        return "db_" + random();
     }
 
     protected String assertTableIsRegistered(String dbName, String tableName) throws Exception {
@@ -336,12 +358,35 @@ public class HiveITBase {
     }
 
     protected String assertDatabaseIsRegistered(String dbName, AssertPredicate assertPredicate) throws Exception {
+        LOG.debug("Searching for database: {}", dbName);
+
+        String dbQualifiedName = HiveMetaStoreBridge.getDBQualifiedName(CLUSTER_NAME, dbName);
+
+        return assertEntityIsRegistered(HIVE_DB.getName(), REFERENCEABLE_ATTRIBUTE_NAME, dbQualifiedName, assertPredicate);
+    }
+
+    public void assertDatabaseIsNotRegistered(String dbName) throws Exception {
         LOG.debug("Searching for database {}", dbName);
         String dbQualifiedName = HiveMetaStoreBridge.getDBQualifiedName(CLUSTER_NAME, dbName);
-        return assertEntityIsRegistered(HiveDataTypes.HIVE_DB.getName(), REFERENCEABLE_ATTRIBUTE_NAME,
-                dbQualifiedName, assertPredicate);
+        assertEntityIsNotRegistered(HIVE_DB.getName(), ATTRIBUTE_QUALIFIED_NAME, dbQualifiedName);
     }
 
+    protected void assertEntityIsNotRegistered(final String typeName, final String property, final String value) throws Exception {
+        // wait for sufficient time before checking if entity is not available.
+        long waitTime = 2000;
+        LOG.debug("Waiting for {} msecs, before asserting entity is not registered.", waitTime);
+        Thread.sleep(waitTime);
+
+        try {
+            atlasClientV2.getEntityByAttribute(typeName, Collections.singletonMap(property, value));
+
+            fail(String.format("Entity was not supposed to exist for typeName = %s, attributeName = %s, attributeValue = %s", typeName, property, value));
+        } catch (AtlasServiceException e) {
+            if (e.getStatus() == NOT_FOUND) {
+                return;
+            }
+        }
+    }
 
     protected AtlasEntity getAtlasEntityByType(String type, String id) throws Exception {
         AtlasEntity atlasEntity = null;
diff --git a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveHookIT.java b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveHookIT.java
index 0a2020a..24de30f 100755
--- a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveHookIT.java
+++ b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveHookIT.java
@@ -28,12 +28,8 @@ import org.apache.atlas.hive.HiveITBase;
 import org.apache.atlas.hive.bridge.HiveMetaStoreBridge;
 import org.apache.atlas.hive.hook.events.BaseHiveEvent;
 import org.apache.atlas.hive.model.HiveDataTypes;
-import org.apache.atlas.model.instance.AtlasClassification;
-import org.apache.atlas.model.instance.AtlasEntity;
-import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.*;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
-import org.apache.atlas.model.instance.AtlasObjectId;
-import org.apache.atlas.model.instance.AtlasStruct;
 import org.apache.atlas.model.lineage.AtlasLineageInfo;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasTypesDef;
@@ -41,18 +37,22 @@ import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.hooks.Entity;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import java.text.ParseException;
@@ -60,17 +60,32 @@ import java.util.*;
 
 import static org.apache.atlas.AtlasClient.NAME;
 import static org.apache.atlas.hive.hook.events.BaseHiveEvent.*;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNotEquals;
-import static org.testng.Assert.assertNotNull;
-import static org.testng.Assert.assertTrue;
-import static org.testng.Assert.fail;
+import static org.testng.Assert.*;
 
 public class HiveHookIT extends HiveITBase {
     private static final Logger LOG = LoggerFactory.getLogger(HiveHookIT.class);
 
     private static final String PART_FILE  = "2015-01-01";
 
+    private Driver driverWithNoHook;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        //  initialize 'driverWithNoHook' with HiveServer2 hook and HiveMetastore hook disabled
+        HiveConf conf = new HiveConf();
+        conf.set("hive.exec.post.hooks", "");
+        conf.set("hive.metastore.event.listeners", "");
+
+        SessionState ss = new SessionState(conf);
+        ss = SessionState.start(ss);
+        SessionState.setCurrentSessionState(ss);
+
+        // Initialize 'driverWithNoHook'  with HS2 hook disabled and HMS hook disabled.
+        driverWithNoHook = new Driver(conf);
+
+        super.setUp();
+    }
+
     @Test
     public void testCreateDatabase() throws Exception {
         String dbName = "db" + random();
@@ -87,7 +102,7 @@ public class HiveHookIT extends HiveITBase {
 
         //There should be just one entity per dbname
         runCommand("drop database " + dbName);
-        assertDBIsNotRegistered(dbName);
+        assertDatabaseIsNotRegistered(dbName);
 
         runCommand("create database " + dbName);
         dbId = assertDatabaseIsRegistered(dbName);
@@ -1557,7 +1572,7 @@ public class HiveHookIT extends HiveITBase {
         String tableName     = tableName();
         String createCommand = "create table " + tableName + " (id int, name string)";
 
-        driverWithoutContext.run(createCommand);
+        driverWithNoHook.run(createCommand);
 
         assertTableIsNotRegistered(DEFAULT_DB, tableName);
 
@@ -1802,7 +1817,7 @@ public class HiveHookIT extends HiveITBase {
             assertTableIsNotRegistered(dbName, tableNames[i]);
         }
 
-        assertDBIsNotRegistered(dbName);
+        assertDatabaseIsNotRegistered(dbName);
     }
 
     @Test
@@ -1849,14 +1864,14 @@ public class HiveHookIT extends HiveITBase {
         //Test Deletion of a non existing DB
         String dbName = "nonexistingdb";
 
-        assertDBIsNotRegistered(dbName);
+        assertDatabaseIsNotRegistered(dbName);
 
         String query = String.format("drop database if exists %s cascade", dbName);
 
         runCommand(query);
 
         //Should have no effect
-        assertDBIsNotRegistered(dbName);
+        assertDatabaseIsNotRegistered(dbName);
     }
 
     @Test
@@ -2209,33 +2224,10 @@ public class HiveHookIT extends HiveITBase {
         assertEntityIsNotRegistered(HiveDataTypes.HIVE_TABLE.getName(), ATTRIBUTE_QUALIFIED_NAME, tableQualifiedName);
     }
 
-    private void assertDBIsNotRegistered(String dbName) throws Exception {
-        LOG.debug("Searching for database {}", dbName);
-        String dbQualifiedName = HiveMetaStoreBridge.getDBQualifiedName(CLUSTER_NAME, dbName);
-        assertEntityIsNotRegistered(HiveDataTypes.HIVE_DB.getName(), ATTRIBUTE_QUALIFIED_NAME, dbQualifiedName);
-    }
-
     private String assertTableIsRegistered(String dbName, String tableName, AssertPredicate assertPredicate) throws Exception {
         return assertTableIsRegistered(dbName, tableName, assertPredicate, false);
     }
 
-    private void assertEntityIsNotRegistered(final String typeName, final String property, final String value) throws Exception {
-        waitFor(80000, new Predicate() {
-            @Override
-            public void evaluate() throws Exception {
-                try {
-                    atlasClientV2.getEntityByAttribute(typeName, Collections.singletonMap(property, value));
-                } catch (AtlasServiceException e) {
-                    if (e.getStatus() == ClientResponse.Status.NOT_FOUND) {
-                        return;
-                    }
-                }
-                fail(String.format("Entity was not supposed to exist for typeName = %s, attributeName = %s, "
-                        + "attributeValue = %s", typeName, property, value));
-            }
-        });
-    }
-
     @Test
     public void testLineage() throws Exception {
         String table1 = createTable(false);
@@ -2267,10 +2259,6 @@ public class HiveHookIT extends HiveITBase {
         runCommand("show transactions");
     }
 
-    private String dbName() {
-        return "db" + random();
-    }
-
     private String createDatabase() throws Exception {
         String dbName = dbName();
 
diff --git a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveMetastoreHookIT.java b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveMetastoreHookIT.java
new file mode 100755
index 0000000..fdd65eb
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveMetastoreHookIT.java
@@ -0,0 +1,364 @@
+/**
+ * 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.atlas.hive.hook;
+
+import org.apache.atlas.AtlasServiceException;
+import org.apache.atlas.hive.HiveITBase;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.Test;
+
+import java.util.Collection;
+import java.util.Map;
+
+import static org.apache.atlas.model.instance.AtlasEntity.Status.ACTIVE;
+import static org.apache.atlas.model.instance.AtlasEntity.Status.DELETED;
+import static org.testng.AssertJUnit.*;
+
+public class HiveMetastoreHookIT extends HiveITBase {
+    private static final Logger LOG = LoggerFactory.getLogger(HiveMetastoreHookIT.class);
+
+    @Test (priority = 1)
+    public void testCreateDatabase() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+
+        AtlasEntity dbEntity = getAtlasEntity(dbId);
+        assertNotNull(dbEntity);
+    }
+
+    @Test (priority = 2)
+    public void testAlterDatabase() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+
+        AtlasEntity dbEntity = getAtlasEntity(dbId);
+        assertNotNull(dbEntity);
+
+        // SET DBPROPERTIES
+        query = "ALTER DATABASE " + dbName + " SET DBPROPERTIES (\"prop1\"=\"val1\", \"prop2\"=\"val2\")";
+        runCommandWithDelay(query);
+
+        dbEntity = getAtlasEntity(dbId);
+        Map parameters = (Map) dbEntity.getAttribute("parameters");
+
+        assertNotNull(parameters);
+        assertEquals(2, parameters.size());
+
+        // SET OWNER to 'hive'
+        query = "ALTER DATABASE " + dbName + " SET OWNER USER hive";
+        runCommandWithDelay(query);
+
+        dbEntity = getAtlasEntity(dbId);
+
+        assertEquals(dbEntity.getAttribute("owner"), "hive");
+        assertEquals(dbEntity.getAttribute("ownerType"), "USER");
+
+        // SET LOCATION
+        String hdfsPath = "hdfs://localhost:8020/warehouse/tablespace/managed/dwx/new_db.db";
+
+        query = String.format("ALTER DATABASE %s SET LOCATION \"%s\"", dbName, hdfsPath);
+        runCommandWithDelay(query);
+
+        dbEntity = getAtlasEntity(dbId);
+
+        String location = (String) dbEntity.getAttribute("location");
+        assertEquals(location, hdfsPath);
+    }
+
+    @Test (priority = 3)
+    public void testDropDatabase() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+
+        AtlasEntity dbEntity = getAtlasEntity(dbId);
+        assertNotNull(dbEntity);
+
+        query = "DROP DATABASE " + dbName;
+        runCommand(query);
+        assertDatabaseIsNotRegistered(dbName);
+
+        dbEntity = getAtlasEntity(dbId);
+        assertEquals(dbEntity.getStatus(), DELETED);
+    }
+
+    @Test (priority = 4)
+    public void testDropDatabaseWithTables() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String table1 = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + table1 + " (name string, age int, dob date)");
+        String table1Id = assertTableIsRegistered(dbName, table1);
+        assertEquals(getAtlasEntity(table1Id).getStatus(), ACTIVE);
+
+        String table2 = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + table2 + " (name string, age int, dob date)");
+        String table2Id = assertTableIsRegistered(dbName, table2);
+        assertEquals(getAtlasEntity(table2Id).getStatus(), ACTIVE);
+
+        query = "DROP DATABASE " + dbName + " CASCADE";
+        runCommand(query);
+        assertDatabaseIsNotRegistered(dbName);
+
+        assertEquals(getAtlasEntity(dbId).getStatus(), DELETED);
+        assertEquals(getAtlasEntity(table1Id).getStatus(), DELETED);
+        assertEquals(getAtlasEntity(table2Id).getStatus(), DELETED);
+    }
+
+    @Test (priority = 5)
+    public void testCreateTable() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+    }
+
+    @Test (priority = 6)
+    public void testCreateView() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        String viewName = tableName();
+        runCommand("CREATE VIEW " + dbName + "." + viewName + " AS SELECT * FROM " + dbName + "." + tableName);
+        tblId = assertTableIsRegistered(dbName, viewName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+    }
+
+    @Test (priority = 7)
+    public void testAlterTableProperties() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        // SET TBLPROPERTIES
+        query = "ALTER TABLE " + dbName + "." + tableName + " SET TBLPROPERTIES (\"prop1\"=\"val1\", \"prop2\"=\"val2\", \"prop3\"=\"val3\")";
+        runCommandWithDelay(query);
+
+        query = "ALTER TABLE " + dbName + "." + tableName + " SET TBLPROPERTIES (\"comment\" = \"sample comment\")";
+        runCommandWithDelay(query);
+
+        // SET SERDE
+        query = "ALTER TABLE " + dbName + "." + tableName + " SET SERDE \"org.apache.hadoop.hive.ql.io.orc.OrcSerde\" WITH SERDEPROPERTIES (\"prop1\"=\"val1\", \"prop2\"=\"val2\")";
+        runCommandWithDelay(query);
+
+        // SET SERDEPROPERTIES
+        query = "ALTER TABLE " + dbName + "." + tableName + " SET SERDEPROPERTIES (\"prop1\"=\"val1\", \"prop2\"=\"val2\")";
+        runCommandWithDelay(query);
+
+        AtlasEntity         tableEntity     = getAtlasEntity(tblId);
+        Map<String, Object> tableParameters = (Map) tableEntity.getAttribute("parameters");
+
+        assertEquals(tableParameters.get("comment"), "sample comment");
+        assertEquals(tableParameters.get("prop1"), "val1");
+        assertEquals(tableParameters.get("prop2"), "val2");
+        assertEquals(tableParameters.get("prop3"), "val3");
+
+        AtlasEntity sdEntity   = getAtlasEntity((String) ((Map) tableEntity.getAttribute("sd")).get("guid"));
+        Map         serdeInfo  = (Map) sdEntity.getAttribute("serdeInfo");
+        Map         serdeAttrs = (Map) serdeInfo.get("attributes");
+
+        assertEquals(serdeAttrs.get("serializationLib"), "org.apache.hadoop.hive.ql.io.orc.OrcSerde");
+        assertEquals(((Map) serdeAttrs.get("parameters")).get("prop1"), "val1");
+        assertEquals(((Map) serdeAttrs.get("parameters")).get("prop2"), "val2");
+    }
+
+    @Test (priority = 8)
+    public void testAlterTableRenameTableName() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        // RENAME TABLE NAME
+        String newTableName = tableName + "_new";
+        query = "ALTER TABLE " + dbName + "." + tableName + " RENAME TO " + dbName + "." + newTableName;
+        runCommandWithDelay(query);
+
+        AtlasEntityWithExtInfo tableEntityWithExtInfo = getAtlasEntityWithExtInfo(tblId);
+        AtlasEntity            tableEntity            = tableEntityWithExtInfo.getEntity();
+
+        // validate table rename in table entity
+        assertEquals(newTableName, tableEntity.getAttribute("name"));
+        assertTrue(((String) tableEntity.getAttribute("qualifiedName")).contains(newTableName));
+
+        // validate table rename in column and sd entity
+        for (AtlasEntity referredEntity : tableEntityWithExtInfo.getReferredEntities().values()) {
+            assertTrue(((String) referredEntity.getAttribute("qualifiedName")).contains(newTableName));
+        }
+    }
+
+    @Test (priority = 9)
+    public void testAlterTableRenameColumnName() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (col1 int, col2 int, col3 int)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        AtlasEntityWithExtInfo tableEntityWithExtInfo = getAtlasEntityWithExtInfo(tblId);
+
+        assertEquals(tableEntityWithExtInfo.getEntity().getStatus(), ACTIVE);
+
+        String col1Id = getColumnId(tableEntityWithExtInfo, "col1");
+        String col2Id = getColumnId(tableEntityWithExtInfo, "col2");
+
+        // RENAME COLUMN NAME
+        query = "ALTER TABLE " + dbName + "." + tableName + " CHANGE col1 col11 int";
+        runCommandWithDelay(query);
+
+        AtlasEntity col1Entity = getAtlasEntity(col1Id);
+        assertEquals(col1Entity.getAttribute("name"), "col11");
+        assertTrue(((String) col1Entity.getAttribute("qualifiedName")).contains("col11"));
+
+        // CHANGE COLUMN NAME and DATATYPE
+        query = "ALTER TABLE " + dbName + "." + tableName + " CHANGE col2 col22 string";
+        runCommandWithDelay(query);
+
+        AtlasEntity col2Entity = getAtlasEntity(col2Id);
+        assertEquals(col2Entity.getAttribute("name"), "col22");
+        assertEquals(col2Entity.getAttribute("type"), "string");
+    }
+
+    @Test (priority = 10)
+    public void testDropTable() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        query = "DROP TABLE " + dbName + "." + tableName;
+        runCommandWithDelay(query);
+
+        assertEquals(getAtlasEntity(tblId).getStatus(), DELETED);
+    }
+
+    @Test (priority = 11)
+    public void testDropView() throws Exception {
+        String dbName = dbName();
+        String query  = "CREATE DATABASE " + dbName;
+
+        runCommand(query);
+        String dbId = assertDatabaseIsRegistered(dbName);
+        assertEquals(getAtlasEntity(dbId).getStatus(), ACTIVE);
+
+        String tableName = tableName();
+        runCommand("CREATE TABLE " + dbName + "." + tableName + " (name string, age int, dob date)");
+        String tblId = assertTableIsRegistered(dbName, tableName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        String viewName = tableName();
+        runCommand("CREATE VIEW " + dbName + "." + viewName + " AS SELECT * FROM " + dbName + "." + tableName);
+        tblId = assertTableIsRegistered(dbName, viewName);
+        assertEquals(getAtlasEntity(tblId).getStatus(), ACTIVE);
+
+        query = "DROP VIEW " + dbName + "." + viewName;
+        runCommandWithDelay(query);
+
+        assertEquals(getAtlasEntity(tblId).getStatus(), DELETED);
+    }
+
+    private String getColumnId(AtlasEntityWithExtInfo entityWithExtInfo, String columnName) {
+        String ret = null;
+
+        for (AtlasEntity entity : entityWithExtInfo.getReferredEntities().values()) {
+
+            if (entity.getTypeName().equals("hive_column") && entity.getAttribute("name").equals(columnName)) {
+                ret = entity.getGuid();
+                break;
+            }
+        }
+
+        return ret;
+    }
+
+    private AtlasEntity getAtlasEntity(String guid) throws AtlasServiceException {
+        return atlasClientV2.getEntityByGuid(guid).getEntity();
+    }
+
+    private AtlasEntityWithExtInfo getAtlasEntityWithExtInfo(String guid) throws AtlasServiceException {
+        return atlasClientV2.getEntityByGuid(guid);
+    }
+
+    protected void runCommand(String cmd) throws Exception {
+        runCommandWithDelay(driverWithoutContext, cmd, 0);
+    }
+
+    protected void runCommandWithDelay(String cmd) throws Exception {
+        int delayTimeInMs = 5000;
+        runCommandWithDelay(driverWithoutContext, cmd, delayTimeInMs);
+    }
+}
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/hive-site.xml b/addons/hive-bridge/src/test/resources/hive-site.xml
index 4605ae3..f9ec577 100644
--- a/addons/hive-bridge/src/test/resources/hive-site.xml
+++ b/addons/hive-bridge/src/test/resources/hive-site.xml
@@ -37,6 +37,11 @@
     </property>
 
     <property>
+        <name>hive.metastore.event.listeners</name>
+        <value>org.apache.atlas.hive.hook.HiveMetastoreHookImpl</value>
+    </property>
+
+    <property>
         <name>hive.support.concurrency</name>
         <value>false</value>
     </property>