You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by su...@apache.org on 2016/05/09 21:27:30 UTC

incubator-atlas git commit: ATLAS-626 Hive temporary table metadata is captured in atlas (sumasai)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master a963e9806 -> 856ee6b5d


ATLAS-626 Hive temporary table metadata is captured in atlas (sumasai)


Project: http://git-wip-us.apache.org/repos/asf/incubator-atlas/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-atlas/commit/856ee6b5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/856ee6b5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/856ee6b5

Branch: refs/heads/master
Commit: 856ee6b5d7f7dd32c8e34c500c055bbb5db9f3cc
Parents: a963e98
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Mon May 9 14:27:11 2016 -0700
Committer: Suma Shivaprasad <su...@gmail.com>
Committed: Mon May 9 14:27:11 2016 -0700

----------------------------------------------------------------------
 .../atlas/hive/bridge/HiveMetaStoreBridge.java  | 67 +++++++++++++-------
 .../org/apache/atlas/hive/hook/HiveHook.java    | 30 ++++++---
 .../hive/bridge/HiveMetaStoreBridgeTest.java    |  4 +-
 .../org/apache/atlas/hive/hook/HiveHookIT.java  | 37 ++++++++---
 release-log.txt                                 |  3 +-
 5 files changed, 96 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/856ee6b5/addons/hive-bridge/src/main/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridge.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridge.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridge.java
index 8768250..675f211 100755
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridge.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridge.java
@@ -34,6 +34,7 @@ import org.apache.atlas.typesystem.json.TypesSerialization;
 import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.utils.AuthenticationUtil;
 import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -65,6 +67,8 @@ public class HiveMetaStoreBridge {
     public static final String DESCRIPTION_ATTR = "description";
     public static final String SEARCH_ENTRY_GUID_ATTR = "__guid";
 
+    public static final String TEMP_TABLE_PREFIX = "_temp-";
+
     private final String clusterName;
     public static final int MILLIS_CONVERT_FACTOR = 1000;
 
@@ -252,21 +256,20 @@ public class HiveMetaStoreBridge {
     /**
      * Gets reference for the table
      *
-     * @param dbName database name
-     * @param tableName table name
+     * @param hiveTable
      * @return table reference if exists, else null
      * @throws Exception
      */
-    private Referenceable getTableReference(String dbName, String tableName) throws Exception {
-        LOG.debug("Getting reference for table {}.{}", dbName, tableName);
+    private Referenceable getTableReference(Table hiveTable)  throws Exception {
+        LOG.debug("Getting reference for table {}.{}", hiveTable.getDbName(), hiveTable.getTableName());
 
         String typeName = HiveDataTypes.HIVE_TABLE.getName();
-        String dslQuery = getTableDSLQuery(getClusterName(), dbName, tableName, typeName);
+        String dslQuery = getTableDSLQuery(getClusterName(), hiveTable.getDbName(), hiveTable.getTableName(), typeName, hiveTable.isTemporary());
         return getEntityReferenceFromDSL(typeName, dslQuery);
     }
 
-    static String getTableDSLQuery(String clusterName, String dbName, String tableName, String typeName) {
-        String entityName = getTableQualifiedName(clusterName, dbName, tableName);
+    static String getTableDSLQuery(String clusterName, String dbName, String tableName, String typeName, boolean isTemporary) {
+        String entityName = getTableQualifiedName(clusterName, dbName, tableName, isTemporary);
         return String.format("%s as t where name = '%s'", typeName, entityName);
     }
 
@@ -277,8 +280,39 @@ public class HiveMetaStoreBridge {
      * @param tableName Name of the Hive table
      * @return Unique qualified name to identify the Table instance in Atlas.
      */
+    public static String getTableQualifiedName(String clusterName, String dbName, String tableName, boolean isTemporaryTable) {
+        String tableTempName = tableName;
+        if (isTemporaryTable) {
+            if (SessionState.get().getSessionId() != null) {
+                tableTempName = tableName + TEMP_TABLE_PREFIX + SessionState.get().getSessionId();
+            } else {
+                tableTempName = tableName + TEMP_TABLE_PREFIX + RandomStringUtils.random(10);
+            }
+        }
+        return String.format("%s.%s@%s", dbName.toLowerCase(), tableTempName.toLowerCase(), clusterName);
+    }
+
+
+
+    /**
+     * Construct the qualified name used to uniquely identify a Table instance in Atlas.
+     * @param clusterName Name of the cluster to which the Hive component belongs
+     * @param table hive table for which the qualified name is needed
+     * @return Unique qualified name to identify the Table instance in Atlas.
+     */
+    public static String getTableQualifiedName(String clusterName, Table table) {
+        return getTableQualifiedName(clusterName, table.getDbName(), table.getTableName(), table.isTemporary());
+    }
+
+    /**
+     * Construct the qualified name used to uniquely identify a Table instance in Atlas.
+     * @param clusterName Name of the cluster to which the Hive component belongs
+     * @param dbName Name of the Hive database to which the Table belongs
+     * @param tableName Name of the Hive table
+     * @return Unique qualified name to identify the Table instance in Atlas.
+     */
     public static String getTableQualifiedName(String clusterName, String dbName, String tableName) {
-        return String.format("%s.%s@%s", dbName.toLowerCase(), tableName.toLowerCase(), clusterName);
+         return getTableQualifiedName(clusterName, dbName, tableName, false);
     }
 
     /**
@@ -300,7 +334,8 @@ public class HiveMetaStoreBridge {
         if (tableReference == null) {
             tableReference = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
         }
-        String tableQualifiedName = getTableQualifiedName(clusterName, hiveTable.getDbName(), hiveTable.getTableName());
+
+        String tableQualifiedName = getTableQualifiedName(clusterName, hiveTable);
         tableReference.set(HiveDataModelGenerator.NAME, tableQualifiedName);
         tableReference.set(HiveDataModelGenerator.TABLE_NAME, hiveTable.getTableName().toLowerCase());
         tableReference.set(HiveDataModelGenerator.OWNER, hiveTable.getOwner());
@@ -361,7 +396,7 @@ public class HiveMetaStoreBridge {
         String dbName = table.getDbName();
         String tableName = table.getTableName();
         LOG.info("Attempting to register table [" + tableName + "]");
-        Referenceable tableReference = getTableReference(dbName, tableName);
+        Referenceable tableReference = getTableReference(table);
         if (tableReference == null) {
             tableReference = createTableInstance(dbReference, table);
             tableReference = registerInstance(tableReference);
@@ -395,18 +430,6 @@ public class HiveMetaStoreBridge {
         return new Referenceable(guid, typeName, null);
     }
 
-    private Referenceable getSDForTable(String dbName, String tableName) throws Exception {
-        Referenceable tableRef = getTableReference(dbName, tableName);
-        if (tableRef == null) {
-            throw new IllegalArgumentException("Table " + dbName + "." + tableName + " doesn't exist");
-        }
-
-        AtlasClient dgiClient = getAtlasClient();
-        Referenceable tableInstance = dgiClient.getEntity(tableRef.getId().id);
-        Referenceable sd = (Referenceable) tableInstance.get(HiveDataModelGenerator.STORAGE_DESC);
-        return new Referenceable(sd.getId().id, sd.getTypeName(), null);
-    }
-
     public Referenceable fillStorageDesc(StorageDescriptor storageDesc, String tableQualifiedName,
         String sdQualifiedName, Id tableId) throws Exception {
         LOG.debug("Filling storage descriptor information for " + storageDesc);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/856ee6b5/addons/hive-bridge/src/main/java/org/apache/atlas/hive/hook/HiveHook.java
----------------------------------------------------------------------
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 e5b0bf6..ffd5d22 100755
--- 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
@@ -370,7 +370,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
     }
 
     private void deleteTable(HiveMetaStoreBridge dgiBridge, HiveEventContext event, WriteEntity output) {
-        final String tblQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(dgiBridge.getClusterName(), output.getTable().getDbName(), output.getTable().getTableName());
+        final String tblQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(dgiBridge.getClusterName(), output.getTable());
         LOG.info("Deleting table {} ", tblQualifiedName);
         messages.add(
             new HookNotification.EntityDeleteRequest(event.getUser(),
@@ -413,13 +413,13 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
                 //Hive sends with both old and new table names in the outputs which is weird. So skipping that with the below check
                 if (!newTable.getDbName().equals(oldTable.getDbName()) || !newTable.getTableName().equals(oldTable.getTableName())) {
                     final String oldQualifiedName = dgiBridge.getTableQualifiedName(dgiBridge.getClusterName(),
-                        oldTable.getDbName(), oldTable.getTableName());
+                        oldTable);
                     final String newQualifiedName = dgiBridge.getTableQualifiedName(dgiBridge.getClusterName(),
-                        newTable.getDbName(), newTable.getTableName());
+                        newTable);
 
                     //Create/update old table entity - create entity with oldQFNme and old tableName if it doesnt exist. If exists, will update
                     //We always use the new entity while creating the table since some flags, attributes of the table are not set in inputEntity and Hive.getTable(oldTableName) also fails since the table doesnt exist in hive anymore
-                    final Referenceable tableEntity = createOrUpdateEntities(dgiBridge, event.getUser(), writeEntity);
+                    final Referenceable tableEntity = createOrUpdateEntities(dgiBridge, event.getUser(), writeEntity, true);
 
                     //Reset regular column QF Name to old Name and create a new partial notification request to replace old column QFName to newName to retain any existing traits
                     replaceColumnQFName(event, (List<Referenceable>) tableEntity.get(HiveDataModelGenerator.COLUMNS), oldQualifiedName, newQualifiedName);
@@ -493,7 +493,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
         return newSDEntity;
     }
 
-    private Referenceable createOrUpdateEntities(HiveMetaStoreBridge dgiBridge, String user, Entity entity) throws Exception {
+    private Referenceable createOrUpdateEntities(HiveMetaStoreBridge dgiBridge, String user, Entity entity, boolean skipTempTables) throws Exception {
         Database db = null;
         Table table = null;
         Partition partition = null;
@@ -524,8 +524,18 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
 
         if (table != null) {
             table = dgiBridge.hiveClient.getTable(table.getDbName(), table.getTableName());
-            tableEntity = dgiBridge.createTableInstance(dbEntity, table);
-            entities.add(tableEntity);
+            //If its an external table, even though the temp table skip flag is on,
+            // we create the table since we need the HDFS path to temp table lineage.
+            if (skipTempTables &&
+                table.isTemporary() &&
+                !TableType.EXTERNAL_TABLE.equals(table.getTableType())) {
+
+               LOG.debug("Skipping temporary table registration {} since it is not an external table {} ", table.getTableName(), table.getTableType().name());
+
+            } else {
+                tableEntity = dgiBridge.createTableInstance(dbEntity, table);
+                entities.add(tableEntity);
+            }
         }
 
         messages.add(new HookNotification.EntityUpdateRequest(user, entities));
@@ -536,7 +546,7 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
         List<Pair<? extends Entity, Referenceable>> entitiesCreatedOrUpdated = new ArrayList<>();
         for (Entity entity : event.getOutputs()) {
             if (entity.getType() == entityType) {
-                Referenceable entityCreatedOrUpdated = createOrUpdateEntities(dgiBridge, event.getUser(), entity);
+                Referenceable entityCreatedOrUpdated = createOrUpdateEntities(dgiBridge, event.getUser(), entity, true);
                 if (entitiesCreatedOrUpdated != null) {
                     entitiesCreatedOrUpdated.add(Pair.of(entity, entityCreatedOrUpdated));
                 }
@@ -600,9 +610,9 @@ public class HiveHook extends AtlasHook implements ExecuteWithHookContext {
 
     private void processHiveEntity(HiveMetaStoreBridge dgiBridge, HiveEventContext event, Entity entity, Map<String, Referenceable> dataSets) throws Exception {
         if (entity.getType() == Type.TABLE || entity.getType() == Type.PARTITION) {
-            final String tblQFName = dgiBridge.getTableQualifiedName(dgiBridge.getClusterName(), entity.getTable().getDbName(), entity.getTable().getTableName());
+            final String tblQFName = dgiBridge.getTableQualifiedName(dgiBridge.getClusterName(), entity.getTable());
             if (!dataSets.containsKey(tblQFName)) {
-                Referenceable inTable = createOrUpdateEntities(dgiBridge, event.getUser(), entity);
+                Referenceable inTable = createOrUpdateEntities(dgiBridge, event.getUser(), entity, false);
                 dataSets.put(tblQFName, inTable);
             }
         } else if (entity.getType() == Type.DFS_DIR) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/856ee6b5/addons/hive-bridge/src/test/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridgeTest.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridgeTest.java b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridgeTest.java
index dac5a18..dec5fcb 100644
--- a/addons/hive-bridge/src/test/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridgeTest.java
+++ b/addons/hive-bridge/src/test/java/org/apache/atlas/hive/bridge/HiveMetaStoreBridgeTest.java
@@ -96,7 +96,7 @@ public class HiveMetaStoreBridgeTest {
 
         // return existing table
         when(atlasClient.searchByDSL(HiveMetaStoreBridge.getTableDSLQuery(CLUSTER_NAME, TEST_DB_NAME, TEST_TABLE_NAME,
-                HiveDataTypes.HIVE_TABLE.getName()))).thenReturn(
+                HiveDataTypes.HIVE_TABLE.getName(), false))).thenReturn(
                 getEntityReference("82e06b34-9151-4023-aa9d-b82103a50e77"));
         when(atlasClient.getEntity("82e06b34-9151-4023-aa9d-b82103a50e77")).thenReturn(createTableReference());
 
@@ -138,7 +138,7 @@ public class HiveMetaStoreBridgeTest {
 
         when(atlasClient.searchByDSL(HiveMetaStoreBridge.getTableDSLQuery(CLUSTER_NAME, TEST_DB_NAME,
             TEST_TABLE_NAME,
-            HiveDataTypes.HIVE_TABLE.getName()))).thenReturn(
+            HiveDataTypes.HIVE_TABLE.getName(), false))).thenReturn(
             getEntityReference("82e06b34-9151-4023-aa9d-b82103a50e77"));
         when(atlasClient.getEntity("82e06b34-9151-4023-aa9d-b82103a50e77")).thenReturn(createTableReference());
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/856ee6b5/addons/hive-bridge/src/test/java/org/apache/atlas/hive/hook/HiveHookIT.java
----------------------------------------------------------------------
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 e68a8f3..da21195 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
@@ -168,6 +168,7 @@ public class HiveHookIT {
         }
         runCommand("create " + (isExternal ? " EXTERNAL " : "") + (isTemporary ? "TEMPORARY " : "") + "table " + tableName + "(id int, name string) comment 'table comment' " + (isPartitioned ?
             " partitioned by(dt string)" : "") + location);
+
         return tableName;
     }
 
@@ -234,9 +235,9 @@ public class HiveHookIT {
         String colName = columnName();
 
         String pFile = createTestDFSPath("parentPath");
-        final String query = String.format("create EXTERNAL table %s.%s( %s, %s) location '%s'", dbName , tableName , colName + " int", "name string",  pFile);
+        final String query = String.format("create TEMPORARY EXTERNAL table %s.%s( %s, %s) location '%s'", dbName , tableName , colName + " int", "name string",  pFile);
         runCommand(query);
-        String tableId = assertTableIsRegistered(dbName, tableName);
+        String tableId = assertTableIsRegistered(dbName, tableName, null, true);
 
         Referenceable processReference = validateProcess(query, 1, 1);
         assertEquals(processReference.get("userName"), UserGroupInformation.getCurrentUser().getShortUserName());
@@ -270,7 +271,7 @@ public class HiveHookIT {
     private String assertColumnIsRegistered(String colName, AssertPredicate assertPredicate) throws Exception {
         LOG.debug("Searching for column {}", colName);
         return assertEntityIsRegistered(HiveDataTypes.HIVE_COLUMN.getName(), AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
-                colName, assertPredicate);
+            colName, assertPredicate);
     }
 
     private String assertSDIsRegistered(String sdQFName, AssertPredicate assertPredicate) throws Exception {
@@ -282,7 +283,7 @@ public class HiveHookIT {
     private void assertColumnIsNotRegistered(String colName) throws Exception {
         LOG.debug("Searching for column {}", colName);
         assertEntityIsNotRegistered(HiveDataTypes.HIVE_COLUMN.getName(), AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
-                colName);
+            colName);
     }
 
     @Test
@@ -487,6 +488,8 @@ public class HiveHookIT {
     public void testInsertIntoTempTable() throws Exception {
         String tableName = createTable();
         String insertTableName = createTable(false, false, true);
+        assertTableIsRegistered(DEFAULT_DB, tableName);
+        assertTableIsNotRegistered(DEFAULT_DB, insertTableName, true);
 
         String query =
             "insert into " + insertTableName + " select id, name from " + tableName;
@@ -495,8 +498,8 @@ public class HiveHookIT {
         validateProcess(query, 1, 1);
 
         String ipTableId = assertTableIsRegistered(DEFAULT_DB, tableName);
-        String opTableId = assertTableIsRegistered(DEFAULT_DB, insertTableName);
-        validateProcess(query, new String[]{ipTableId}, new String[]{opTableId});
+        String opTableId = assertTableIsRegistered(DEFAULT_DB, insertTableName, null, true);
+        validateProcess(query, new String[] {ipTableId}, new String[] {opTableId});
     }
 
     @Test
@@ -1315,9 +1318,15 @@ public class HiveHookIT {
         assertEntityIsNotRegistered(HiveDataTypes.HIVE_PROCESS.getName(), AtlasClient.NAME, normalize(queryStr));
     }
 
+    private void assertTableIsNotRegistered(String dbName, String tableName, boolean isTemporaryTable) throws Exception {
+        LOG.debug("Searching for table {}.{}", dbName, tableName);
+        String tableQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, dbName, tableName, isTemporaryTable);
+        assertEntityIsNotRegistered(HiveDataTypes.HIVE_TABLE.getName(), AtlasClient.NAME, tableQualifiedName);
+    }
+
     private void assertTableIsNotRegistered(String dbName, String tableName) throws Exception {
         LOG.debug("Searching for table {}.{}", dbName, tableName);
-        String tableQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, dbName, tableName);
+        String tableQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, dbName, tableName, false);
         assertEntityIsNotRegistered(HiveDataTypes.HIVE_TABLE.getName(), AtlasClient.NAME, tableQualifiedName);
     }
 
@@ -1328,16 +1337,24 @@ public class HiveHookIT {
     }
 
     private String assertTableIsRegistered(String dbName, String tableName) throws Exception {
-        return assertTableIsRegistered(dbName, tableName, null);
+        return assertTableIsRegistered(dbName, tableName, null, false);
     }
 
-    private String assertTableIsRegistered(String dbName, String tableName, AssertPredicate assertPredicate) throws Exception {
+    private String assertTableIsRegistered(String dbName, String tableName, boolean isTemporary) throws Exception {
+        return assertTableIsRegistered(dbName, tableName, null, isTemporary);
+    }
+
+    private String assertTableIsRegistered(String dbName, String tableName, AssertPredicate assertPredicate, boolean isTemporary) throws Exception {
         LOG.debug("Searching for table {}.{}", dbName, tableName);
-        String tableQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, dbName, tableName);
+        String tableQualifiedName = HiveMetaStoreBridge.getTableQualifiedName(CLUSTER_NAME, dbName, tableName, isTemporary);
         return assertEntityIsRegistered(HiveDataTypes.HIVE_TABLE.getName(), AtlasClient.NAME, tableQualifiedName,
                 assertPredicate);
     }
 
+    private String assertTableIsRegistered(String dbName, String tableName, AssertPredicate assertPredicate) throws Exception {
+        return assertTableIsRegistered(dbName, tableName, assertPredicate, false);
+    }
+
     private String assertDatabaseIsRegistered(String dbName) throws Exception {
         return assertDatabaseIsRegistered(dbName, null);
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/856ee6b5/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 1afc1ed..152314c 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -20,6 +20,7 @@ ATLAS-409 Atlas will not import avro tables with schema read from a file (dosset
 ATLAS-379 Create sqoop and falcon metadata addons (venkatnrangan,bvellanki,sowmyaramesh via shwethags)
 
 ALL CHANGES:
+ATLAS-626 Hive temporary table metadata is captured in atlas (sumasai)
 ATLAS-747 Hive CTAS entity registration fails because userName is null (shwethags)
 ATLAS-759 HiveHookIT.testAlterTableChangeColumn is consistently failing on master (yhemanth)
 ATLAS-690 Read timed out exceptions when tables are imported into Atlas (yhemanth via shwethags)
@@ -37,7 +38,7 @@ ATLAS-543 Entity Instance requests should not require ID element for new Entitie
 ATLAS-681 update committer/ppmc members in the pom.xml (sneethiraj via shwethags)
 ATLAS-616 Resolve OOM - Zookeeper throws exceptions when trying to fire DSL queries at Atlas at large scale. (yhemanth via sumasai)
 ATLAS-530 Add table information to column class (sumasai)
-ATLAS-538 Rename table should retain traits/tags assigned to columns/storage descriptors (sumasai)
+ATLAS-583 Rename table should retain traits/tags assigned to columns/storage descriptors (sumasai)
 ATLAS-628 Starting two Atlas instances at the same time causes exceptions in HA mode (yhemanth via sumasai)
 ATLAS-594 alter table rename doesnt work across databases (sumasai via shwethags)
 ATLAS-586 While updating the multiple attributes, Atlas returns the response with escape characters (dkantor via shwethags)