You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sh...@apache.org on 2015/09/22 11:15:04 UTC

[7/7] incubator-atlas git commit: ATLAS-58 Make hive hook reliable (shwethags)

ATLAS-58 Make hive hook reliable (shwethags)


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

Branch: refs/heads/master
Commit: 1bfda02a11762cee733bf5c5b1b82e1e256acda0
Parents: 016e36c
Author: Shwetha GS <ss...@hortonworks.com>
Authored: Tue Sep 22 14:44:42 2015 +0530
Committer: Shwetha GS <ss...@hortonworks.com>
Committed: Tue Sep 22 14:44:42 2015 +0530

----------------------------------------------------------------------
 addons/hive-bridge/pom.xml                      |  41 +-
 addons/hive-bridge/src/bin/import-hive.sh       |   2 +-
 .../atlas/hive/bridge/HiveMetaStoreBridge.java  | 408 +++++++++----------
 .../org/apache/atlas/hive/hook/HiveHook.java    | 259 +++++++-----
 .../hive/model/HiveDataModelGenerator.java      |  65 +--
 .../src/site/twiki/Bridge-Hive.twiki            |  72 ----
 .../org/apache/atlas/hive/hook/HiveHookIT.java  | 112 +++--
 client/pom.xml                                  |   5 -
 .../org/apache/atlas/ApplicationProperties.java |  86 ----
 .../main/java/org/apache/atlas/AtlasClient.java | 153 +++++--
 .../apache/atlas/ApplicationPropertiesTest.java |  55 ---
 .../src/test/resources/application.properties   |  29 --
 common/pom.xml                                  |  45 ++
 .../java/org/apache/atlas/service/Service.java  |  31 ++
 .../java/org/apache/atlas/service/Services.java |  59 +++
 .../public/modules/details/detailsResource.js   |   2 +-
 .../public/modules/search/searchController.js   |   4 +-
 distro/pom.xml                                  |  50 ++-
 distro/src/bin/atlas_start.py                   |   2 +-
 distro/src/conf/application.properties          |   9 +-
 .../src/main/assemblies/standalone-package.xml  |  20 +-
 distro/src/test/python/scripts/TestMetadata.py  |   4 +-
 docs/pom.xml                                    |  63 +--
 docs/src/site/resources/architecture.png        | Bin 58775 -> 0 bytes
 docs/src/site/resources/data-types.png          | Bin 413738 -> 0 bytes
 docs/src/site/resources/guide-class-diagram.png | Bin 40375 -> 0 bytes
 .../src/site/resources/guide-instance-graph.png | Bin 179941 -> 0 bytes
 .../resources/images/twiki/architecture.png     | Bin 0 -> 58775 bytes
 .../site/resources/images/twiki/data-types.png  | Bin 0 -> 413738 bytes
 .../images/twiki/guide-class-diagram.png        | Bin 0 -> 40375 bytes
 .../images/twiki/guide-instance-graph.png       | Bin 0 -> 179941 bytes
 .../resources/images/twiki/notification.png     | Bin 0 -> 137448 bytes
 .../resources/images/twiki/types-instance.png   | Bin 0 -> 445893 bytes
 docs/src/site/resources/types-instance.png      | Bin 445893 -> 0 bytes
 docs/src/site/site.xml                          |  15 +-
 docs/src/site/twiki/Architecture.twiki          |  29 +-
 docs/src/site/twiki/Bridge-Hive.twiki           |  83 ++++
 docs/src/site/twiki/Configuration.twiki         |  20 +
 docs/src/site/twiki/InstallationSteps.twiki     |   2 +-
 docs/src/site/twiki/QuickStart.twiki            |   4 +-
 docs/src/site/twiki/TypeSystem.twiki            |   4 +-
 docs/src/site/twiki/index.twiki                 |   4 +-
 notification/pom.xml                            |  11 +-
 .../apache/atlas/kafka/KafkaNotification.java   | 121 +++---
 .../atlas/kafka/KafkaNotificationProvider.java  |  41 ++
 .../notification/NotificationHookConsumer.java  |  49 ++-
 .../notification/NotificationInterface.java     |  31 +-
 .../atlas/notification/NotificationModule.java  |  12 +-
 .../atlas/kafka/KafkaNotificationTest.java      |  13 +-
 pom.xml                                         |  19 +-
 release-log.txt                                 |   1 +
 .../atlas/GraphTransactionInterceptor.java      |   1 +
 .../apache/atlas/RepositoryMetadataModule.java  |   5 +-
 .../atlas/listener/EntityChangeListener.java    |   6 +-
 .../atlas/repository/EntityExistsException.java |  32 ++
 .../atlas/repository/MetadataRepository.java    |  15 +-
 .../graph/GraphBackedMetadataRepository.java    | 254 +++++++-----
 .../atlas/repository/graph/GraphHelper.java     |  37 +-
 .../atlas/services/DefaultMetadataService.java  | 130 ++++--
 .../apache/atlas/services/MetadataService.java  |  12 +-
 .../atlas/RepositoryServiceLoadingTest.java     |   3 -
 .../test/java/org/apache/atlas/TestUtils.java   | 123 +++++-
 .../GraphBackedDiscoveryServiceTest.java        |   4 +-
 .../atlas/discovery/HiveLineageServiceTest.java |   8 +-
 .../org/apache/atlas/repository/BaseTest.java   | 159 --------
 .../GraphBackedMetadataRepositoryTest.java      | 178 ++------
 .../graph/GraphRepoMapperScaleTest.java         | 126 ++----
 .../atlas/repository/memory/ClassTest.java      |  76 ----
 .../atlas/repository/memory/EnumTest.java       | 293 -------------
 .../repository/memory/InstanceE2ETest.java      | 157 -------
 .../atlas/repository/memory/StorageTest.java    | 127 ------
 .../atlas/repository/memory/StructTest.java     | 111 -----
 .../atlas/repository/memory/TraitTest.java      | 203 ---------
 .../service/DefaultMetadataServiceTest.java     | 157 +++++++
 .../apache/atlas/query/HiveTitanSample.scala    |   7 +-
 .../apache/atlas/query/QueryTestsUtils.scala    |   3 +-
 typesystem/pom.xml                              |   5 +
 .../org/apache/atlas/ApplicationProperties.java |  86 ++++
 .../org/apache/atlas/TypeExistsException.java   |  25 ++
 .../typesystem/persistence/StructInstance.java  |   6 +-
 .../typesystem/types/AttributeDefinition.java   |   1 +
 .../atlas/typesystem/types/AttributeInfo.java   |   1 +
 .../typesystem/types/ObjectGraphWalker.java     |   6 +-
 .../atlas/typesystem/types/TypeSystem.java      |  69 ++--
 .../types/ValueConversionException.java         |   7 +
 .../src/main/resources/application.properties   |  10 +-
 typesystem/src/main/resources/atlas-log4j.xml   |  54 +++
 .../typesystem/json/TypesSerialization.scala    |   2 +-
 .../apache/atlas/ApplicationPropertiesTest.java |  55 +++
 .../typesystem/json/SerializationJavaTest.java  |  11 +-
 .../apache/atlas/typesystem/types/BaseTest.java |   3 +-
 .../atlas/typesystem/types/TypeSystemTest.java  |  13 +-
 .../src/test/resources/application.properties   |  29 ++
 .../typesystem/json/SerializationTest.scala     |   2 +-
 .../json/TypesSerializationTest.scala           |   6 +-
 webapp/pom.xml                                  |  55 ++-
 webapp/src/main/java/org/apache/atlas/Main.java |   1 +
 .../org/apache/atlas/examples/QuickStart.java   |  16 +-
 .../atlas/web/listeners/GuiceServletConfig.java |  22 +-
 .../atlas/web/resources/EntitiesResource.java   | 133 ++++++
 .../atlas/web/resources/EntityResource.java     |  80 ++--
 .../resources/MetadataDiscoveryResource.java    |  21 +-
 .../atlas/web/resources/TypesResource.java      |   4 +
 .../atlas/web/service/SecureEmbeddedServer.java |   2 -
 webapp/src/main/resources/atlas-log4j.xml       |   1 -
 .../NotificationHookConsumerIT.java             | 103 +++++
 .../web/listeners/TestGuiceServletConfig.java   |  28 +-
 .../atlas/web/resources/BaseResourceIT.java     | 191 ++++++++-
 .../web/resources/EntityJerseyResourceIT.java   | 240 ++---------
 .../resources/HiveLineageJerseyResourceIT.java  | 115 +-----
 .../MetadataDiscoveryJerseyResourceIT.java      |   6 +-
 .../atlas/web/security/BaseSecurityTest.java    |   1 +
 .../web/security/NegativeSSLAndKerberosIT.java  | 127 ------
 .../security/NegativeSSLAndKerberosTest.java    | 127 ++++++
 .../atlas/web/security/SSLAndKerberosIT.java    | 163 --------
 .../atlas/web/security/SSLAndKerberosTest.java  | 163 ++++++++
 .../org/apache/atlas/web/security/SSLIT.java    | 137 -------
 .../org/apache/atlas/web/security/SSLTest.java  | 137 +++++++
 .../web/service/SecureEmbeddedServerIT.java     |  63 ---
 .../web/service/SecureEmbeddedServerITBase.java | 201 ---------
 .../web/service/SecureEmbeddedServerTest.java   |  65 +++
 .../service/SecureEmbeddedServerTestBase.java   | 205 ++++++++++
 122 files changed, 3497 insertions(+), 3563 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/addons/hive-bridge/pom.xml
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/pom.xml b/addons/hive-bridge/pom.xml
index 914d8c6..8a838ef 100755
--- a/addons/hive-bridge/pom.xml
+++ b/addons/hive-bridge/pom.xml
@@ -33,7 +33,7 @@
     <packaging>jar</packaging>
 
     <properties>
-        <hive.version>1.2.0</hive.version>
+        <hive.version>1.2.1</hive.version>
         <calcite.version>0.9.2-incubating</calcite.version>
     </properties>
 
@@ -98,6 +98,11 @@
             <artifactId>atlas-client</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-notification</artifactId>
+        </dependency>
+
         <!-- to bring up atlas server for integration tests -->
         <dependency>
             <groupId>org.apache.atlas</groupId>
@@ -149,7 +154,7 @@
                         </configuration>
                     </execution>
                     <execution>
-                        <id>copy</id>
+                        <id>copy-hook</id>
                         <phase>package</phase>
                         <goals>
                             <goal>copy</goal>
@@ -209,6 +214,16 @@
                                     <version>${project.version}</version>
                                 </artifactItem>
                                 <artifactItem>
+                                    <groupId>${project.groupId}</groupId>
+                                    <artifactId>atlas-notification</artifactId>
+                                    <version>${project.version}</version>
+                                </artifactItem>
+                                <artifactItem>
+                                    <groupId>${project.groupId}</groupId>
+                                    <artifactId>atlas-common</artifactId>
+                                    <version>${project.version}</version>
+                                </artifactItem>
+                                <artifactItem>
                                     <groupId>org.scala-lang</groupId>
                                     <artifactId>scala-compiler</artifactId>
                                     <version>${scala.version}</version>
@@ -228,6 +243,21 @@
                                     <artifactId>scalap</artifactId>
                                     <version>${scala.version}</version>
                                 </artifactItem>
+                                <artifactItem>
+                                    <groupId>com.google.inject.extensions</groupId>
+                                    <artifactId>guice-multibindings</artifactId>
+                                    <version>${guice.version}</version>
+                                </artifactItem>
+                                <artifactItem>
+                                    <groupId>org.apache.kafka</groupId>
+                                    <artifactId>kafka_${scala.binary.version}</artifactId>
+                                    <version>${kafka.version}</version>
+                                </artifactItem>
+                                <artifactItem>
+                                    <groupId>org.apache.kafka</groupId>
+                                    <artifactId>kafka-clients</artifactId>
+                                    <version>${kafka.version}</version>
+                                </artifactItem>
                             </artifactItems>
                         </configuration>
                     </execution>
@@ -253,12 +283,16 @@
                     <useTestScope>true</useTestScope>
                     <systemProperties>
                         <systemProperty>
+                            <name>log4j.configuration</name>
+                            <value>atlas-log4j.xml</value>
+                        </systemProperty>
+                        <systemProperty>
                             <name>atlas.log.dir</name>
                             <value>${project.build.directory}/logs</value>
                         </systemProperty>
                     </systemProperties>
                     <stopKey>atlas-stop</stopKey>
-                    <stopPort>41001</stopPort>
+                    <stopPort>21001</stopPort>
                 </configuration>
                 <executions>
                     <execution>
@@ -302,7 +336,6 @@
                 <configuration>
                     <generateProjectInfo>false</generateProjectInfo>
                     <generateReports>false</generateReports>
-                    <skip>false</skip>
                 </configuration>
             </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/addons/hive-bridge/src/bin/import-hive.sh
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/bin/import-hive.sh b/addons/hive-bridge/src/bin/import-hive.sh
index 99300e8..8cb1868 100755
--- a/addons/hive-bridge/src/bin/import-hive.sh
+++ b/addons/hive-bridge/src/bin/import-hive.sh
@@ -67,7 +67,7 @@ done
 METADATA_LOG_DIR="${METADATA_LOG_DIR:-$BASEDIR/logs}"
 export METADATA_LOG_DIR
 
-JAVA_PROPERTIES="$METADATA_OPTS -Datlas.log.dir=$METADATA_LOG_DIR -Datlas.log.file=import-hive.log"
+JAVA_PROPERTIES="$METADATA_OPTS -Datlas.log.dir=$METADATA_LOG_DIR -Datlas.log.file=import-hive.log -Dlog4j.configuration=atlas-log4j.xml"
 shift
 
 while [[ ${1} =~ ^\-D ]]; do

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/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 4d1af4e..a4d5330 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
@@ -18,6 +18,7 @@
 
 package org.apache.atlas.hive.bridge;
 
+import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasServiceException;
 import org.apache.atlas.hive.model.HiveDataModelGenerator;
@@ -25,18 +26,17 @@ import org.apache.atlas.hive.model.HiveDataTypes;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.Struct;
 import org.apache.atlas.typesystem.json.InstanceSerialization;
-import org.apache.atlas.typesystem.persistence.Id;
-import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 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.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 /**
  * A Bridge Utility that imports metadata from the Hive Meta Store
@@ -64,21 +63,23 @@ public class HiveMetaStoreBridge {
 
     private static final Logger LOG = LoggerFactory.getLogger(HiveMetaStoreBridge.class);
 
-    private final Hive hiveClient;
+    public final Hive hiveClient;
     private final AtlasClient atlasClient;
 
-    public HiveMetaStoreBridge(HiveConf hiveConf) throws Exception {
-        this(hiveConf, null, null);
+    public HiveMetaStoreBridge(HiveConf hiveConf, Configuration atlasConf) throws Exception {
+        this(hiveConf, atlasConf, null, null);
     }
 
     /**
      * Construct a HiveMetaStoreBridge.
      * @param hiveConf hive conf
      */
-    public HiveMetaStoreBridge(HiveConf hiveConf, String doAsUser, UserGroupInformation ugi) throws Exception {
+    public HiveMetaStoreBridge(HiveConf hiveConf, Configuration atlasConf, String doAsUser,
+                               UserGroupInformation ugi) throws Exception {
         clusterName = hiveConf.get(HIVE_CLUSTER_NAME, DEFAULT_CLUSTER_NAME);
         hiveClient = Hive.get(hiveConf);
-        atlasClient = new AtlasClient(hiveConf.get(ATLAS_ENDPOINT, DEFAULT_DGI_URL), ugi, doAsUser);
+
+        atlasClient = new AtlasClient(atlasConf.getString(ATLAS_ENDPOINT, DEFAULT_DGI_URL), ugi, doAsUser);
     }
 
     public AtlasClient getAtlasClient() {
@@ -95,72 +96,78 @@ public class HiveMetaStoreBridge {
         for (String databaseName : databases) {
             Referenceable dbReference = registerDatabase(databaseName);
 
-            importTables(databaseName, dbReference);
+            importTables(dbReference, databaseName);
         }
     }
 
-    public Referenceable registerDatabase(String databaseName) throws Exception {
-        Referenceable dbRef = getDatabaseReference(databaseName, clusterName);
-        if (dbRef == null) {
-            LOG.info("Importing objects from databaseName : " + databaseName);
-            Database hiveDB = hiveClient.getDatabase(databaseName);
-
-            dbRef = new Referenceable(HiveDataTypes.HIVE_DB.getName());
-            dbRef.set(HiveDataModelGenerator.NAME, hiveDB.getName().toLowerCase());
-            dbRef.set(HiveDataModelGenerator.CLUSTER_NAME, clusterName);
-            dbRef.set("description", hiveDB.getDescription());
-            dbRef.set("locationUri", hiveDB.getLocationUri());
-            dbRef.set("parameters", hiveDB.getParameters());
-            dbRef.set("ownerName", hiveDB.getOwnerName());
-            if (hiveDB.getOwnerType() != null) {
-                dbRef.set("ownerType", hiveDB.getOwnerType().getValue());
-            }
+    /**
+     * Creates db entity
+     * @param hiveDB
+     * @return
+     * @throws HiveException
+     */
+    public Referenceable createDBInstance(Database hiveDB) throws HiveException {
+        LOG.info("Importing objects from databaseName : " + hiveDB.getName());
+
+        Referenceable dbRef = new Referenceable(HiveDataTypes.HIVE_DB.getName());
+        String dbName = hiveDB.getName().toLowerCase();
+        dbRef.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, getDBQualifiedName(clusterName, dbName));
+        dbRef.set(HiveDataModelGenerator.NAME, dbName);
+        dbRef.set(HiveDataModelGenerator.CLUSTER_NAME, clusterName);
+        dbRef.set("description", hiveDB.getDescription());
+        dbRef.set("locationUri", hiveDB.getLocationUri());
+        dbRef.set("parameters", hiveDB.getParameters());
+        dbRef.set("ownerName", hiveDB.getOwnerName());
+        if (hiveDB.getOwnerType() != null) {
+            dbRef.set("ownerType", hiveDB.getOwnerType().getValue());
+        }
+        return dbRef;
+    }
 
-            dbRef = createInstance(dbRef);
+    /**
+     * Checks if db is already registered, else creates and registers db entity
+     * @param databaseName
+     * @return
+     * @throws Exception
+     */
+    private Referenceable registerDatabase(String databaseName) throws Exception {
+        Referenceable dbRef = getDatabaseReference(clusterName, databaseName);
+        if (dbRef == null) {
+            Database db = hiveClient.getDatabase(databaseName);
+            dbRef = createDBInstance(db);
+            dbRef = registerInstance(dbRef);
         } else {
             LOG.info("Database {} is already registered with id {}", databaseName, dbRef.getId().id);
         }
         return dbRef;
     }
 
-    public Referenceable createInstance(Referenceable referenceable) throws Exception {
+    /**
+     * Registers an entity in atlas
+     * @param referenceable
+     * @return
+     * @throws Exception
+     */
+    public Referenceable registerInstance(Referenceable referenceable) throws Exception {
         String typeName = referenceable.getTypeName();
         LOG.debug("creating instance of type " + typeName);
 
         String entityJSON = InstanceSerialization.toJson(referenceable, true);
         LOG.debug("Submitting new entity {} = {}", referenceable.getTypeName(), entityJSON);
-        JSONObject jsonObject = atlasClient.createEntity(entityJSON);
-        String guid = jsonObject.getString(AtlasClient.GUID);
-        LOG.debug("created instance for type " + typeName + ", guid: " + guid);
+        JSONArray guids = atlasClient.createEntity(entityJSON);
+        LOG.debug("created instance for type " + typeName + ", guid: " + guids);
 
-        return new Referenceable(guid, referenceable.getTypeName(), null);
-    }
-
-    private void importTables(String databaseName, Referenceable databaseReferenceable) throws Exception {
-        List<String> hiveTables = hiveClient.getAllTables(databaseName);
-
-        for (String tableName : hiveTables) {
-            Referenceable tableReferenceable = registerTable(databaseReferenceable, databaseName, tableName);
-
-            // Import Partitions
-            Referenceable sdReferenceable = getSDForTable(databaseName, tableName);
-            registerPartitions(databaseName, tableName, tableReferenceable, sdReferenceable);
-
-            // Import Indexes
-            importIndexes(databaseName, tableName, databaseReferenceable, tableReferenceable);
-        }
+        return new Referenceable(guids.getString(0), referenceable.getTypeName(), null);
     }
 
     /**
-     * Gets reference for the database
-     *
-     *
+     * Gets reference to the atlas entity for the database
      * @param databaseName  database Name
      * @param clusterName    cluster name
      * @return Reference for database if exists, else null
      * @throws Exception
      */
-    private Referenceable getDatabaseReference(String databaseName, String clusterName) throws Exception {
+    private Referenceable getDatabaseReference(String clusterName, String databaseName) throws Exception {
         LOG.debug("Getting reference for database {}", databaseName);
         String typeName = HiveDataTypes.HIVE_DB.getName();
 
@@ -169,20 +176,6 @@ public class HiveMetaStoreBridge {
         return getEntityReferenceFromDSL(typeName, dslQuery);
     }
 
-    public Referenceable getProcessReference(String queryStr) throws Exception {
-        LOG.debug("Getting reference for process with query {}", queryStr);
-        String typeName = HiveDataTypes.HIVE_PROCESS.getName();
-
-        //todo enable DSL
-        //        String dslQuery = String.format("%s where queryText = \"%s\"", typeName, queryStr);
-        //        return getEntityReferenceFromDSL(typeName, dslQuery);
-
-        String gremlinQuery =
-                String.format("g.V.has('__typeName', '%s').has('%s.queryText', \"%s\").toList()", typeName, typeName,
-                        StringEscapeUtils.escapeJava(queryStr));
-        return getEntityReferenceFromGremlin(typeName, gremlinQuery);
-    }
-
     private Referenceable getEntityReferenceFromDSL(String typeName, String dslQuery) throws Exception {
         AtlasClient dgiClient = getAtlasClient();
         JSONArray results = dgiClient.searchByDSL(dslQuery);
@@ -200,8 +193,27 @@ public class HiveMetaStoreBridge {
         }
     }
 
-    public static String getTableName(String clusterName, String dbName, String tableName) {
-        return String.format("%s.%s@%s", dbName.toLowerCase(), tableName.toLowerCase(), clusterName);
+    public static String getDBQualifiedName(String clusterName, String dbName) {
+        return String.format("%s.%s", clusterName, dbName.toLowerCase());
+    }
+
+    /**
+     * Imports all tables for the given db
+     * @param databaseName
+     * @param databaseReferenceable
+     * @throws Exception
+     */
+    private void importTables(Referenceable databaseReferenceable, String databaseName) throws Exception {
+        List<String> hiveTables = hiveClient.getAllTables(databaseName);
+
+        for (String tableName : hiveTables) {
+            Table table = hiveClient.getTable(databaseName, tableName);
+            Referenceable tableReferenceable = registerTable(databaseReferenceable, table);
+
+            // Import Partitions
+            Referenceable sdReferenceable = getSDForTable(databaseName, tableName);
+            registerPartitions(tableReferenceable, sdReferenceable, table);
+        }
     }
 
     /**
@@ -216,16 +228,78 @@ public class HiveMetaStoreBridge {
         LOG.debug("Getting reference for table {}.{}", dbName, tableName);
 
         String typeName = HiveDataTypes.HIVE_TABLE.getName();
-        String entityName = getTableName(clusterName, dbName, tableName);
+        String entityName = getTableQualifiedName(clusterName, dbName, tableName);
         String dslQuery = String.format("%s as t where name = '%s'", typeName, entityName);
         return getEntityReferenceFromDSL(typeName, dslQuery);
     }
 
+    public static String getTableQualifiedName(String clusterName, String dbName, String tableName) {
+        return String.format("%s.%s.%s", clusterName, dbName.toLowerCase(), tableName.toLowerCase());
+    }
+
+    public Referenceable createTableInstance(Referenceable dbReference, Table hiveTable)
+            throws Exception {
+        LOG.info("Importing objects from {}.{}", hiveTable.getDbName(), hiveTable.getTableName());
+
+        Referenceable tableRef = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
+        String tableQualifiedName = getTableQualifiedName(clusterName, hiveTable.getDbName(), hiveTable.getTableName());
+        tableRef.set(HiveDataModelGenerator.NAME, tableQualifiedName);
+        tableRef.set(HiveDataModelGenerator.TABLE_NAME, hiveTable.getTableName().toLowerCase());
+        tableRef.set("owner", hiveTable.getOwner());
+
+        tableRef.set("createTime", hiveTable.getMetadata().getProperty(hive_metastoreConstants.DDL_TIME));
+        tableRef.set("lastAccessTime", hiveTable.getLastAccessTime());
+        tableRef.set("retention", hiveTable.getRetention());
+
+        tableRef.set(HiveDataModelGenerator.COMMENT, hiveTable.getParameters().get(HiveDataModelGenerator.COMMENT));
+
+        // add reference to the database
+        tableRef.set(HiveDataModelGenerator.DB, dbReference);
+
+        tableRef.set("columns", getColumns(hiveTable.getCols(), tableQualifiedName));
+
+        // add reference to the StorageDescriptor
+        Referenceable sdReferenceable = fillStorageDescStruct(hiveTable.getSd(), tableQualifiedName, tableQualifiedName);
+        tableRef.set("sd", sdReferenceable);
+
+        // add reference to the Partition Keys
+        List<Referenceable> partKeys = getColumns(hiveTable.getPartitionKeys(), tableQualifiedName);
+        tableRef.set("partitionKeys", partKeys);
+
+        tableRef.set("parameters", hiveTable.getParameters());
+
+        if (hiveTable.getViewOriginalText() != null) {
+            tableRef.set("viewOriginalText", hiveTable.getViewOriginalText());
+        }
+
+        if (hiveTable.getViewExpandedText() != null) {
+            tableRef.set("viewExpandedText", hiveTable.getViewExpandedText());
+        }
+
+        tableRef.set("tableType", hiveTable.getTableType().name());
+        tableRef.set("temporary", hiveTable.isTemporary());
+        return tableRef;
+    }
+
+    private Referenceable registerTable(Referenceable dbReference, Table table) throws Exception {
+        String dbName = table.getDbName();
+        String tableName = table.getTableName();
+        LOG.info("Attempting to register table [" + tableName + "]");
+        Referenceable tableRef = getTableReference(dbName, tableName);
+        if (tableRef == null) {
+            tableRef = createTableInstance(dbReference, table);
+            tableRef = registerInstance(tableRef);
+        } else {
+            LOG.info("Table {}.{} is already registered with id {}", dbName, tableName, tableRef.getId().id);
+        }
+        return tableRef;
+    }
+
+
     private Referenceable getEntityReferenceFromGremlin(String typeName, String gremlinQuery)
     throws AtlasServiceException, JSONException {
         AtlasClient client = getAtlasClient();
-        JSONObject response = client.searchByGremlin(gremlinQuery);
-        JSONArray results = response.getJSONArray(AtlasClient.RESULTS);
+        JSONArray results = client.searchByGremlin(gremlinQuery);
         if (results.length() == 0) {
             return null;
         }
@@ -245,7 +319,7 @@ public class HiveMetaStoreBridge {
         //                dbName, clusterName);
 
         String datasetType = AtlasClient.DATA_SET_SUPER_TYPE;
-        String tableEntityName = getTableName(clusterName, dbName, tableName);
+        String tableEntityName = getTableQualifiedName(clusterName, dbName, tableName);
 
         String gremlinQuery = String.format("g.V.has('__typeName', '%s').has('%s.values', %s).as('p')."
                         + "out('__%s.table').has('%s.name', '%s').back('p').toList()", typeName, typeName, valuesStr,
@@ -262,92 +336,24 @@ public class HiveMetaStoreBridge {
 
         AtlasClient dgiClient = getAtlasClient();
         Referenceable tableInstance = dgiClient.getEntity(tableRef.getId().id);
-        Id sdId = (Id) tableInstance.get("sd");
-        return new Referenceable(sdId.id, sdId.getTypeName(), null);
-    }
-
-    public Referenceable registerTable(String dbName, String tableName) throws Exception {
-        Referenceable dbReferenceable = registerDatabase(dbName);
-        return registerTable(dbReferenceable, dbName, tableName);
+        Referenceable sd = (Referenceable) tableInstance.get("sd");
+        return new Referenceable(sd.getId().id, sd.getTypeName(), null);
     }
 
-    public Referenceable registerTable(Referenceable dbReference, String dbName, String tableName) throws Exception {
-        LOG.info("Attempting to register table [" + tableName + "]");
-        Referenceable tableRef = getTableReference(dbName, tableName);
-        if (tableRef == null) {
-            LOG.info("Importing objects from " + dbName + "." + tableName);
-
-            Table hiveTable = hiveClient.getTable(dbName, tableName);
-
-            tableRef = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
-            tableRef.set(HiveDataModelGenerator.NAME,
-                    getTableName(clusterName, hiveTable.getDbName(), hiveTable.getTableName()));
-            tableRef.set(HiveDataModelGenerator.TABLE_NAME, hiveTable.getTableName().toLowerCase());
-            tableRef.set("owner", hiveTable.getOwner());
-
-            tableRef.set("createTime", hiveTable.getMetadata().getProperty(hive_metastoreConstants.DDL_TIME));
-            tableRef.set("lastAccessTime", hiveTable.getLastAccessTime());
-            tableRef.set("retention", hiveTable.getRetention());
-
-            tableRef.set(HiveDataModelGenerator.COMMENT, hiveTable.getParameters().get(HiveDataModelGenerator.COMMENT));
-
-            // add reference to the database
-            tableRef.set(HiveDataModelGenerator.DB, dbReference);
-
-            List<Referenceable> colList = getColumns(hiveTable.getCols());
-            tableRef.set("columns", colList);
-
-            // add reference to the StorageDescriptor
-            StorageDescriptor storageDesc = hiveTable.getSd();
-            Referenceable sdReferenceable = fillStorageDescStruct(storageDesc, colList);
-            tableRef.set("sd", sdReferenceable);
-
-            // add reference to the Partition Keys
-            List<Referenceable> partKeys = getColumns(hiveTable.getPartitionKeys());
-            tableRef.set("partitionKeys", partKeys);
-
-            tableRef.set("parameters", hiveTable.getParameters());
-
-            if (hiveTable.getViewOriginalText() != null) {
-                tableRef.set("viewOriginalText", hiveTable.getViewOriginalText());
-            }
-
-            if (hiveTable.getViewExpandedText() != null) {
-                tableRef.set("viewExpandedText", hiveTable.getViewExpandedText());
-            }
-
-            tableRef.set("tableType", hiveTable.getTableType().name());
-            tableRef.set("temporary", hiveTable.isTemporary());
-
+    private void registerPartitions(Referenceable tableReferenceable, Referenceable sdReferenceable,
+                                    Table table) throws Exception {
+        String dbName = table.getDbName();
+        String tableName = table.getTableName();
+        LOG.info("Registering partitions for {}.{}", dbName, tableName);
+        List<Partition> tableParts = hiveClient.getPartitions(table);
 
-            tableRef = createInstance(tableRef);
-        } else {
-            LOG.info("Table {}.{} is already registered with id {}", dbName, tableName, tableRef.getId().id);
-        }
-        return tableRef;
-    }
-
-    private void registerPartitions(String db, String tableName, Referenceable tableReferenceable,
-            Referenceable sdReferenceable) throws Exception {
-        Set<Partition> tableParts = hiveClient.getAllPartitionsOf(new Table(Table.getEmptyTable(db, tableName)));
-
-        if (tableParts.size() > 0) {
-            for (Partition hivePart : tableParts) {
-                registerPartition(hivePart, tableReferenceable, sdReferenceable);
-            }
+        for (Partition hivePart : tableParts) {
+            registerPartition(tableReferenceable, sdReferenceable, hivePart);
         }
     }
 
-    public Referenceable registerPartition(Partition partition) throws Exception {
-        String dbName = partition.getTable().getDbName();
-        String tableName = partition.getTable().getTableName();
-        Referenceable tableRef = registerTable(dbName, tableName);
-        Referenceable sdRef = getSDForTable(dbName, tableName);
-        return registerPartition(partition, tableRef, sdRef);
-    }
-
-    private Referenceable registerPartition(Partition hivePart, Referenceable tableReferenceable,
-            Referenceable sdReferenceable) throws Exception {
+    private Referenceable registerPartition(Referenceable tableReferenceable, Referenceable sdReferenceable,
+                                            Partition hivePart) throws Exception {
         LOG.info("Registering partition for {} with values {}", tableReferenceable,
                 StringUtils.join(hivePart.getValues(), ","));
         String dbName = hivePart.getTable().getDbName();
@@ -355,22 +361,8 @@ public class HiveMetaStoreBridge {
 
         Referenceable partRef = getPartitionReference(dbName, tableName, hivePart.getValues());
         if (partRef == null) {
-            partRef = new Referenceable(HiveDataTypes.HIVE_PARTITION.getName());
-            partRef.set("values", hivePart.getValues());
-
-            partRef.set(HiveDataModelGenerator.TABLE, tableReferenceable);
-
-            //todo fix
-            partRef.set("createTime", hivePart.getLastAccessTime());
-            partRef.set("lastAccessTime", hivePart.getLastAccessTime());
-
-            // sdStruct = fillStorageDescStruct(hivePart.getSd());
-            // Instead of creating copies of the sdstruct for partitions we are reusing existing
-            // ones will fix to identify partitions with differing schema.
-            partRef.set("sd", sdReferenceable);
-
-            partRef.set("parameters", hivePart.getParameters());
-            partRef = createInstance(partRef);
+            partRef = createPartitionReferenceable(tableReferenceable, sdReferenceable, hivePart);
+            partRef = registerInstance(partRef);
         } else {
             LOG.info("Partition {}.{} with values {} is already registered with id {}", dbName, tableName,
                     StringUtils.join(hivePart.getValues(), ","), partRef.getId().id);
@@ -378,47 +370,38 @@ public class HiveMetaStoreBridge {
         return partRef;
     }
 
-    private void importIndexes(String db, String table, Referenceable dbReferenceable, Referenceable tableReferenceable)
-    throws Exception {
-        List<Index> indexes = hiveClient.getIndexes(db, table, Short.MAX_VALUE);
-        if (indexes.size() > 0) {
-            for (Index index : indexes) {
-                importIndex(index, dbReferenceable, tableReferenceable);
-            }
-        }
-    }
-
-    //todo should be idempotent
-    private void importIndex(Index index, Referenceable dbReferenceable, Referenceable tableReferenceable)
-            throws Exception {
-        LOG.info("Importing index {} for {}.{}", index.getIndexName(), dbReferenceable, tableReferenceable);
-        Referenceable indexRef = new Referenceable(HiveDataTypes.HIVE_INDEX.getName());
-
-        indexRef.set(HiveDataModelGenerator.NAME, index.getIndexName());
-        indexRef.set("indexHandlerClass", index.getIndexHandlerClass());
-
-        indexRef.set(HiveDataModelGenerator.DB, dbReferenceable);
+    public Referenceable createPartitionReferenceable(Referenceable tableReferenceable, Referenceable sdReferenceable,
+                                                      Partition hivePart) {
+        Referenceable partRef = new Referenceable(HiveDataTypes.HIVE_PARTITION.getName());
+        partRef.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, getPartitionQualifiedName(hivePart));
+        partRef.set("values", hivePart.getValues());
 
-        indexRef.set("createTime", index.getCreateTime());
-        indexRef.set("lastAccessTime", index.getLastAccessTime());
-        indexRef.set("origTable", index.getOrigTableName());
-        indexRef.set("indexTable", index.getIndexTableName());
+        partRef.set(HiveDataModelGenerator.TABLE, tableReferenceable);
 
-        Referenceable sdReferenceable = fillStorageDescStruct(index.getSd(), null);
-        indexRef.set("sd", sdReferenceable);
+        //todo fix
+        partRef.set("createTime", hivePart.getLastAccessTime());
+        partRef.set("lastAccessTime", hivePart.getLastAccessTime());
 
-        indexRef.set("parameters", index.getParameters());
+        // sdStruct = fillStorageDescStruct(hivePart.getSd());
+        // Instead of creating copies of the sdstruct for partitions we are reusing existing
+        // ones will fix to identify partitions with differing schema.
+        partRef.set("sd", sdReferenceable);
 
-        tableReferenceable.set("deferredRebuild", index.isDeferredRebuild());
+        partRef.set("parameters", hivePart.getParameters());
+        return partRef;
+    }
 
-        createInstance(indexRef);
+    private String getPartitionQualifiedName(Partition partition) {
+        return String.format("%s.%s.%s.%s", clusterName, partition.getTable().getDbName(),
+                partition.getTable().getTableName(), StringUtils.join(partition.getValues(), "/"));
     }
 
-    private Referenceable fillStorageDescStruct(StorageDescriptor storageDesc, List<Referenceable> colList)
-    throws Exception {
+    private Referenceable fillStorageDescStruct(StorageDescriptor storageDesc, String tableQualifiedName,
+                                                String sdQualifiedName) throws Exception {
         LOG.debug("Filling storage descriptor information for " + storageDesc);
 
         Referenceable sdReferenceable = new Referenceable(HiveDataTypes.HIVE_STORAGEDESC.getName());
+        sdReferenceable.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, sdQualifiedName);
 
         SerDeInfo serdeInfo = storageDesc.getSerdeInfo();
         LOG.debug("serdeInfo = " + serdeInfo);
@@ -439,11 +422,7 @@ public class HiveMetaStoreBridge {
         //Use the passed column list if not null, ex: use same references for table and SD
         List<FieldSchema> columns = storageDesc.getCols();
         if (columns != null && !columns.isEmpty()) {
-            if (colList != null) {
-                sdReferenceable.set("cols", colList);
-            } else {
-                sdReferenceable.set("cols", getColumns(columns));
-            }
+            sdReferenceable.set("cols", getColumns(columns, tableQualifiedName));
         }
 
         List<Struct> sortColsStruct = new ArrayList<>();
@@ -471,19 +450,25 @@ public class HiveMetaStoreBridge {
         sdReferenceable.set("parameters", storageDesc.getParameters());
         sdReferenceable.set("storedAsSubDirectories", storageDesc.isStoredAsSubDirectories());
 
-        return createInstance(sdReferenceable);
+        return sdReferenceable;
+    }
+
+    private String getColumnQualifiedName(String tableQualifiedName, String colName) {
+        return String.format("%s.%s", tableQualifiedName, colName);
     }
 
-    private List<Referenceable> getColumns(List<FieldSchema> schemaList) throws Exception {
+    private List<Referenceable> getColumns(List<FieldSchema> schemaList, String tableQualifiedName) throws Exception {
         List<Referenceable> colList = new ArrayList<>();
         for (FieldSchema fs : schemaList) {
             LOG.debug("Processing field " + fs);
             Referenceable colReferenceable = new Referenceable(HiveDataTypes.HIVE_COLUMN.getName());
+            colReferenceable.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
+                    getColumnQualifiedName(tableQualifiedName, fs.getName()));
             colReferenceable.set(HiveDataModelGenerator.NAME, fs.getName());
             colReferenceable.set("type", fs.getType());
             colReferenceable.set(HiveDataModelGenerator.COMMENT, fs.getComment());
 
-            colList.add(createInstance(colReferenceable));
+            colList.add(colReferenceable);
         }
         return colList;
     }
@@ -502,16 +487,9 @@ public class HiveMetaStoreBridge {
     }
 
     public static void main(String[] argv) throws Exception {
-        HiveMetaStoreBridge hiveMetaStoreBridge = new HiveMetaStoreBridge(new HiveConf());
+        Configuration atlasConf = ApplicationProperties.get(ApplicationProperties.CLIENT_PROPERTIES);
+        HiveMetaStoreBridge hiveMetaStoreBridge = new HiveMetaStoreBridge(new HiveConf(), atlasConf);
         hiveMetaStoreBridge.registerHiveDataModel();
         hiveMetaStoreBridge.importHiveMetadata();
     }
-
-    public void updateTable(Referenceable tableReferenceable, Table newTable) throws AtlasServiceException {
-        AtlasClient client = getAtlasClient();
-        client.updateEntity(tableReferenceable.getId()._getId(), HiveDataModelGenerator.TABLE_NAME,
-                newTable.getTableName().toLowerCase());
-        client.updateEntity(tableReferenceable.getId()._getId(), HiveDataModelGenerator.NAME,
-                getTableName(clusterName, newTable.getDbName(), newTable.getTableName()));
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/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 fe26446..7ab1b35 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
@@ -20,28 +20,44 @@ package org.apache.atlas.hive.hook;
 
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.inject.Guice;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasException;
 import org.apache.atlas.hive.bridge.HiveMetaStoreBridge;
 import org.apache.atlas.hive.model.HiveDataTypes;
+import org.apache.atlas.notification.NotificationInterface;
+import org.apache.atlas.notification.NotificationModule;
 import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.json.InstanceSerialization;
+import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.hooks.Entity;
+import org.apache.hadoop.hive.ql.hooks.Entity.Type;
 import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.codehaus.jettison.json.JSONArray;
 import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -55,31 +71,48 @@ public class HiveHook implements ExecuteWithHookContext {
 
     private static final Logger LOG = LoggerFactory.getLogger(HiveHook.class);
 
-    // wait time determines how long we wait before we exit the jvm on
-    // shutdown. Pending requests after that will not be sent.
-    private static final int WAIT_TIME = 3;
-    private static ExecutorService executor;
-
     public static final String CONF_PREFIX = "atlas.hook.hive.";
-
     private static final String MIN_THREADS = CONF_PREFIX + "minThreads";
     private static final String MAX_THREADS = CONF_PREFIX + "maxThreads";
     private static final String KEEP_ALIVE_TIME = CONF_PREFIX + "keepAliveTime";
     public static final String CONF_SYNC = CONF_PREFIX + "synchronous";
 
+    public static final String HOOK_NUM_RETRIES = CONF_PREFIX + "numRetries";
+
+    // wait time determines how long we wait before we exit the jvm on
+    // shutdown. Pending requests after that will not be sent.
+    private static final int WAIT_TIME = 3;
+    private static ExecutorService executor;
+
     private static final int minThreadsDefault = 5;
     private static final int maxThreadsDefault = 5;
     private static final long keepAliveTimeDefault = 10;
+
     private static boolean typesRegistered = false;
+    private final Configuration atlasProperties;
+
+    class HiveEvent {
+        public HiveConf conf;
 
-    static {
-        // anything shared should be initialized here and destroyed in the
-        // shutdown hook The hook contract is weird in that it creates a
-        // boatload of hooks.
+        public Set<ReadEntity> inputs;
+        public Set<WriteEntity> outputs;
+
+        public String user;
+        public UserGroupInformation ugi;
+        public HiveOperation operation;
+        public QueryPlan queryPlan;
+        public HookContext.HookType hookType;
+        public JSONObject jsonPlan;
+    }
+
+    @Inject
+    private NotificationInterface notifInterface;
+
+    public HiveHook() throws AtlasException {
+        atlasProperties = ApplicationProperties.get(ApplicationProperties.CLIENT_PROPERTIES);
 
         // initialize the async facility to process hook calls. We don't
-        // want to do this inline since it adds plenty of overhead for the
-        // query.
+        // want to do this inline since it adds plenty of overhead for the query.
         HiveConf hiveConf = new HiveConf();
         int minThreads = hiveConf.getInt(MIN_THREADS, minThreadsDefault);
         int maxThreads = hiveConf.getInt(MAX_THREADS, maxThreadsDefault);
@@ -87,7 +120,7 @@ public class HiveHook implements ExecuteWithHookContext {
 
         executor = new ThreadPoolExecutor(minThreads, maxThreads, keepAliveTime, TimeUnit.MILLISECONDS,
                 new LinkedBlockingQueue<Runnable>(),
-                new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Atlas Logger %d").build());
+                new ThreadFactoryBuilder().setNameFormat("Atlas Logger %d").build());
 
         try {
             Runtime.getRuntime().addShutdownHook(new Thread() {
@@ -108,33 +141,16 @@ public class HiveHook implements ExecuteWithHookContext {
         }
 
         LOG.info("Created Atlas Hook");
-    }
-
-    class HiveEvent {
-        public HiveConf conf;
 
-        public Set<ReadEntity> inputs;
-        public Set<WriteEntity> outputs;
-
-        public String user;
-        public UserGroupInformation ugi;
-        public HiveOperation operation;
-        public QueryPlan queryPlan;
-        public HookContext.HookType hookType;
-        public JSONObject jsonPlan;
+        Injector injector = Guice.createInjector(new NotificationModule());
+        notifInterface = injector.getInstance(NotificationInterface.class);
     }
 
     @Override
     public void run(final HookContext hookContext) throws Exception {
-        if (executor == null) {
-            LOG.info("No executor running. Bail.");
-            return;
-        }
-
         // clone to avoid concurrent access
         final HiveEvent event = new HiveEvent();
         final HiveConf conf = new HiveConf(hookContext.getConf());
-        boolean debug = conf.get(CONF_SYNC, "false").equals("true");
 
         event.conf = conf;
         event.inputs = hookContext.getInputs();
@@ -148,7 +164,8 @@ public class HiveHook implements ExecuteWithHookContext {
 
         event.jsonPlan = getQueryPlan(event);
 
-        if (debug) {
+        boolean sync = conf.get(CONF_SYNC, "false").equals("true");
+        if (sync) {
             fireAndForget(event);
         } else {
             executor.submit(new Runnable() {
@@ -168,7 +185,7 @@ public class HiveHook implements ExecuteWithHookContext {
         assert event.hookType == HookContext.HookType.POST_EXEC_HOOK : "Non-POST_EXEC_HOOK not supported!";
 
         LOG.info("Entered Atlas hook for hook type {} operation {}", event.hookType, event.operation);
-        HiveMetaStoreBridge dgiBridge = new HiveMetaStoreBridge(event.conf, event.user, event.ugi);
+        HiveMetaStoreBridge dgiBridge = new HiveMetaStoreBridge(event.conf, atlasProperties, event.user, event.ugi);
 
         if (!typesRegistered) {
             dgiBridge.registerHiveDataModel();
@@ -177,11 +194,11 @@ public class HiveHook implements ExecuteWithHookContext {
 
         switch (event.operation) {
         case CREATEDATABASE:
-            handleCreateDB(dgiBridge, event);
+            handleEventOutputs(dgiBridge, event, Type.DATABASE);
             break;
 
         case CREATETABLE:
-            handleCreateTable(dgiBridge, event);
+            handleEventOutputs(dgiBridge, event, Type.TABLE);
             break;
 
         case CREATETABLE_AS_SELECT:
@@ -211,6 +228,7 @@ public class HiveHook implements ExecuteWithHookContext {
         }
     }
 
+    //todo re-write with notification
     private void renameTable(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception {
         //crappy, no easy of getting new name
         assert event.inputs != null && event.inputs.size() == 1;
@@ -232,30 +250,91 @@ public class HiveHook implements ExecuteWithHookContext {
             LOG.warn("Failed to deduct new name for " + event.queryPlan.getQueryStr());
             return;
         }
+    }
+
+    private Map<Type, Referenceable> createEntities(HiveMetaStoreBridge dgiBridge, Entity entity) throws Exception {
+        Map<Type, Referenceable> entities = new LinkedHashMap<>();
+        Database db = null;
+        Table table = null;
+        Partition partition = null;
+
+        switch (entity.getType()) {
+            case DATABASE:
+                db = entity.getDatabase();
+                break;
+
+            case TABLE:
+                table = entity.getTable();
+                db = dgiBridge.hiveClient.getDatabase(table.getDbName());
+                break;
+
+            case PARTITION:
+                partition = entity.getPartition();
+                table = partition.getTable();
+                db = dgiBridge.hiveClient.getDatabase(table.getDbName());
+                break;
+        }
+
+        db = dgiBridge.hiveClient.getDatabase(db.getName());
+        Referenceable dbReferenceable = dgiBridge.createDBInstance(db);
+        entities.put(Type.DATABASE, dbReferenceable);
 
-        Referenceable dbReferenceable = dgiBridge.registerDatabase(oldTable.getDbName());
-        Referenceable tableReferenceable =
-                dgiBridge.registerTable(dbReferenceable, oldTable.getDbName(), oldTable.getTableName());
-        LOG.info("Updating entity name {}.{} to {}", oldTable.getDbName(), oldTable.getTableName(),
-                newTable.getTableName());
-        dgiBridge.updateTable(tableReferenceable, newTable);
+        Referenceable tableReferenceable = null;
+        if (table != null) {
+            table = dgiBridge.hiveClient.getTable(table.getDbName(), table.getTableName());
+            tableReferenceable = dgiBridge.createTableInstance(dbReferenceable, table);
+            entities.put(Type.TABLE, tableReferenceable);
+        }
+
+        if (partition != null) {
+            Referenceable partitionReferenceable = dgiBridge.createPartitionReferenceable(tableReferenceable,
+                    (Referenceable) tableReferenceable.get("sd"), partition);
+            entities.put(Type.PARTITION, partitionReferenceable);
+        }
+        return entities;
     }
 
-    private void handleCreateTable(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception {
+    private void handleEventOutputs(HiveMetaStoreBridge dgiBridge, HiveEvent event, Type entityType) throws Exception {
+        List<Referenceable> entities = new ArrayList<>();
         for (WriteEntity entity : event.outputs) {
-            if (entity.getType() == Entity.Type.TABLE) {
-
-                Table table = entity.getTable();
-                Referenceable dbReferenceable = dgiBridge.registerDatabase(table.getDbName());
-                dgiBridge.registerTable(dbReferenceable, table.getDbName(), table.getTableName());
+            if (entity.getType() == entityType) {
+                entities.addAll(createEntities(dgiBridge, entity).values());
             }
         }
+        notifyEntity(entities);
     }
 
-    private void handleCreateDB(HiveMetaStoreBridge dgiBridge, HiveEvent event) throws Exception {
-        for (WriteEntity entity : event.outputs) {
-            if (entity.getType() == Entity.Type.DATABASE) {
-                dgiBridge.registerDatabase(entity.getDatabase().getName());
+    private void notifyEntity(Collection<Referenceable> entities) {
+        JSONArray entitiesArray = new JSONArray();
+        for (Referenceable entity : entities) {
+            String entityJson = InstanceSerialization.toJson(entity, true);
+            entitiesArray.put(entityJson);
+        }
+        notifyEntity(entitiesArray);
+    }
+
+    /**
+     * Notify atlas of the entity through message. The entity can be a complex entity with reference to other entities.
+     * De-duping of entities is done on server side depending on the unique attribute on the
+     * @param entities
+     */
+    private void notifyEntity(JSONArray entities) {
+        int maxRetries = atlasProperties.getInt(HOOK_NUM_RETRIES, 3);
+        String message = entities.toString();
+
+        int numRetries = 0;
+        while (true) {
+            try {
+                notifInterface.send(NotificationInterface.NotificationType.HOOK, message);
+                return;
+            } catch(Exception e) {
+                numRetries++;
+                if(numRetries < maxRetries) {
+                    LOG.debug("Failed to notify atlas for entity {}. Retrying", message, e);
+                } else {
+                    LOG.error("Failed to notify atlas for entity {} after {} retries. Quitting", message,
+                            maxRetries, e);
+                }
             }
         }
     }
@@ -284,50 +363,42 @@ public class HiveHook implements ExecuteWithHookContext {
         String queryStr = normalize(event.queryPlan.getQueryStr());
         long queryStartTime = event.queryPlan.getQueryStartTime();
 
-        LOG.debug("Registering CTAS query: {}", queryStr);
-        Referenceable processReferenceable = dgiBridge.getProcessReference(queryStr);
-        if (processReferenceable == null) {
-            processReferenceable = new Referenceable(HiveDataTypes.HIVE_PROCESS.getName());
-            processReferenceable.set("name", event.operation.getOperationName());
-            processReferenceable.set("startTime", queryStartTime);
-            processReferenceable.set("userName", event.user);
-
-            List<Referenceable> source = new ArrayList<>();
-            for (ReadEntity readEntity : inputs) {
-                if (readEntity.getType() == Entity.Type.TABLE) {
-                    Table table = readEntity.getTable();
-                    String dbName = table.getDbName();
-                    source.add(dgiBridge.registerTable(dbName, table.getTableName()));
-                }
-                if (readEntity.getType() == Entity.Type.PARTITION) {
-                    dgiBridge.registerPartition(readEntity.getPartition());
-                }
+        LOG.debug("Registering query: {}", queryStr);
+        List<Referenceable> entities = new ArrayList<>();
+        Referenceable processReferenceable = new Referenceable(HiveDataTypes.HIVE_PROCESS.getName());
+        processReferenceable.set("name", queryStr);
+        processReferenceable.set("operationType", event.operation.getOperationName());
+        processReferenceable.set("startTime", queryStartTime);
+        processReferenceable.set("userName", event.user);
+
+        List<Referenceable> source = new ArrayList<>();
+        for (ReadEntity readEntity : inputs) {
+            if (readEntity.getType() == Type.TABLE || readEntity.getType() == Type.PARTITION) {
+                Map<Type, Referenceable> localEntities = createEntities(dgiBridge, readEntity);
+                source.add(localEntities.get(Type.TABLE));
+                entities.addAll(localEntities.values());
             }
-            processReferenceable.set("inputs", source);
-
-            List<Referenceable> target = new ArrayList<>();
-            for (WriteEntity writeEntity : outputs) {
-                if (writeEntity.getType() == Entity.Type.TABLE || writeEntity.getType() == Entity.Type.PARTITION) {
-                    Table table = writeEntity.getTable();
-                    String dbName = table.getDbName();
-                    target.add(dgiBridge.registerTable(dbName, table.getTableName()));
-                }
-                if (writeEntity.getType() == Entity.Type.PARTITION) {
-                    dgiBridge.registerPartition(writeEntity.getPartition());
-                }
+        }
+        processReferenceable.set("inputs", source);
+
+        List<Referenceable> target = new ArrayList<>();
+        for (WriteEntity writeEntity : outputs) {
+            if (writeEntity.getType() == Type.TABLE || writeEntity.getType() == Type.PARTITION) {
+                Map<Type, Referenceable> localEntities = createEntities(dgiBridge, writeEntity);
+                target.add(localEntities.get(Type.TABLE));
+                entities.addAll(localEntities.values());
             }
-            processReferenceable.set("outputs", target);
-            processReferenceable.set("queryText", queryStr);
-            processReferenceable.set("queryId", queryId);
-            processReferenceable.set("queryPlan", event.jsonPlan.toString());
-            processReferenceable.set("endTime", System.currentTimeMillis());
-
-            //TODO set
-            processReferenceable.set("queryGraph", "queryGraph");
-            dgiBridge.createInstance(processReferenceable);
-        } else {
-            LOG.debug("Query {} is already registered", queryStr);
         }
+        processReferenceable.set("outputs", target);
+        processReferenceable.set("queryText", queryStr);
+        processReferenceable.set("queryId", queryId);
+        processReferenceable.set("queryPlan", event.jsonPlan.toString());
+        processReferenceable.set("endTime", System.currentTimeMillis());
+
+        //TODO set
+        processReferenceable.set("queryGraph", "queryGraph");
+        entities.add(processReferenceable);
+        notifyEntity(entities);
     }
 
 
@@ -338,7 +409,7 @@ public class HiveHook implements ExecuteWithHookContext {
             List<Task<?>> rootTasks = event.queryPlan.getRootTasks();
             return explain.getJSONPlan(null, null, rootTasks, event.queryPlan.getFetchTask(), true, false, false);
         } catch (Exception e) {
-            LOG.warn("Failed to get queryplan", e);
+            LOG.info("Failed to get queryplan", e);
             return new JSONObject();
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
index cb8359e..2571295 100755
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
@@ -35,6 +35,7 @@ import org.apache.atlas.typesystem.types.StructType;
 import org.apache.atlas.typesystem.types.StructTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.TypeUtils;
+import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -94,7 +95,6 @@ public class HiveDataModelGenerator {
         createColumnClass();
         createPartitionClass();
         createTableClass();
-        createIndexClass();
         createRoleClass();
 
         // DDL/DML Process
@@ -177,7 +177,7 @@ public class HiveDataModelGenerator {
     private void createStorageDescClass() throws AtlasException {
         AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
                 new AttributeDefinition("cols", String.format("array<%s>", HiveDataTypes.HIVE_COLUMN.getName()),
-                        Multiplicity.COLLECTION, false, null),
+                        Multiplicity.COLLECTION, true, null),
                 new AttributeDefinition("location", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false,
                         null),
                 new AttributeDefinition("inputFormat", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false,
@@ -188,7 +188,7 @@ public class HiveDataModelGenerator {
                         null),
                 new AttributeDefinition(STORAGE_NUM_BUCKETS, DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false,
                         null),
-                new AttributeDefinition("serdeInfo", HiveDataTypes.HIVE_SERDE.getName(), Multiplicity.OPTIONAL, false,
+                new AttributeDefinition("serdeInfo", HiveDataTypes.HIVE_SERDE.getName(), Multiplicity.OPTIONAL, true,
                         null),
                 new AttributeDefinition("bucketCols", String.format("array<%s>", DataTypes.STRING_TYPE.getName()),
                         Multiplicity.OPTIONAL, false, null),
@@ -201,8 +201,8 @@ public class HiveDataModelGenerator {
                         Multiplicity.OPTIONAL, false, null),};
 
         HierarchicalTypeDefinition<ClassType> definition =
-                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_STORAGEDESC.getName(), null,
-                        attributeDefinitions);
+                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_STORAGEDESC.getName(),
+                        ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
         classTypeDefinitions.put(HiveDataTypes.HIVE_STORAGEDESC.getName(), definition);
         LOG.debug("Created definition for " + HiveDataTypes.HIVE_STORAGEDESC.getName());
     }
@@ -236,8 +236,8 @@ public class HiveDataModelGenerator {
                         false, null),};
 
         HierarchicalTypeDefinition<ClassType> definition =
-                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_DB.getName(), null,
-                        attributeDefinitions);
+                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_DB.getName(),
+                        ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
         classTypeDefinitions.put(HiveDataTypes.HIVE_DB.getName(), definition);
         LOG.debug("Created definition for " + HiveDataTypes.HIVE_DB.getName());
     }
@@ -263,8 +263,8 @@ public class HiveDataModelGenerator {
                 new AttributeDefinition("type", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, null),
                 new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),};
         HierarchicalTypeDefinition<ClassType> definition =
-                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_COLUMN.getName(), null,
-                        attributeDefinitions);
+                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_COLUMN.getName(),
+                        ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
         classTypeDefinitions.put(HiveDataTypes.HIVE_COLUMN.getName(), definition);
         LOG.debug("Created definition for " + HiveDataTypes.HIVE_COLUMN.getName());
     }
@@ -278,14 +278,14 @@ public class HiveDataModelGenerator {
                         null),
                 new AttributeDefinition("lastAccessTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
                         null),
-                new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, false,
+                new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, true,
                         null),
                 new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
                         Multiplicity.OPTIONAL, true, null),
                 new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),};
         HierarchicalTypeDefinition<ClassType> definition =
-                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_PARTITION.getName(), null,
-                        attributeDefinitions);
+                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_PARTITION.getName(),
+                        ImmutableList.of(AtlasClient.REFERENCEABLE_SUPER_TYPE), attributeDefinitions);
         classTypeDefinitions.put(HiveDataTypes.HIVE_PARTITION.getName(), definition);
         LOG.debug("Created definition for " + HiveDataTypes.HIVE_PARTITION.getName());
     }
@@ -302,10 +302,10 @@ public class HiveDataModelGenerator {
                         null),
                 new AttributeDefinition(COMMENT, DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
                 new AttributeDefinition("retention", DataTypes.INT_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
-                new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.OPTIONAL, false,
+                new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.OPTIONAL, true,
                         null),
                 new AttributeDefinition("partitionKeys", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
-                        Multiplicity.OPTIONAL, false, null),
+                        Multiplicity.OPTIONAL, true, null),
                 new AttributeDefinition("columns", DataTypes.arrayTypeName(HiveDataTypes.HIVE_COLUMN.getName()),
                         Multiplicity.OPTIONAL, true, null),
                 new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
@@ -324,33 +324,6 @@ public class HiveDataModelGenerator {
         LOG.debug("Created definition for " + HiveDataTypes.HIVE_TABLE.getName());
     }
 
-    private void createIndexClass() throws AtlasException {
-        AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
-                new AttributeDefinition(NAME, DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false, null),
-                new AttributeDefinition("indexHandlerClass", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED,
-                        false, null),
-                new AttributeDefinition(DB, HiveDataTypes.HIVE_DB.getName(), Multiplicity.REQUIRED, false, null),
-                new AttributeDefinition("createTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
-                        null),
-                new AttributeDefinition("lastAccessTime", DataTypes.LONG_TYPE.getName(), Multiplicity.OPTIONAL, false,
-                        null),
-                new AttributeDefinition("origTable", HiveDataTypes.HIVE_TABLE.getName(), Multiplicity.REQUIRED, false,
-                        null),
-                new AttributeDefinition("indexTable", HiveDataTypes.HIVE_TABLE.getName(), Multiplicity.OPTIONAL, false,
-                        null),
-                new AttributeDefinition("sd", HiveDataTypes.HIVE_STORAGEDESC.getName(), Multiplicity.REQUIRED, false,
-                        null),
-                new AttributeDefinition("parameters", STRING_MAP_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
-                new AttributeDefinition("deferredRebuild", DataTypes.BOOLEAN_TYPE.getName(), Multiplicity.OPTIONAL,
-                        false, null),};
-
-        HierarchicalTypeDefinition<ClassType> definition =
-                new HierarchicalTypeDefinition<>(ClassType.class, HiveDataTypes.HIVE_INDEX.getName(),
-                        ImmutableList.of(AtlasClient.DATA_SET_SUPER_TYPE), attributeDefinitions);
-        classTypeDefinitions.put(HiveDataTypes.HIVE_INDEX.getName(), definition);
-        LOG.debug("Created definition for " + HiveDataTypes.HIVE_INDEX.getName());
-    }
-
     private void createRoleClass() throws AtlasException {
         AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
                 new AttributeDefinition("roleName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
@@ -373,6 +346,8 @@ public class HiveDataModelGenerator {
                 new AttributeDefinition("endTime", DataTypes.LONG_TYPE.getName(), Multiplicity.REQUIRED, false, null),
                 new AttributeDefinition("userName", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
                         null),
+                new AttributeDefinition("operationType", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
+                        null),
                 new AttributeDefinition("queryText", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
                         null),
                 new AttributeDefinition("queryPlan", DataTypes.STRING_TYPE.getName(), Multiplicity.REQUIRED, false,
@@ -399,16 +374,16 @@ public class HiveDataModelGenerator {
 
         TypesDef typesDef = hiveDataModelGenerator.getTypesDef();
         for (EnumTypeDefinition enumType : typesDef.enumTypesAsJavaList()) {
-            System.out.println(String.format("%s(%s) - %s", enumType.name, EnumType.class.getSimpleName(),
+            System.out.println(String.format("%s(%s) - values %s", enumType.name, EnumType.class.getSimpleName(),
                     Arrays.toString(enumType.enumValues)));
         }
         for (StructTypeDefinition structType : typesDef.structTypesAsJavaList()) {
-            System.out.println(String.format("%s(%s) - %s", structType.typeName, StructType.class.getSimpleName(),
+            System.out.println(String.format("%s(%s) - attributes %s", structType.typeName, StructType.class.getSimpleName(),
                     Arrays.toString(structType.attributeDefinitions)));
         }
         for (HierarchicalTypeDefinition<ClassType> classType : typesDef.classTypesAsJavaList()) {
-            System.out.println(String.format("%s(%s) - %s", classType.typeName, ClassType.class.getSimpleName(),
-                    Arrays.toString(classType.attributeDefinitions)));
+            System.out.println(String.format("%s(%s) - super types [%s] - attributes %s", classType.typeName, ClassType.class.getSimpleName(),
+                    StringUtils.join(classType.superTypes, ","), Arrays.toString(classType.attributeDefinitions)));
         }
         for (HierarchicalTypeDefinition<TraitType> traitType : typesDef.traitTypesAsJavaList()) {
             System.out.println(String.format("%s(%s) - %s", traitType.typeName, TraitType.class.getSimpleName(),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/addons/hive-bridge/src/site/twiki/Bridge-Hive.twiki
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/site/twiki/Bridge-Hive.twiki b/addons/hive-bridge/src/site/twiki/Bridge-Hive.twiki
deleted file mode 100644
index 647af27..0000000
--- a/addons/hive-bridge/src/site/twiki/Bridge-Hive.twiki
+++ /dev/null
@@ -1,72 +0,0 @@
----+ Hive Atlas Bridge
-Hive metadata can be modelled in Atlas using its Type System. The default modelling is available in org.apache.atlas.hive.model.HiveDataModelGenerator. It defines the following types:
-   * hive_resource_type(EnumType) - [JAR, FILE, ARCHIVE]
-   * hive_principal_type(EnumType) - [USER, ROLE, GROUP]
-   * hive_function_type(EnumType) - [JAVA]
-   * hive_order(StructType) - [col, order]
-   * hive_resourceuri(StructType) - [resourceType, uri]
-   * hive_serde(StructType) - [name, serializationLib, parameters]
-   * hive_process(ClassType) - [name, startTime, endTime, userName, sourceTableNames, targetTableNames, queryText, queryPlan, queryId, queryGraph]
-   * hive_function(ClassType) - [functionName, dbName, className, ownerName, ownerType, createTime, functionType, resourceUris]
-   * hive_type(ClassType) - [name, type1, type2, fields]
-   * hive_partition(ClassType) - [values, dbName, tableName, createTime, lastAccessTime, sd, parameters]
-   * hive_storagedesc(ClassType) - [cols, location, inputFormat, outputFormat, compressed, numBuckets, serdeInfo, bucketCols, sortCols, parameters, storedAsSubDirectories]
-   * hive_index(ClassType) - [indexName, indexHandlerClass, dbName, createTime, lastAccessTime, origTableName, indexTableName, sd, parameters, deferredRebuild]
-   * hive_role(ClassType) - [roleName, createTime, ownerName]
-   * hive_column(ClassType) - [name, type, comment]
-   * hive_db(ClassType) - [name, description, locationUri, parameters, ownerName, ownerType]
-   * hive_table(ClassType) - [name, dbName, owner, createTime, lastAccessTime, retention, sd, partitionKeys, columns, parameters, viewOriginalText, viewExpandedText, tableType, temporary]
-
-
----++ Importing Hive Metadata
-org.apache.atlas.hive.bridge.HiveMetaStoreBridge imports the hive metadata into Atlas using the typesystem defined in org.apache.atlas.hive.model.HiveDataModelGenerator. import-hive.sh command can be used to facilitate this.
-Set-up the following configs in hive-site.xml of your hive set-up and set environment variable HIVE_CONFIG to the
-hive conf directory:
-   * Atlas endpoint - Add the following property with the Atlas endpoint for your set-up
-<verbatim>
-<property>
-  <name>atlas.rest.address</name>
-  <value>http://localhost:21000/</value>
-</property>
-<property>
-  <name>atlas.cluster.name</name>
-  <value>primary</value>
-</property>
-</verbatim>
-
-Usage: <dgi package>/bin/import-hive.sh. The logs are in <dgi package>/logs/import-hive.log
-
-
----++ Hive Hook
-Hive supports listeners on hive command execution using hive hooks. This is used to add/update/remove entities in Atlas using the model defined in org.apache.atlas.hive.model.HiveDataModelGenerator.
-The hook submits the request to a thread pool executor to avoid blocking the command execution. Follow the these instructions in your hive set-up to add hive hook for Atlas:
-   * Add org.apache.atlas.hive.hook.HiveHook as post execution hook in hive-site.xml
-<verbatim>
-<property>
-  <name>hive.exec.post.hooks</name>
-  <value>org.apache.atlas.hive.hook.HiveHook</value>
-</property>
-</verbatim>
-   * Add the following properties in hive-ste.xml with the Atlas endpoint for your set-up
-<verbatim>
-<property>
-  <name>atlas.rest.address</name>
-  <value>http://localhost:21000/</value>
-</property>
-<property>
-  <name>atlas.cluster.name</name>
-  <value>primary</value>
-</property>
-</verbatim>
-   * Add 'export HIVE_AUX_JARS_PATH=<dgi package>/hook/hive' in hive-env.sh
-
-The following properties in hive-site.xml control the thread pool details:
-   * atlas.hook.hive.minThreads - core number of threads. default 5
-   * atlas.hook.hive.maxThreads - maximum number of threads. default 5
-   * atlas.hook.hive.keepAliveTime - keep alive time in msecs. default 10
-   * atlas.hook.hive.synchronous - boolean, true to run the hook synchronously. default false
-
----++ Limitations
-   * Since database name, table name and column names are case insensitive in hive, the corresponding names in entities are lowercase. So, any search APIs should use lowercase while querying on the entity names
-   * Only the following hive operations are captured by hive hook currently - create database, create table, create view, CTAS, load, import, export, query, alter table rename and alter view rename
-