You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by am...@apache.org on 2021/04/12 16:46:09 UTC

[atlas] branch master updated: ATLAS-4204: Configure HS2 hook to send lineage-only messages.

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

amestry 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 2588540  ATLAS-4204: Configure HS2 hook to send lineage-only messages.
2588540 is described below

commit 2588540ebef16c6fa4890bb15c7841042b13ab3c
Author: Ashutosh Mestry <am...@cloudera.com>
AuthorDate: Mon Apr 12 09:19:42 2021 -0700

    ATLAS-4204: Configure HS2 hook to send lineage-only messages.
---
 .../atlas/hive/hook/AtlasHiveHookContext.java      |   4 +
 .../java/org/apache/atlas/hive/hook/HiveHook.java  |   8 +-
 .../atlas/hive/hook/events/BaseHiveEvent.java      |   2 +
 .../atlas/hive/hook/utils/ActiveEntityFilter.java  |  49 ++++
 .../apache/atlas/hive/hook/utils/EntityFilter.java |  26 ++
 .../atlas/hive/hook/utils/HiveDDLEntityFilter.java | 258 +++++++++++++++++++
 .../atlas/hive/hook/utils/PassthroughFilter.java   |  29 +++
 .../hive/hook/utils/ActiveEntityFilterTest.java    | 204 +++++++++++++++
 .../test/resources/atlas-application.properties    |   1 +
 .../src/test/resources/json/hs2-alter-view-v2.json |  35 +++
 .../src/test/resources/json/hs2-alter-view.json    | 268 +++++++++++++++++++
 .../src/test/resources/json/hs2-create-db-v2.json  |  30 +++
 .../src/test/resources/json/hs2-create-db.json     |  73 ++++++
 .../test/resources/json/hs2-create-process-v2.json | 144 +++++++++++
 .../test/resources/json/hs2-create-process.json    | 283 +++++++++++++++++++++
 .../test/resources/json/hs2-create-table-v2.json   | 102 ++++++++
 .../src/test/resources/json/hs2-create-table.json  | 255 +++++++++++++++++++
 .../src/test/resources/json/hs2-drop-db-v2.json    |   4 +
 .../src/test/resources/json/hs2-drop-db.json       |  10 +
 .../src/test/resources/json/hs2-drop-table-v2.json |   1 +
 .../src/test/resources/json/hs2-drop-table.json    |  50 ++++
 .../test/resources/json/hs2-table-rename-v2.json   |  35 +++
 .../src/test/resources/json/hs2-table-rename.json  | 276 ++++++++++++++++++++
 .../main/java/org/apache/atlas/type/AtlasType.java |   3 +
 .../java/org/apache/atlas/utils/AtlasJson.java     |  14 +
 25 files changed, 2162 insertions(+), 2 deletions(-)

diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/AtlasHiveHookContext.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/AtlasHiveHookContext.java
index 1286471..ab8e183 100644
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/AtlasHiveHookContext.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/AtlasHiveHookContext.java
@@ -246,6 +246,10 @@ public class AtlasHiveHookContext {
     }
 
     private void init() {
+        if (hiveOperation == null) {
+            return;
+        }
+
         String operation = hiveOperation.getOperationName();
 
         if (knownObjects == null || !isCreateAlterOperation(operation)) {
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/HiveHook.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/HiveHook.java
index 79e87c7..cf918ef 100644
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/HiveHook.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/HiveHook.java
@@ -19,6 +19,7 @@
 package org.apache.atlas.hive.hook;
 
 import org.apache.atlas.hive.hook.events.*;
+import org.apache.atlas.hive.hook.utils.ActiveEntityFilter;
 import org.apache.atlas.hook.AtlasHook;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.utils.LruCache;
@@ -68,6 +69,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
     public static final String HOOK_HIVE_TABLE_IGNORE_PATTERN                            = CONF_PREFIX + "hive_table.ignore.pattern";
     public static final String HOOK_HIVE_TABLE_PRUNE_PATTERN                             = CONF_PREFIX + "hive_table.prune.pattern";
     public static final String HOOK_HIVE_TABLE_CACHE_SIZE                                = CONF_PREFIX + "hive_table.cache.size";
+    public static final String HOOK_HIVE_IGNORE_DDL_OPERATIONS                           = CONF_PREFIX + "hs2.ignore.ddl.operations";
     public static final String DEFAULT_HOST_NAME = "localhost";
 
     private static final Map<String, HiveOperation> OPERATION_MAP = new HashMap<>();
@@ -88,7 +90,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
     private static final List                          ignoreDummyTableName;
     private static final String                        ignoreValuesTmpTableNamePrefix;
     private static final boolean                       hiveProcessPopulateDeprecatedAttributes;
-    private static HiveHookObjectNamesCache knownObjects = null;
+    private static HiveHookObjectNamesCache            knownObjects = null;
     private static String hostName;
 
     static {
@@ -158,6 +160,8 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
             LOG.warn("No hostname found. Setting the hostname to default value {}", DEFAULT_HOST_NAME, e);
             hostName = DEFAULT_HOST_NAME;
         }
+
+        ActiveEntityFilter.init(atlasProperties);
     }
 
 
@@ -249,7 +253,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
             if (event != null) {
                 final UserGroupInformation ugi = hookContext.getUgi() == null ? Utils.getUGI() : hookContext.getUgi();
 
-                super.notifyEntities(event.getNotificationMessages(), ugi);
+                super.notifyEntities(ActiveEntityFilter.apply(event.getNotificationMessages()), ugi);
             }
         } catch (Throwable t) {
             LOG.error("HiveHook.run(): failed to process operation {}", hookContext.getOperationName(), t);
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/events/BaseHiveEvent.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/events/BaseHiveEvent.java
index 7c269ce..21c0ae3 100644
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/events/BaseHiveEvent.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/events/BaseHiveEvent.java
@@ -136,6 +136,8 @@ public abstract class BaseHiveEvent {
     public static final String ATTRIBUTE_EXEC_TIME                 = "execTime";
     public static final String ATTRIBUTE_DDL_QUERIES               = "ddlQueries";
     public static final String ATTRIBUTE_SERVICE_TYPE              = "serviceType";
+    public static final String ATTRIBUTE_GUID                      = "guid";
+    public static final String ATTRIBUTE_UNIQUE_ATTRIBUTES         = "uniqueAttributes";
     public static final String HBASE_STORAGE_HANDLER_CLASS         = "org.apache.hadoop.hive.hbase.HBaseStorageHandler";
     public static final String HBASE_DEFAULT_NAMESPACE             = "default";
     public static final String HBASE_NAMESPACE_TABLE_DELIMITER     = ":";
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilter.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilter.java
new file mode 100644
index 0000000..737c637
--- /dev/null
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilter.java
@@ -0,0 +1,49 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.utils;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.atlas.model.notification.HookNotification;
+import org.apache.commons.configuration.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import static org.apache.atlas.hive.hook.HiveHook.HOOK_HIVE_IGNORE_DDL_OPERATIONS;
+
+public class ActiveEntityFilter {
+    private static final Logger LOG = LoggerFactory.getLogger(ActiveEntityFilter.class);
+
+    private static EntityFilter entityFilter;
+
+    public static void init(Configuration configuration) {
+        boolean skipDdlOperations = configuration.getBoolean(HOOK_HIVE_IGNORE_DDL_OPERATIONS, false);
+        init(skipDdlOperations);
+        LOG.info("atlas.hook.hive.ignore.ddl.operations={} - {}", skipDdlOperations, entityFilter.getClass().getSimpleName());
+    }
+
+    @VisibleForTesting
+    static void init(boolean lineageOnlyFilter) {
+        entityFilter = lineageOnlyFilter ? new HiveDDLEntityFilter() : new PassthroughFilter();
+    }
+
+    public static List<HookNotification> apply(List<HookNotification> incoming) {
+        return entityFilter.apply(incoming);
+    }
+}
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/EntityFilter.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/EntityFilter.java
new file mode 100644
index 0000000..f02ee52
--- /dev/null
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/EntityFilter.java
@@ -0,0 +1,26 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.utils;
+
+import org.apache.atlas.model.notification.HookNotification;
+
+import java.util.List;
+
+public interface EntityFilter {
+    List<HookNotification> apply(List<HookNotification> incoming);
+}
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/HiveDDLEntityFilter.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/HiveDDLEntityFilter.java
new file mode 100644
index 0000000..2b39e81
--- /dev/null
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/HiveDDLEntityFilter.java
@@ -0,0 +1,258 @@
+/**
+ * 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.utils;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.atlas.hive.hook.events.BaseHiveEvent;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.instance.AtlasRelatedObjectId;
+import org.apache.atlas.model.notification.HookNotification;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class HiveDDLEntityFilter implements EntityFilter {
+    private static final Set<String> typesToRetain = new HashSet<String>() {{
+        add(BaseHiveEvent.HIVE_TYPE_PROCESS);
+        add(BaseHiveEvent.HIVE_TYPE_PROCESS_EXECUTION);
+        add(BaseHiveEvent.HIVE_TYPE_COLUMN_LINEAGE);
+        add(BaseHiveEvent.HIVE_DB_DDL);
+        add(BaseHiveEvent.HIVE_TABLE_DDL);
+    }};
+
+    public List<HookNotification> apply(List<HookNotification> incoming) {
+        if (CollectionUtils.isEmpty(incoming)) {
+            return incoming;
+        }
+
+        List<HookNotification> ret = new ArrayList<>();
+        for (HookNotification notification : incoming) {
+            HookNotification filteredNotification = apply(notification);
+            if (filteredNotification == null) {
+                continue;
+            }
+
+            ret.add(filteredNotification);
+        }
+
+        return ret;
+    }
+
+    @VisibleForTesting
+    AtlasEntity.AtlasEntityWithExtInfo apply(AtlasEntity.AtlasEntityWithExtInfo incoming) {
+        AtlasEntity.AtlasEntityWithExtInfo ret = new AtlasEntity.AtlasEntityWithExtInfo();
+
+        AtlasEntity entity = filter(incoming.getEntity());
+        if (entity == null) {
+            return null;
+        }
+
+        ret.setEntity(entity);
+
+        Map<String, AtlasEntity> refEntities = filter(incoming.getReferredEntities());
+        if (!MapUtils.isEmpty(refEntities)) {
+            ret.setReferredEntities(refEntities);
+        }
+
+        return ret;
+    }
+
+    @VisibleForTesting
+    public AtlasEntity.AtlasEntitiesWithExtInfo apply(AtlasEntity.AtlasEntitiesWithExtInfo incoming) {
+        if (incoming == null) {
+            return incoming;
+        }
+
+        AtlasEntity.AtlasEntitiesWithExtInfo ret = new AtlasEntity.AtlasEntitiesWithExtInfo();
+
+        filterEntities(incoming, ret);
+        filterReferences(incoming, ret);
+
+        return ret;
+    }
+
+    @VisibleForTesting
+    List<AtlasObjectId> applyForObjectIds(List<AtlasObjectId> incoming) {
+        if (incoming == null || CollectionUtils.isEmpty(incoming)) {
+            return null;
+        }
+
+        List<AtlasObjectId> ret = new ArrayList<>();
+        for (AtlasObjectId o : incoming) {
+            if (filterObjectId(o) != null) {
+                ret.add(o);
+            }
+        }
+
+        return ret;
+    }
+
+    private AtlasObjectId filterObjectId(AtlasObjectId o) {
+        if (o != null && typesToRetain.contains(o.getTypeName())) {
+            return o;
+        }
+
+        return null;
+    }
+
+    private static void filterEntities(AtlasEntity.AtlasEntitiesWithExtInfo incoming, AtlasEntity.AtlasEntitiesWithExtInfo ret) {
+        ret.setEntities(filter(incoming.getEntities()));
+    }
+
+    private static void filterReferences(AtlasEntity.AtlasEntitiesWithExtInfo incoming, AtlasEntity.AtlasEntitiesWithExtInfo ret) {
+        ret.setReferredEntities(filter(incoming.getReferredEntities()));
+    }
+
+    private static Map<String, AtlasEntity> filter(Map<String, AtlasEntity> incoming) {
+        if (incoming == null || MapUtils.isEmpty(incoming)) {
+            return null;
+        }
+
+        return incoming.values()
+                .stream()
+                .filter(x -> typesToRetain.contains(x.getTypeName()))
+                .collect(Collectors.toMap(AtlasEntity::getGuid, Function.identity()));
+    }
+
+    private static List<AtlasEntity> filter(List<AtlasEntity> incoming) {
+        if (incoming == null) {
+            return null;
+        }
+
+        List<AtlasEntity> ret = incoming.stream()
+                .filter(x -> typesToRetain.contains(x.getTypeName()))
+                .collect(Collectors.toList());
+
+        for (AtlasEntity e : ret) {
+            for (Object o : e.getRelationshipAttributes().values()) {
+                if (o instanceof List) {
+                    List list = (List) o;
+                    for (Object ox : list) {
+                        inferObjectTypeResetGuid(ox);
+                    }
+                } else {
+                    inferObjectTypeResetGuid(o);
+                }
+            }
+        }
+
+        return ret;
+    }
+
+    private static void inferObjectTypeResetGuid(Object o) {
+        if (o instanceof AtlasRelatedObjectId) {
+            AtlasRelatedObjectId oid = (AtlasRelatedObjectId) o;
+            if (oid.getUniqueAttributes() != null) {
+                oid.setGuid(null);
+            }
+        }
+        if (o instanceof AtlasObjectId) {
+            AtlasObjectId oid = (AtlasObjectId) o;
+            if (oid.getUniqueAttributes() != null) {
+                oid.setGuid(null);
+            }
+        } else {
+            LinkedHashMap hm = (LinkedHashMap) o;
+            if (!hm.containsKey(BaseHiveEvent.ATTRIBUTE_GUID)) {
+                return;
+            }
+
+            if (hm.containsKey(BaseHiveEvent.ATTRIBUTE_UNIQUE_ATTRIBUTES)) {
+                hm.put(BaseHiveEvent.ATTRIBUTE_GUID, null);
+            }
+        }
+    }
+
+    private static AtlasEntity filter(AtlasEntity incoming) {
+        if (incoming == null) {
+            return null;
+        }
+
+        return typesToRetain.contains(incoming.getTypeName()) ? incoming : null;
+    }
+
+    private HookNotification apply(HookNotification notification) {
+        if (notification instanceof HookNotification.EntityCreateRequestV2) {
+            return apply((HookNotification.EntityCreateRequestV2) notification);
+        }
+
+        if (notification instanceof HookNotification.EntityUpdateRequestV2) {
+            return apply((HookNotification.EntityUpdateRequestV2) notification);
+        }
+
+        if (notification instanceof HookNotification.EntityPartialUpdateRequestV2) {
+            return apply((HookNotification.EntityPartialUpdateRequestV2) notification);
+        }
+
+        if (notification instanceof HookNotification.EntityDeleteRequestV2) {
+            return apply((HookNotification.EntityDeleteRequestV2) notification);
+        }
+
+        return null;
+    }
+
+    private HookNotification.EntityCreateRequestV2 apply(HookNotification.EntityCreateRequestV2 notification) {
+        AtlasEntity.AtlasEntitiesWithExtInfo entities = apply(notification.getEntities());
+        if (entities == null || CollectionUtils.isEmpty(entities.getEntities())) {
+            return null;
+        }
+
+        return new HookNotification.EntityCreateRequestV2(notification.getUser(), entities);
+    }
+
+    private HookNotification.EntityUpdateRequestV2 apply(HookNotification.EntityUpdateRequestV2 notification) {
+        AtlasEntity.AtlasEntitiesWithExtInfo entities = apply(notification.getEntities());
+        if (entities == null || CollectionUtils.isEmpty(entities.getEntities())) {
+            return null;
+        }
+
+        return new HookNotification.EntityUpdateRequestV2(notification.getUser(), entities);
+    }
+
+    private HookNotification.EntityPartialUpdateRequestV2 apply(HookNotification.EntityPartialUpdateRequestV2 notification) {
+        AtlasObjectId objectId = filterObjectId(notification.getEntityId());
+        if (objectId == null) {
+            return null;
+        }
+
+        AtlasEntity.AtlasEntityWithExtInfo entityWithExtInfo = apply(notification.getEntity());
+        if (entityWithExtInfo == null) {
+            return null;
+        }
+
+        return new HookNotification.EntityPartialUpdateRequestV2(notification.getUser(), objectId, entityWithExtInfo);
+    }
+
+    private HookNotification.EntityDeleteRequestV2 apply(HookNotification.EntityDeleteRequestV2 notification) {
+        List<AtlasObjectId> objectIds = applyForObjectIds(notification.getEntities());
+        if (CollectionUtils.isEmpty(objectIds)) {
+            return null;
+        }
+
+        return new HookNotification.EntityDeleteRequestV2(notification.getUser(), objectIds);
+    }
+}
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/PassthroughFilter.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/PassthroughFilter.java
new file mode 100644
index 0000000..f61c120
--- /dev/null
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/utils/PassthroughFilter.java
@@ -0,0 +1,29 @@
+/**
+ * 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.utils;
+
+import org.apache.atlas.model.notification.HookNotification;
+
+import java.util.List;
+
+class PassthroughFilter implements EntityFilter {
+    @Override
+    public List<HookNotification> apply(List<HookNotification> incoming) {
+        return incoming;
+    }
+}
diff --git a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilterTest.java b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilterTest.java
new file mode 100644
index 0000000..6a7af5e
--- /dev/null
+++ b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/utils/ActiveEntityFilterTest.java
@@ -0,0 +1,204 @@
+/**
+ * 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.utils;
+
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.notification.HookNotification;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.utils.TestResourceFileUtils;
+import org.apache.commons.collections.MapUtils;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+public class ActiveEntityFilterTest {
+    private static String FILE_SUFFIX = "-v2";
+
+    @BeforeClass
+    public void setup() {
+        ActiveEntityFilter.init(true);
+    }
+
+    @Test
+    public void verifyMessages() throws IOException {
+        assertAtlasEntitiesWithExtInfoFromFile("hs2-drop-db");
+        assertAtlasEntitiesWithExtInfoFromFile("hs2-create-db");
+        assertAtlasEntitiesWithExtInfoFromFile("hs2-create-table");
+        assertMessageFromFile("hs2-table-rename");
+        assertMessageFromFile("hs2-alter-view");
+        assertMessageFromFile("hs2-drop-table");
+        assertAtlasEntitiesWithExtInfoFromFile("hs2-create-process");
+    }
+
+    private void assertMessageFromFile(String msgFile) throws IOException {
+        List incoming = loadList(msgFile);
+        List expected = loadList(msgFile + FILE_SUFFIX);
+        int expectedSize = expected.size();
+
+        List<HookNotification> actual = ActiveEntityFilter.apply((List<HookNotification>) incoming);
+        assertEquals(actual.size(), expected.size());
+        for (int i = 0; i < expectedSize; i++) {
+            if (actual.get(i) instanceof HookNotification.EntityCreateRequestV2) {
+                HookNotification.EntityCreateRequestV2 actualN = (HookNotification.EntityCreateRequestV2) actual.get(i);
+                HookNotification.EntityCreateRequestV2 expectedN = (HookNotification.EntityCreateRequestV2) expected.get(i);
+
+                assertAtlasEntitiesWithExtInfo(actualN.getEntities(), expectedN.getEntities());
+            }
+
+            if (actual.get(i) instanceof HookNotification.EntityUpdateRequestV2) {
+                HookNotification.EntityUpdateRequestV2 actualN = (HookNotification.EntityUpdateRequestV2) actual.get(i);
+                HookNotification.EntityUpdateRequestV2 expectedN = (HookNotification.EntityUpdateRequestV2) expected.get(i);
+
+                assertAtlasEntitiesWithExtInfo(actualN.getEntities(), expectedN.getEntities());
+            }
+
+            if (actual.get(i) instanceof HookNotification.EntityPartialUpdateRequestV2) {
+                HookNotification.EntityPartialUpdateRequestV2 actualN = (HookNotification.EntityPartialUpdateRequestV2) actual.get(i);
+                HookNotification.EntityPartialUpdateRequestV2 expectedN = (HookNotification.EntityPartialUpdateRequestV2) expected.get(i);
+
+                assertAtlasEntitiesWithExtInfo(actualN.getEntity(), expectedN.getEntity());
+            }
+        }
+    }
+
+    private List<HookNotification> loadList(String msgFile) throws IOException {
+        List list = TestResourceFileUtils.readObjectFromJson("", msgFile, List.class);
+        List<HookNotification> ret = new ArrayList<>();
+
+        for (int i = 0; i < list.size(); i++) {
+            HookNotification notification = createNotification((LinkedHashMap) list.get(i));
+            if (notification == null) {
+                continue;
+            }
+
+            ret.add(notification);
+        }
+
+        return ret;
+    }
+
+    private HookNotification createNotification(LinkedHashMap<String, Object> linkedHashMap) {
+        assertTrue(linkedHashMap.containsKey("type"));
+
+        String type = (String) linkedHashMap.get("type");
+        switch (type) {
+            case "ENTITY_CREATE_V2":
+                return AtlasType.fromLinkedHashMap(linkedHashMap, HookNotification.EntityCreateRequestV2.class);
+
+            case "ENTITY_FULL_UPDATE_V2":
+                return AtlasType.fromLinkedHashMap(linkedHashMap, HookNotification.EntityUpdateRequestV2.class);
+
+            case "ENTITY_PARTIAL_UPDATE_V2":
+                return AtlasType.fromLinkedHashMap(linkedHashMap, HookNotification.EntityPartialUpdateRequestV2.class);
+
+            default:
+                return null;
+        }
+    }
+
+
+    private void assertAtlasEntitiesWithExtInfo(AtlasEntity.AtlasEntityWithExtInfo actual, AtlasEntity.AtlasEntityWithExtInfo expected) {
+        String actualJson = AtlasType.toJson(actual);
+        String expectedJson = AtlasType.toJson(expected);
+
+        LinkedHashMap<String, Object> actualLHM = AtlasType.fromJson(actualJson, LinkedHashMap.class);
+        LinkedHashMap<String, Object> expectedLHM = AtlasType.fromJson(expectedJson, LinkedHashMap.class);
+
+        AssertLinkedHashMap.assertEquals(actualLHM, expectedLHM);
+    }
+
+    private void assertAtlasEntitiesWithExtInfoFromFile(String entityFile) throws IOException {
+        AtlasEntity.AtlasEntitiesWithExtInfo incoming = TestResourceFileUtils.readObjectFromJson("", entityFile, AtlasEntity.AtlasEntitiesWithExtInfo.class);
+        AtlasEntity.AtlasEntitiesWithExtInfo expected = TestResourceFileUtils.readObjectFromJson("", entityFile + FILE_SUFFIX, AtlasEntity.AtlasEntitiesWithExtInfo.class);
+
+        HiveDDLEntityFilter hiveLineageEntityFilter = new HiveDDLEntityFilter();
+        AtlasEntity.AtlasEntitiesWithExtInfo actual = hiveLineageEntityFilter.apply(incoming);
+        assertAtlasEntitiesWithExtInfo(actual, expected);
+    }
+
+    private void assertAtlasEntitiesWithExtInfo(AtlasEntity.AtlasEntitiesWithExtInfo actual, AtlasEntity.AtlasEntitiesWithExtInfo expected) {
+        assertNotNull(actual);
+        assertNotNull(expected);
+
+        assertEquals(actual.getEntities().size(), expected.getEntities().size());
+        assertEntity(actual.getEntities(), expected.getEntities());
+
+        assertEquals(MapUtils.isEmpty(actual.getReferredEntities()), MapUtils.isEmpty(expected.getReferredEntities()));
+        if (expected.getReferredEntities() != null && actual.getReferredEntities() != null) {
+            assertEntity(actual.getReferredEntities(), expected.getReferredEntities());
+        }
+    }
+
+    private void assertEntity(Map<String, AtlasEntity> actual, Map<String, AtlasEntity> expected) {
+        assertEquals(actual.size(), expected.size());
+    }
+
+    private void assertEntity(List<AtlasEntity> actual, List<AtlasEntity> expected) {
+        AssertLinkedHashMap.assertEquals(actual, expected);
+    }
+
+    private static class AssertLinkedHashMap {
+        private static final String MISMATCH_KEY_FORMAT = "Mismatch: Key: %s";
+        private  static final Set<String> excludeKeys = new HashSet<String>() {{
+            add("guid");
+            add("owner");
+        }};
+
+        public static void assertEquals(LinkedHashMap<String, Object> actual, LinkedHashMap<String, Object> expected) {
+            for (String key : expected.keySet()) {
+                assertTrue(actual.containsKey(key), "Key: " + key + " Not found!");
+
+                if (excludeKeys.contains(key)) {
+                    continue;
+                }
+
+                if (actual.get(key) instanceof LinkedHashMap) {
+                    assertEquals((LinkedHashMap) actual.get(key), (LinkedHashMap) expected.get(key));
+                    continue;
+                }
+
+                Assert.assertEquals(actual.get(key), actual.get(key), String.format(MISMATCH_KEY_FORMAT, key));
+            }
+        }
+
+        public static void assertEquals(List<AtlasEntity> actual, List<AtlasEntity> expected) {
+            Assert.assertEquals(actual.size(), expected.size());
+            for (int i = 0; i < actual.size(); i++) {
+                AtlasEntity actualEntity = actual.get(i);
+                AtlasEntity expectedEntity = expected.get(i);
+
+                String actualJson = AtlasType.toJson(actualEntity);
+                String expectedJson = AtlasType.toJson(expectedEntity);
+
+                assertEquals(AtlasType.fromJson(actualJson, LinkedHashMap.class), AtlasType.fromJson(expectedJson, LinkedHashMap.class));
+            }
+        }
+    }
+}
diff --git a/addons/hive-bridge/src/test/resources/atlas-application.properties b/addons/hive-bridge/src/test/resources/atlas-application.properties
index 898b69c..2848ceb 100644
--- a/addons/hive-bridge/src/test/resources/atlas-application.properties
+++ b/addons/hive-bridge/src/test/resources/atlas-application.properties
@@ -122,3 +122,4 @@ atlas.authentication.method.file=true
 atlas.authentication.method.ldap.type=none
 atlas.authentication.method.kerberos=false
 # atlas.authentication.method.file.filename=users-credentials.properties
+atlas.hook.hive.ignore.ddl.operations=false
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-alter-view-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-alter-view-v2.json
new file mode 100644
index 0000000..ebe896b
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-alter-view-v2.json
@@ -0,0 +1,35 @@
+[
+  {
+    "type": "ENTITY_CREATE_V2",
+    "user": "hive",
+    "entities": {
+      "entities": [
+        {
+          "typeName": "hive_table_ddl",
+          "attributes": {
+            "serviceType": "hive",
+            "qualifiedName": "cadb02.hh6_renamed_view2@cm:1616450675937",
+            "execTime": 1616450675937,
+            "queryText": "ALTER VIEW hh6_renamed_view RENAME TO hh6_renamed_view2",
+            "name": "ALTER VIEW hh6_renamed_view RENAME TO hh6_renamed_view2",
+            "userName": "hive"
+          },
+          "guid": "-14529329955589487",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed_view2@cm"
+              },
+              "relationshipType": "hive_table_ddl_queries"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  }
+]
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-alter-view.json b/addons/hive-bridge/src/test/resources/json/hs2-alter-view.json
new file mode 100644
index 0000000..cfe7784
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-alter-view.json
@@ -0,0 +1,268 @@
+[
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_column",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6_renamed_view.col1@cm"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_column",
+        "attributes": {
+          "qualifiedName": "cadb02.hh6_renamed_view2.col1@cm"
+        },
+        "guid": "-14529329955589486",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_storagedesc",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6_renamed_view@cm_storage"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_storagedesc",
+        "attributes": {
+          "qualifiedName": "cadb02.hh6_renamed_view2@cm_storage",
+          "storedAsSubDirectories": false,
+          "location": null,
+          "compressed": false,
+          "inputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
+          "parameters": {},
+          "outputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
+          "serdeInfo": {
+            "typeName": "hive_serde",
+            "attributes": {
+              "serializationLib": null,
+              "name": null,
+              "parameters": {}
+            }
+          },
+          "numBuckets": -1
+        },
+        "guid": "-14529329955589484",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_table",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6_renamed_view@cm"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_table",
+        "attributes": {
+          "owner": "hive",
+          "temporary": false,
+          "lastAccessTime": 1616450675000,
+          "aliases": [
+            "hh6_renamed_view"
+          ],
+          "qualifiedName": "cadb02.hh6_renamed_view2@cm",
+          "viewExpandedText": "select `hh6_renamed`.`col1` from `cadb02`.`hh6_renamed`",
+          "tableType": "VIRTUAL_VIEW",
+          "createTime": 1616450675000,
+          "name": "hh6_renamed_view2",
+          "comment": null,
+          "parameters": {
+            "transient_lastDdlTime": "1616450676",
+            "bucketing_version": "2",
+            "last_modified_time": "1616450676",
+            "last_modified_by": "hive"
+          },
+          "retention": 0,
+          "viewOriginalText": "select * from hh6_renamed"
+        },
+        "guid": "-14529329955589483",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_FULL_UPDATE_V2",
+    "user": "hive",
+    "entities": {
+      "referredEntities": {
+        "-14529329955589485": {
+          "typeName": "hive_column",
+          "attributes": {
+            "owner": "hive",
+            "qualifiedName": "cadb02.hh6_renamed_view2.col1@cm",
+            "name": "col1",
+            "comment": null,
+            "position": 0,
+            "type": "int"
+          },
+          "guid": "-14529329955589485",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "guid": "-14529329955589483",
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed_view2@cm"
+              },
+              "relationshipType": "hive_table_columns"
+            }
+          },
+          "proxy": false
+        },
+        "-14529329955589484": {
+          "typeName": "hive_storagedesc",
+          "attributes": {
+            "qualifiedName": "cadb02.hh6_renamed_view2@cm_storage",
+            "storedAsSubDirectories": false,
+            "location": null,
+            "compressed": false,
+            "inputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
+            "parameters": {},
+            "outputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
+            "serdeInfo": {
+              "typeName": "hive_serde",
+              "attributes": {
+                "serializationLib": null,
+                "name": null,
+                "parameters": {}
+              }
+            },
+            "numBuckets": -1
+          },
+          "guid": "-14529329955589484",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "guid": "-14529329955589483",
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed_view2@cm"
+              },
+              "relationshipType": "hive_table_storagedesc"
+            }
+          },
+          "proxy": false
+        }
+      },
+      "entities": [
+        {
+          "typeName": "hive_table",
+          "attributes": {
+            "owner": "hive",
+            "temporary": false,
+            "lastAccessTime": 1616450675000,
+            "aliases": [
+              "hh6_renamed_view"
+            ],
+            "qualifiedName": "cadb02.hh6_renamed_view2@cm",
+            "viewExpandedText": "select `hh6_renamed`.`col1` from `cadb02`.`hh6_renamed`",
+            "tableType": "VIRTUAL_VIEW",
+            "createTime": 1616450675000,
+            "name": "hh6_renamed_view2",
+            "comment": null,
+            "parameters": {
+              "transient_lastDdlTime": "1616450676",
+              "bucketing_version": "2",
+              "last_modified_time": "1616450676",
+              "last_modified_by": "hive"
+            },
+            "retention": 0,
+            "viewOriginalText": "select * from hh6_renamed"
+          },
+          "guid": "-14529329955589483",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "sd": {
+              "guid": "-14529329955589484",
+              "typeName": "hive_storagedesc",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed_view2@cm_storage"
+              },
+              "relationshipType": "hive_table_storagedesc"
+            },
+            "columns": [
+              {
+                "guid": "-14529329955589485",
+                "typeName": "hive_column",
+                "uniqueAttributes": {
+                  "qualifiedName": "cadb02.hh6_renamed_view2.col1@cm"
+                },
+                "relationshipType": "hive_table_columns"
+              }
+            ],
+            "partitionKeys": [],
+            "db": {
+              "typeName": "hive_db",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02@cm"
+              },
+              "relationshipType": "hive_table_db"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  },
+  {
+    "type": "ENTITY_CREATE_V2",
+    "user": "hive",
+    "entities": {
+      "entities": [
+        {
+          "typeName": "hive_table_ddl",
+          "attributes": {
+            "serviceType": "hive",
+            "qualifiedName": "cadb02.hh6_renamed_view2@cm:1616450675937",
+            "execTime": 1616450675937,
+            "queryText": "ALTER VIEW hh6_renamed_view RENAME TO hh6_renamed_view2",
+            "name": "ALTER VIEW hh6_renamed_view RENAME TO hh6_renamed_view2",
+            "userName": "hive"
+          },
+          "guid": "-14529329955589487",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed_view2@cm"
+              },
+              "relationshipType": "hive_table_ddl_queries"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  }
+]
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-db-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-create-db-v2.json
new file mode 100644
index 0000000..42553b5
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-db-v2.json
@@ -0,0 +1,30 @@
+{
+  "referredEntities": {},
+  "entities": [
+    {
+      "typeName": "hive_db_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb02@cm:1616450673617",
+        "execTime": 1616450673617,
+        "queryText": "create database cadb02",
+        "name": "create database cadb02",
+        "userName": "hive"
+      },
+      "guid": "-14530890092409411",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "db": {
+          "typeName": "hive_db",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02@cm"
+          },
+          "relationshipType": "hive_db_ddl_queries"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-db.json b/addons/hive-bridge/src/test/resources/json/hs2-create-db.json
new file mode 100644
index 0000000..a5b810f
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-db.json
@@ -0,0 +1,73 @@
+{
+  "referredEntities": {},
+  "entities": [
+    {
+      "typeName": "hive_db",
+      "attributes": {
+        "owner": "hive",
+        "ownerType": "USER",
+        "managedLocation": null,
+        "qualifiedName": "cadb02@cm",
+        "clusterName": "cm",
+        "name": "cadb02",
+        "location": "hdfs://ve0126.halxg.cloudera.com:8020/warehouse/tablespace/external/hive/cadb02.db",
+        "parameters": {}
+      },
+      "guid": "-14529329955589448",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "proxy": false
+    },
+    {
+      "typeName": "hive_db_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb02@cm:1616450673617",
+        "execTime": 1616450673617,
+        "queryText": "create database cadb02",
+        "name": "create database cadb02",
+        "userName": "hive"
+      },
+      "guid": "-14529329955589449",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "db": {
+          "guid": "-14529329955589448",
+          "typeName": "hive_db",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02@cm"
+          },
+          "relationshipType": "hive_db_ddl_queries"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hdfs_path",
+      "attributes": {
+        "path": "hdfs://ve0126.halxg.cloudera.com:8020/warehouse/tablespace/external/hive/cadb02.db",
+        "qualifiedName": "hdfs://ve0126.halxg.cloudera.com:8020/warehouse/tablespace/external/hive/cadb02.db@cm",
+        "clusterName": "cm",
+        "name": "/warehouse/tablespace/external/hive/cadb02.db"
+      },
+      "guid": "-14529329955589450",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "hiveDb": {
+          "guid": "-14529329955589448",
+          "typeName": "hive_db",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02@cm"
+          },
+          "relationshipType": "hive_db_location"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-process-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-create-process-v2.json
new file mode 100644
index 0000000..8e55b72
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-process-v2.json
@@ -0,0 +1,144 @@
+{
+  "referredEntities": {},
+  "entities": [
+    {
+      "typeName": "hive_table_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb202.vw202@primary:1616604468798",
+        "execTime": 1616604468798,
+        "queryText": "create table vw202 as select * from hh202",
+        "name": "create table vw202 as select * from hh202",
+        "userName": "hive"
+      },
+      "guid": "-44808597128613",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-44808597128610",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary"
+          },
+          "relationshipType": "hive_table_ddl_queries"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process",
+      "attributes": {
+        "recentQueries": [
+          "create table vw202 as select * from hh202"
+        ],
+        "qualifiedName": "cadb202.vw202@primary:1616604471000",
+        "clusterName": "primary",
+        "name": "cadb202.vw202@primary:1616604471000",
+        "queryText": "",
+        "operationType": "CREATETABLE_AS_SELECT",
+        "startTime": 1616604475518,
+        "queryPlan": "Not Supported",
+        "endTime": 1616604475518,
+        "userName": "",
+        "queryId": ""
+      },
+      "guid": "-44808597128614",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-44808597128610",
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.vw202@primary"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.hh202@primary"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ]
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process_execution",
+      "attributes": {
+        "hostName": "21806.local",
+        "qualifiedName": "cadb202.vw202@primary:1616604471000:1616604468798:1616604475518",
+        "name": "cadb202.vw202@primary:1616604471000:1616604468798:1616604475518",
+        "queryText": "create table vw202 as select * from hh202",
+        "startTime": 1616604468798,
+        "queryPlan": "Not Supported",
+        "endTime": 1616604475518,
+        "userName": "hive",
+        "queryId": "hive_20210324094633_2144da6f-70ac-476e-aeec-0e758cdf1fa6"
+      },
+      "guid": "-44808597128615",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "process": {
+          "guid": "-44808597128614",
+          "typeName": "hive_process",
+          "relationshipType": "hive_process_process_executions"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_column_lineage",
+      "attributes": {
+        "expression": null,
+        "qualifiedName": "cadb202.vw202@primary:1616604471000:col202",
+        "name": "cadb202.vw202@primary:1616604471000:col202",
+        "depenendencyType": "SIMPLE"
+      },
+      "guid": "-44808597128616",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-44808597128612",
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.vw202.col202@primary"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.hh202.col202@primary"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ],
+        "query": {
+          "guid": "-44808597128614",
+          "typeName": "hive_process",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary:1616604471000"
+          },
+          "relationshipType": "hive_process_column_lineage"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-process.json b/addons/hive-bridge/src/test/resources/json/hs2-create-process.json
new file mode 100644
index 0000000..1bbd95a
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-process.json
@@ -0,0 +1,283 @@
+{
+  "referredEntities": {
+    "-44808597128606": {
+      "typeName": "hive_db",
+      "attributes": {
+        "owner": "hive",
+        "ownerType": "USER",
+        "qualifiedName": "cadb202@primary",
+        "clusterName": "primary",
+        "name": "cadb202",
+        "location": "file:/Users/hive/Apache/atlas-wip/addons/hive-bridge/target/metastore/cadb202.db",
+        "parameters": {}
+      },
+      "guid": "-44808597128606",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "proxy": false
+    },
+    "-44808597128612": {
+      "typeName": "hive_column",
+      "attributes": {
+        "owner": "hive",
+        "qualifiedName": "cadb202.vw202.col202@primary",
+        "name": "col202",
+        "comment": null,
+        "position": 0,
+        "type": "string"
+      },
+      "guid": "-44808597128612",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-44808597128610",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary"
+          },
+          "relationshipType": "hive_table_columns"
+        }
+      },
+      "proxy": false
+    },
+    "-44808597128611": {
+      "typeName": "hive_storagedesc",
+      "attributes": {
+        "qualifiedName": "cadb202.vw202@primary_storage",
+        "storedAsSubDirectories": false,
+        "location": "file:/Users/hive/Apache/atlas-wip/addons/hive-bridge/target/metastore/cadb202.db/vw202",
+        "compressed": false,
+        "inputFormat": "org.apache.hadoop.mapred.TextInputFormat",
+        "parameters": {},
+        "outputFormat": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
+        "serdeInfo": {
+          "typeName": "hive_serde",
+          "attributes": {
+            "serializationLib": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
+            "name": null,
+            "parameters": {
+              "serialization.format": "1"
+            }
+          }
+        },
+        "numBuckets": -1
+      },
+      "guid": "-44808597128611",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-44808597128610",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary"
+          },
+          "relationshipType": "hive_table_storagedesc"
+        }
+      },
+      "proxy": false
+    },
+    "-44808597128610": {
+      "typeName": "hive_table",
+      "attributes": {
+        "owner": "hive",
+        "tableType": "MANAGED_TABLE",
+        "temporary": false,
+        "lastAccessTime": 1616604471000,
+        "createTime": 1616604471000,
+        "qualifiedName": "cadb202.vw202@primary",
+        "name": "vw202",
+        "comment": null,
+        "parameters": {
+          "totalSize": "0",
+          "numRows": "0",
+          "rawDataSize": "0",
+          "COLUMN_STATS_ACCURATE": "{\"BASIC_STATS\":\"true\"}",
+          "numFiles": "0",
+          "transient_lastDdlTime": "1616604472",
+          "bucketing_version": "2"
+        },
+        "retention": 0
+      },
+      "guid": "-44808597128610",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "sd": {
+          "guid": "-44808597128611",
+          "typeName": "hive_storagedesc",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary_storage"
+          },
+          "relationshipType": "hive_table_storagedesc"
+        },
+        "columns": [
+          {
+            "guid": "-44808597128612",
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.vw202.col202@primary"
+            },
+            "relationshipType": "hive_table_columns"
+          }
+        ],
+        "partitionKeys": [],
+        "db": {
+          "guid": "-44808597128606",
+          "typeName": "hive_db",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202@primary"
+          },
+          "relationshipType": "hive_table_db"
+        }
+      },
+      "proxy": false
+    }
+  },
+  "entities": [
+    {
+      "typeName": "hive_table_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb202.vw202@primary:1616604468798",
+        "execTime": 1616604468798,
+        "queryText": "create table vw202 as select * from hh202",
+        "name": "create table vw202 as select * from hh202",
+        "userName": "hive"
+      },
+      "guid": "-44808597128613",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-44808597128610",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary"
+          },
+          "relationshipType": "hive_table_ddl_queries"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process",
+      "attributes": {
+        "recentQueries": [
+          "create table vw202 as select * from hh202"
+        ],
+        "qualifiedName": "cadb202.vw202@primary:1616604471000",
+        "clusterName": "primary",
+        "name": "cadb202.vw202@primary:1616604471000",
+        "queryText": "",
+        "operationType": "CREATETABLE_AS_SELECT",
+        "startTime": 1616604475518,
+        "queryPlan": "Not Supported",
+        "endTime": 1616604475518,
+        "userName": "",
+        "queryId": ""
+      },
+      "guid": "-44808597128614",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-44808597128610",
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.vw202@primary"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.hh202@primary"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ]
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process_execution",
+      "attributes": {
+        "hostName": "21806.local",
+        "qualifiedName": "cadb202.vw202@primary:1616604471000:1616604468798:1616604475518",
+        "name": "cadb202.vw202@primary:1616604471000:1616604468798:1616604475518",
+        "queryText": "create table vw202 as select * from hh202",
+        "startTime": 1616604468798,
+        "queryPlan": "Not Supported",
+        "endTime": 1616604475518,
+        "userName": "hive",
+        "queryId": "hive_20210324094633_2144da6f-70ac-476e-aeec-0e758cdf1fa6"
+      },
+      "guid": "-44808597128615",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "process": {
+          "guid": "-44808597128614",
+          "typeName": "hive_process",
+          "relationshipType": "hive_process_process_executions"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_column_lineage",
+      "attributes": {
+        "expression": null,
+        "qualifiedName": "cadb202.vw202@primary:1616604471000:col202",
+        "name": "cadb202.vw202@primary:1616604471000:col202",
+        "depenendencyType": "SIMPLE"
+      },
+      "guid": "-44808597128616",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-44808597128612",
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.vw202.col202@primary"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb202.hh202.col202@primary"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ],
+        "query": {
+          "guid": "-44808597128614",
+          "typeName": "hive_process",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb202.vw202@primary:1616604471000"
+          },
+          "relationshipType": "hive_process_column_lineage"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-table-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-create-table-v2.json
new file mode 100644
index 0000000..b67f2ff
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-table-v2.json
@@ -0,0 +1,102 @@
+{
+  "referredEntities": {},
+  "entities": [
+    {
+      "typeName": "hive_process",
+      "attributes": {
+        "recentQueries": [
+          "create external table hh6(col1 int) location '/tmp/external/hh6.csv'"
+        ],
+        "qualifiedName": "cadb02.hh6@cm:1616450674000",
+        "clusterName": "cm",
+        "name": "cadb02.hh6@cm:1616450674000",
+        "queryText": "",
+        "operationType": "CREATETABLE",
+        "startTime": 1616450674217,
+        "queryPlan": "Not Supported",
+        "endTime": 1616450674217,
+        "userName": "",
+        "queryId": ""
+      },
+      "guid": "-14529329955589456",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-14529329955589452",
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb02.hh6@cm"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "guid": "-14529329955589455",
+            "typeName": "hdfs_path",
+            "uniqueAttributes": {
+              "qualifiedName": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv@cm"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ]
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process_execution",
+      "attributes": {
+        "hostName": "ve0126.halxg.cloudera.com",
+        "qualifiedName": "cadb02.hh6@cm:1616450674000:1616450673854:1616450674217",
+        "name": "cadb02.hh6@cm:1616450674000:1616450673854:1616450674217",
+        "queryText": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "startTime": 1616450673854,
+        "queryPlan": "Not Supported",
+        "endTime": 1616450674217,
+        "userName": "hive",
+        "queryId": "hive_20210322150433_52b16d58-6a44-49e2-afe4-4bb1a5bb7484"
+      },
+      "guid": "-14529329955589457",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "process": {
+          "guid": "-14529329955589456",
+          "typeName": "hive_process",
+          "relationshipType": "hive_process_process_executions"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_table_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb02.hh6@cm:1616450673854",
+        "execTime": 1616450673854,
+        "queryText": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "name": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "userName": "hive"
+      },
+      "guid": "-14529329955589458",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-14529329955589452",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02.hh6@cm"
+          },
+          "relationshipType": "hive_table_ddl_queries"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-create-table.json b/addons/hive-bridge/src/test/resources/json/hs2-create-table.json
new file mode 100644
index 0000000..4e13ed1
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-create-table.json
@@ -0,0 +1,255 @@
+{
+  "referredEntities": {
+    "-14529329955589453": {
+      "typeName": "hive_storagedesc",
+      "attributes": {
+        "qualifiedName": "cadb02.hh6@cm_storage",
+        "storedAsSubDirectories": false,
+        "location": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv",
+        "compressed": false,
+        "inputFormat": "org.apache.hadoop.mapred.TextInputFormat",
+        "parameters": {},
+        "outputFormat": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
+        "serdeInfo": {
+          "typeName": "hive_serde",
+          "attributes": {
+            "serializationLib": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
+            "name": null,
+            "parameters": {
+              "serialization.format": "1"
+            }
+          }
+        },
+        "numBuckets": -1
+      },
+      "guid": "-14529329955589453",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-14529329955589452",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02.hh6@cm"
+          },
+          "relationshipType": "hive_table_storagedesc"
+        }
+      },
+      "proxy": false
+    },
+    "-14529329955589451": {
+      "typeName": "hive_db",
+      "attributes": {
+        "owner": "hive",
+        "ownerType": "USER",
+        "managedLocation": null,
+        "qualifiedName": "cadb02@cm",
+        "clusterName": "cm",
+        "name": "cadb02",
+        "location": "hdfs://ve0126.halxg.cloudera.com:8020/warehouse/tablespace/external/hive/cadb02.db",
+        "parameters": {}
+      },
+      "guid": "-14529329955589451",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "proxy": false
+    },
+    "-14529329955589455": {
+      "typeName": "hdfs_path",
+      "attributes": {
+        "path": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv",
+        "qualifiedName": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv@cm",
+        "clusterName": "cm",
+        "name": "/tmp/external/hh6.csv"
+      },
+      "guid": "-14529329955589455",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "proxy": false
+    },
+    "-14529329955589454": {
+      "typeName": "hive_column",
+      "attributes": {
+        "owner": "hive",
+        "qualifiedName": "cadb02.hh6.col1@cm",
+        "name": "col1",
+        "comment": null,
+        "position": 0,
+        "type": "int"
+      },
+      "guid": "-14529329955589454",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-14529329955589452",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02.hh6@cm"
+          },
+          "relationshipType": "hive_table_columns"
+        }
+      },
+      "proxy": false
+    }
+  },
+  "entities": [
+    {
+      "typeName": "hive_table",
+      "attributes": {
+        "owner": "hive",
+        "tableType": "EXTERNAL_TABLE",
+        "temporary": false,
+        "lastAccessTime": 1616450674000,
+        "createTime": 1616450674000,
+        "qualifiedName": "cadb02.hh6@cm",
+        "name": "hh6",
+        "comment": null,
+        "parameters": {
+          "totalSize": "0",
+          "EXTERNAL": "TRUE",
+          "numFiles": "0",
+          "transient_lastDdlTime": "1616450674",
+          "bucketing_version": "2",
+          "numFilesErasureCoded": "0"
+        },
+        "retention": 0
+      },
+      "guid": "-14529329955589452",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "sd": {
+          "guid": "-14529329955589453",
+          "typeName": "hive_storagedesc",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02.hh6@cm_storage"
+          },
+          "relationshipType": "hive_table_storagedesc"
+        },
+        "columns": [
+          {
+            "guid": "-14529329955589454",
+            "typeName": "hive_column",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb02.hh6.col1@cm"
+            },
+            "relationshipType": "hive_table_columns"
+          }
+        ],
+        "partitionKeys": [],
+        "db": {
+          "guid": "-14529329955589451",
+          "typeName": "hive_db",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02@cm"
+          },
+          "relationshipType": "hive_table_db"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process",
+      "attributes": {
+        "recentQueries": [
+          "create external table hh6(col1 int) location '/tmp/external/hh6.csv'"
+        ],
+        "qualifiedName": "cadb02.hh6@cm:1616450674000",
+        "clusterName": "cm",
+        "name": "cadb02.hh6@cm:1616450674000",
+        "queryText": "",
+        "operationType": "CREATETABLE",
+        "startTime": 1616450674217,
+        "queryPlan": "Not Supported",
+        "endTime": 1616450674217,
+        "userName": "",
+        "queryId": ""
+      },
+      "guid": "-14529329955589456",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "outputs": [
+          {
+            "guid": "-14529329955589452",
+            "typeName": "hive_table",
+            "uniqueAttributes": {
+              "qualifiedName": "cadb02.hh6@cm"
+            },
+            "relationshipType": "process_dataset_outputs"
+          }
+        ],
+        "inputs": [
+          {
+            "guid": "-14529329955589455",
+            "typeName": "hdfs_path",
+            "uniqueAttributes": {
+              "qualifiedName": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv@cm"
+            },
+            "relationshipType": "dataset_process_inputs"
+          }
+        ]
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_process_execution",
+      "attributes": {
+        "hostName": "ve0126.halxg.cloudera.com",
+        "qualifiedName": "cadb02.hh6@cm:1616450674000:1616450673854:1616450674217",
+        "name": "cadb02.hh6@cm:1616450674000:1616450673854:1616450674217",
+        "queryText": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "startTime": 1616450673854,
+        "queryPlan": "Not Supported",
+        "endTime": 1616450674217,
+        "userName": "hive",
+        "queryId": "hive_20210322150433_52b16d58-6a44-49e2-afe4-4bb1a5bb7484"
+      },
+      "guid": "-14529329955589457",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "process": {
+          "guid": "-14529329955589456",
+          "typeName": "hive_process",
+          "relationshipType": "hive_process_process_executions"
+        }
+      },
+      "proxy": false
+    },
+    {
+      "typeName": "hive_table_ddl",
+      "attributes": {
+        "serviceType": "hive",
+        "qualifiedName": "cadb02.hh6@cm:1616450673854",
+        "execTime": 1616450673854,
+        "queryText": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "name": "create external table hh6(col1 int) location '/tmp/external/hh6.csv'",
+        "userName": "hive"
+      },
+      "guid": "-14529329955589458",
+      "isIncomplete": false,
+      "provenanceType": 0,
+      "version": 0,
+      "relationshipAttributes": {
+        "table": {
+          "guid": "-14529329955589452",
+          "typeName": "hive_table",
+          "uniqueAttributes": {
+            "qualifiedName": "cadb02.hh6@cm"
+          },
+          "relationshipType": "hive_table_ddl_queries"
+        }
+      },
+      "proxy": false
+    }
+  ]
+}
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-drop-db-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-drop-db-v2.json
new file mode 100644
index 0000000..76860ce
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-drop-db-v2.json
@@ -0,0 +1,4 @@
+{
+  "entities": [
+  ]
+}
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-drop-db.json b/addons/hive-bridge/src/test/resources/json/hs2-drop-db.json
new file mode 100644
index 0000000..429f95f
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-drop-db.json
@@ -0,0 +1,10 @@
+{
+  "entities": [
+    {
+      "typeName": "hive_db",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02@cm"
+      }
+    }
+  ]
+}
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-drop-table-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-drop-table-v2.json
new file mode 100644
index 0000000..0637a08
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-drop-table-v2.json
@@ -0,0 +1 @@
+[]
\ No newline at end of file
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-drop-table.json b/addons/hive-bridge/src/test/resources/json/hs2-drop-table.json
new file mode 100644
index 0000000..ed62032
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-drop-table.json
@@ -0,0 +1,50 @@
+[
+  {
+    "type": "ENTITY_DELETE_V2",
+    "user": "hive",
+    "entities": [
+      {
+        "typeName": "hive_table",
+        "uniqueAttributes": {
+          "qualifiedName": "cadb02.hh6_renamed@cm"
+        }
+      }
+    ]
+  },
+  {
+    "type": "ENTITY_DELETE_V2",
+    "user": "hive",
+    "entities": [
+      {
+        "typeName": "hive_table",
+        "uniqueAttributes": {
+          "qualifiedName": "cadb02.hh6_renamed_view2@cm"
+        }
+      }
+    ]
+  },
+  {
+    "type": "ENTITY_DELETE_V2",
+    "user": "hive",
+    "entities": [
+      {
+        "typeName": "hive_table",
+        "uniqueAttributes": {
+          "qualifiedName": "cadb02.cc1@cm"
+        }
+      }
+    ]
+  },
+  {
+    "type": "ENTITY_DELETE_V2",
+    "user": "hive",
+    "entities": [
+      {
+        "typeName": "hive_db",
+        "uniqueAttributes": {
+          "qualifiedName": "cadb02@cm"
+        }
+      }
+    ]
+  }
+]
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-table-rename-v2.json b/addons/hive-bridge/src/test/resources/json/hs2-table-rename-v2.json
new file mode 100644
index 0000000..f2b2bf7
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-table-rename-v2.json
@@ -0,0 +1,35 @@
+[
+  {
+    "type": "ENTITY_CREATE_V2",
+    "user": "hive",
+    "entities": {
+      "entities": [
+        {
+          "typeName": "hive_table_ddl",
+          "attributes": {
+            "serviceType": "hive",
+            "qualifiedName": "cadb02.hh6_renamed@cm:1616452234545",
+            "execTime": 1616452234545,
+            "queryText": "ALTER TABLE hh6 RENAME TO hh6_renamed",
+            "name": "ALTER TABLE hh6 RENAME TO hh6_renamed",
+            "userName": "hive"
+          },
+          "guid": "-14530890092409428",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed@cm"
+              },
+              "relationshipType": "hive_table_ddl_queries"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  }
+]
diff --git a/addons/hive-bridge/src/test/resources/json/hs2-table-rename.json b/addons/hive-bridge/src/test/resources/json/hs2-table-rename.json
new file mode 100644
index 0000000..6b8094d
--- /dev/null
+++ b/addons/hive-bridge/src/test/resources/json/hs2-table-rename.json
@@ -0,0 +1,276 @@
+[
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_column",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6.col1@cm"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_column",
+        "attributes": {
+          "qualifiedName": "cadb02.hh6_renamed.col1@cm"
+        },
+        "guid": "-14529329955589466",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_storagedesc",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6@cm_storage"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_storagedesc",
+        "attributes": {
+          "qualifiedName": "cadb02.hh6_renamed@cm_storage",
+          "storedAsSubDirectories": false,
+          "location": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv",
+          "compressed": false,
+          "inputFormat": "org.apache.hadoop.mapred.TextInputFormat",
+          "parameters": {},
+          "outputFormat": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
+          "serdeInfo": {
+            "typeName": "hive_serde",
+            "attributes": {
+              "serializationLib": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
+              "name": null,
+              "parameters": {
+                "serialization.format": "1"
+              }
+            }
+          },
+          "numBuckets": -1
+        },
+        "guid": "-14529329955589464",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_PARTIAL_UPDATE_V2",
+    "user": "hive",
+    "entityId": {
+      "typeName": "hive_table",
+      "uniqueAttributes": {
+        "qualifiedName": "cadb02.hh6@cm"
+      }
+    },
+    "entity": {
+      "entity": {
+        "typeName": "hive_table",
+        "attributes": {
+          "owner": "hive",
+          "tableType": "EXTERNAL_TABLE",
+          "temporary": false,
+          "lastAccessTime": 1616450674000,
+          "aliases": [
+            "hh6"
+          ],
+          "createTime": 1616450674000,
+          "qualifiedName": "cadb02.hh6_renamed@cm",
+          "name": "hh6_renamed",
+          "comment": null,
+          "parameters": {
+            "last_modified_time": "1616450674",
+            "totalSize": "0",
+            "EXTERNAL": "TRUE",
+            "numFiles": "0",
+            "transient_lastDdlTime": "1616450674",
+            "bucketing_version": "2",
+            "last_modified_by": "hive",
+            "numFilesErasureCoded": "0"
+          },
+          "retention": 0
+        },
+        "guid": "-14529329955589463",
+        "isIncomplete": false,
+        "provenanceType": 0,
+        "version": 0,
+        "proxy": false
+      }
+    }
+  },
+  {
+    "type": "ENTITY_FULL_UPDATE_V2",
+    "user": "hive",
+    "entities": {
+      "referredEntities": {
+        "-14529329955589464": {
+          "typeName": "hive_storagedesc",
+          "attributes": {
+            "qualifiedName": "cadb02.hh6_renamed@cm_storage",
+            "storedAsSubDirectories": false,
+            "location": "hdfs://ve0126.halxg.cloudera.com:8020/tmp/external/hh6.csv",
+            "compressed": false,
+            "inputFormat": "org.apache.hadoop.mapred.TextInputFormat",
+            "parameters": {},
+            "outputFormat": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
+            "serdeInfo": {
+              "typeName": "hive_serde",
+              "attributes": {
+                "serializationLib": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
+                "name": null,
+                "parameters": {
+                  "serialization.format": "1"
+                }
+              }
+            },
+            "numBuckets": -1
+          },
+          "guid": "-14529329955589464",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "guid": "-14529329955589463",
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed@cm"
+              },
+              "relationshipType": "hive_table_storagedesc"
+            }
+          },
+          "proxy": false
+        },
+        "-14529329955589465": {
+          "typeName": "hive_column",
+          "attributes": {
+            "owner": "hive",
+            "qualifiedName": "cadb02.hh6_renamed.col1@cm",
+            "name": "col1",
+            "comment": null,
+            "position": 0,
+            "type": "int"
+          },
+          "guid": "-14529329955589465",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "guid": "-14529329955589463",
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed@cm"
+              },
+              "relationshipType": "hive_table_columns"
+            }
+          },
+          "proxy": false
+        }
+      },
+      "entities": [
+        {
+          "typeName": "hive_table",
+          "attributes": {
+            "owner": "hive",
+            "tableType": "EXTERNAL_TABLE",
+            "temporary": false,
+            "lastAccessTime": 1616450674000,
+            "aliases": [
+              "hh6"
+            ],
+            "createTime": 1616450674000,
+            "qualifiedName": "cadb02.hh6_renamed@cm",
+            "name": "hh6_renamed",
+            "comment": null,
+            "parameters": {
+              "last_modified_time": "1616450674",
+              "totalSize": "0",
+              "EXTERNAL": "TRUE",
+              "numFiles": "0",
+              "transient_lastDdlTime": "1616450674",
+              "bucketing_version": "2",
+              "last_modified_by": "hive",
+              "numFilesErasureCoded": "0"
+            },
+            "retention": 0
+          },
+          "guid": "-14529329955589463",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "sd": {
+              "guid": "-14529329955589464",
+              "typeName": "hive_storagedesc",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed@cm_storage"
+              },
+              "relationshipType": "hive_table_storagedesc"
+            },
+            "columns": [
+              {
+                "guid": "-14529329955589465",
+                "typeName": "hive_column",
+                "uniqueAttributes": {
+                  "qualifiedName": "cadb02.hh6_renamed.col1@cm"
+                },
+                "relationshipType": "hive_table_columns"
+              }
+            ],
+            "partitionKeys": [],
+            "db": {
+              "typeName": "hive_db",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02@cm"
+              },
+              "relationshipType": "hive_table_db"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  },
+  {
+    "type": "ENTITY_CREATE_V2",
+    "user": "hive",
+    "entities": {
+      "entities": [
+        {
+          "typeName": "hive_table_ddl",
+          "attributes": {
+            "serviceType": "hive",
+            "qualifiedName": "cadb02.hh6_renamed@cm:1616450674247",
+            "execTime": 1616450674247,
+            "queryText": "ALTER TABLE hh6 RENAME TO hh6_renamed",
+            "name": "ALTER TABLE hh6 RENAME TO hh6_renamed",
+            "userName": "hive"
+          },
+          "guid": "-14529329955589467",
+          "isIncomplete": false,
+          "provenanceType": 0,
+          "version": 0,
+          "relationshipAttributes": {
+            "table": {
+              "typeName": "hive_table",
+              "uniqueAttributes": {
+                "qualifiedName": "cadb02.hh6_renamed@cm"
+              },
+              "relationshipType": "hive_table_ddl_queries"
+            }
+          },
+          "proxy": false
+        }
+      ]
+    }
+  }
+]
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasType.java b/intg/src/main/java/org/apache/atlas/type/AtlasType.java
index b30b483..c573877 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasType.java
@@ -143,6 +143,9 @@ public abstract class AtlasType {
         return AtlasJson.fromJson(jsonStr, type);
     }
 
+    public static <T> T fromLinkedHashMap(Object obj, Class<T> type) {
+        return AtlasJson.fromLinkedHashMap(obj, type);
+    }
     public static String toV1Json(Object obj) {
         return AtlasJson.toV1Json(obj);
     }
diff --git a/intg/src/main/java/org/apache/atlas/utils/AtlasJson.java b/intg/src/main/java/org/apache/atlas/utils/AtlasJson.java
index abeddf6..732c557 100644
--- a/intg/src/main/java/org/apache/atlas/utils/AtlasJson.java
+++ b/intg/src/main/java/org/apache/atlas/utils/AtlasJson.java
@@ -117,6 +117,20 @@ public class AtlasJson {
         return ret;
     }
 
+    public static <T> T fromLinkedHashMap(Object linkedHashMap, Class<T> type) {
+        T ret = null;
+
+        if (linkedHashMap != null) {
+            ret = mapper.convertValue(linkedHashMap, type);
+
+            if (ret instanceof Struct) {
+                ((Struct) ret).normalize();
+            }
+        }
+
+        return ret;
+    }
+
     public static <T> T fromJson(String jsonStr, Class<T> type) {
         T ret = null;