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:00 UTC

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

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/ApplicationProperties.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/ApplicationProperties.java b/typesystem/src/main/java/org/apache/atlas/ApplicationProperties.java
new file mode 100644
index 0000000..738ec53
--- /dev/null
+++ b/typesystem/src/main/java/org/apache/atlas/ApplicationProperties.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas;
+
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Iterator;
+
+public class ApplicationProperties extends PropertiesConfiguration {
+    private static final Logger LOG = LoggerFactory.getLogger(ApplicationProperties.class);
+
+    public static final String APPLICATION_PROPERTIES = "application.properties";
+    public static final String CLIENT_PROPERTIES = "client.properties";
+
+    private static Configuration INSTANCE = null;
+
+    private ApplicationProperties(URL url) throws ConfigurationException {
+        super(url);
+    }
+
+    public static Configuration get() throws AtlasException {
+        if (INSTANCE == null) {
+            synchronized (ApplicationProperties.class) {
+                if (INSTANCE == null) {
+                    Configuration applicationProperties = get(APPLICATION_PROPERTIES);
+                    Configuration clientProperties = get(CLIENT_PROPERTIES);
+                    INSTANCE = new CompositeConfiguration(Arrays.asList(applicationProperties, clientProperties));
+                }
+            }
+        }
+        return INSTANCE;
+    }
+
+    public static Configuration get(String fileName) throws AtlasException {
+        String confLocation = System.getProperty("atlas.conf");
+        try {
+            URL url = confLocation == null ? ApplicationProperties.class.getResource("/" + fileName)
+                    : new File(confLocation, fileName).toURI().toURL();
+            LOG.info("Loading {} from {}", fileName, url);
+
+            Configuration configuration = new ApplicationProperties(url).interpolatedConfiguration();
+            logConfiguration(configuration);
+            return configuration;
+        } catch (Exception e) {
+            throw new AtlasException("Failed to load application properties", e);
+        }
+    }
+
+    private static void logConfiguration(Configuration configuration) {
+        if (LOG.isDebugEnabled()) {
+            Iterator<String> keys = configuration.getKeys();
+            LOG.debug("Configuration loaded:");
+            while (keys.hasNext()) {
+                String key = keys.next();
+                LOG.debug("{} = {}", key, configuration.getProperty(key));
+            }
+        }
+    }
+
+    public static final Configuration getSubsetConfiguration(Configuration inConf, String prefix) {
+        return inConf.subset(prefix);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/TypeExistsException.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/TypeExistsException.java b/typesystem/src/main/java/org/apache/atlas/TypeExistsException.java
new file mode 100644
index 0000000..1a2cb7c
--- /dev/null
+++ b/typesystem/src/main/java/org/apache/atlas/TypeExistsException.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas;
+
+public class TypeExistsException extends AtlasException {
+    public TypeExistsException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java b/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
index 309ab11..ea4a3cb 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
@@ -115,7 +115,11 @@ public class StructInstance implements ITypedStruct {
             clsType.validateId((Id) val);
             cVal = val;
         } else {
-            cVal = i.dataType().convert(val, i.multiplicity);
+            try {
+                cVal = i.dataType().convert(val, i.multiplicity);
+            } catch(ValueConversionException.NullConversionException e) {
+                throw new ValueConversionException.NullConversionException("For field '" + attrName + "'", e);
+            }
         }
         if (cVal == null) {
             nullFlags[nullPos] = true;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeDefinition.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeDefinition.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeDefinition.java
index 31b5f47..2fea1be 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeDefinition.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeDefinition.java
@@ -25,6 +25,7 @@ public final class AttributeDefinition {
     public final String name;
     public final String dataTypeName;
     public final Multiplicity multiplicity;
+    //A composite is the one whose lifecycle is dependent on the enclosing type and is not just a reference
     public final boolean isComposite;
     public final boolean isUnique;
     public final boolean isIndexable;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
index e74f8d5..3e1ef18 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
@@ -27,6 +27,7 @@ import java.util.Map;
 public class AttributeInfo {
     public final String name;
     public final Multiplicity multiplicity;
+    //A composite is the one whose lifecycle is dependent on the enclosing type and is not just a reference
     public final boolean isComposite;
     public final boolean isUnique;
     public final boolean isIndexable;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ObjectGraphWalker.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ObjectGraphWalker.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/ObjectGraphWalker.java
index 399902c..5178a6b 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ObjectGraphWalker.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/ObjectGraphWalker.java
@@ -55,8 +55,8 @@ public class ObjectGraphWalker {
     throws AtlasException {
         this.typeSystem = typeSystem;
         this.nodeProcessor = nodeProcessor;
-        queue = new LinkedList<IReferenceableInstance>();
-        processedIds = new HashSet<Id>();
+        queue = new LinkedList<>();
+        processedIds = new HashSet<>();
         if (start != null) {
             visitReferenceableInstance(start);
         }
@@ -194,7 +194,7 @@ public class ObjectGraphWalker {
         }
     }
 
-    public static interface NodeProcessor {
+    public interface NodeProcessor {
 
         void processNode(Node nd) throws AtlasException;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
index d1e8dbd..b03b780 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
@@ -22,6 +22,7 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Multimap;
 import org.apache.atlas.AtlasException;
+import org.apache.atlas.TypeExistsException;
 import org.apache.atlas.TypeNotFoundException;
 import org.apache.atlas.classification.InterfaceAudience;
 import org.apache.atlas.typesystem.TypesDef;
@@ -169,9 +170,9 @@ public class TypeSystem {
     public StructType defineStructType(String name, boolean errorIfExists, AttributeDefinition... attrDefs)
     throws AtlasException {
         StructTypeDefinition structDef = new StructTypeDefinition(name, attrDefs);
-        defineTypes(ImmutableList.of(structDef), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+        defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.of(structDef),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
                 ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
-
         return getDataType(StructType.class, structDef.typeName);
     }
 
@@ -196,56 +197,50 @@ public class TypeSystem {
     }
 
     public TraitType defineTraitType(HierarchicalTypeDefinition<TraitType> traitDef) throws AtlasException {
-
-        defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.of(traitDef),
-                ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
-
+        defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(traitDef), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
         return getDataType(TraitType.class, traitDef.typeName);
     }
 
     public ClassType defineClassType(HierarchicalTypeDefinition<ClassType> classDef) throws AtlasException {
-
-        defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
-                ImmutableList.of(classDef));
-
+        defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.of(classDef));
         return getDataType(ClassType.class, classDef.typeName);
     }
 
     public Map<String, IDataType> defineTraitTypes(HierarchicalTypeDefinition<TraitType>... traitDefs)
     throws AtlasException {
         TransientTypeSystem transientTypes =
-                new TransientTypeSystem(ImmutableList.<StructTypeDefinition>of(), ImmutableList.copyOf(traitDefs),
+                new TransientTypeSystem(ImmutableList.<EnumTypeDefinition>of(),
+                        ImmutableList.<StructTypeDefinition>of(), ImmutableList.copyOf(traitDefs),
                         ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
         return transientTypes.defineTypes();
     }
 
     public Map<String, IDataType> defineClassTypes(HierarchicalTypeDefinition<ClassType>... classDefs)
     throws AtlasException {
-        TransientTypeSystem transientTypes = new TransientTypeSystem(ImmutableList.<StructTypeDefinition>of(),
-                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.copyOf(classDefs));
+        TransientTypeSystem transientTypes = new TransientTypeSystem(ImmutableList.<EnumTypeDefinition>of(),
+                ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+                ImmutableList.copyOf(classDefs));
         return transientTypes.defineTypes();
     }
 
     public Map<String, IDataType> defineTypes(TypesDef typesDef) throws AtlasException {
-        Map<String, IDataType> typesAdded = new HashMap<>();
-        for (EnumTypeDefinition enumDef : typesDef.enumTypesAsJavaList()) {
-            typesAdded.put(enumDef.name, defineEnumType(enumDef));
-        }
-
+        ImmutableList<EnumTypeDefinition> enumDefs = ImmutableList.copyOf(typesDef.enumTypesAsJavaList());
         ImmutableList<StructTypeDefinition> structDefs = ImmutableList.copyOf(typesDef.structTypesAsJavaList());
         ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs =
                 ImmutableList.copyOf(typesDef.traitTypesAsJavaList());
         ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs =
                 ImmutableList.copyOf(typesDef.classTypesAsJavaList());
 
-        typesAdded.putAll(defineTypes(structDefs, traitDefs, classDefs));
-        return typesAdded;
+        return defineTypes(enumDefs, structDefs, traitDefs, classDefs);
     }
 
-    public Map<String, IDataType> defineTypes(ImmutableList<StructTypeDefinition> structDefs,
+    public Map<String, IDataType> defineTypes(ImmutableList<EnumTypeDefinition> enumDefs,
+            ImmutableList<StructTypeDefinition> structDefs,
             ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
             ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException {
-        TransientTypeSystem transientTypes = new TransientTypeSystem(structDefs, traitDefs, classDefs);
+        TransientTypeSystem transientTypes = new TransientTypeSystem(enumDefs, structDefs, traitDefs, classDefs);
         Map<String, IDataType> definedTypes = transientTypes.defineTypes();
         //        LOG.debug("Defined new types " + Arrays.toString(definedTypes.keySet().toArray(new
         // String[definedTypes.size()])));
@@ -307,6 +302,7 @@ public class TypeSystem {
         final ImmutableList<StructTypeDefinition> structDefs;
         final ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs;
         final ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs;
+        private final ImmutableList<EnumTypeDefinition> enumDefs;
         Map<String, StructTypeDefinition> structNameToDefMap = new HashMap<>();
         Map<String, HierarchicalTypeDefinition<TraitType>> traitNameToDefMap = new HashMap<>();
         Map<String, HierarchicalTypeDefinition<ClassType>> classNameToDefMap = new HashMap<>();
@@ -318,10 +314,10 @@ public class TypeSystem {
         List<DataTypes.MapType> recursiveMapTypes;
 
 
-        TransientTypeSystem(ImmutableList<StructTypeDefinition> structDefs,
-                ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
-                ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) {
-
+        TransientTypeSystem(ImmutableList<EnumTypeDefinition> enumDefs, ImmutableList<StructTypeDefinition> structDefs,
+                            ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
+                            ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) {
+            this.enumDefs = enumDefs;
             this.structDefs = structDefs;
             this.traitDefs = traitDefs;
             this.classDefs = classDefs;
@@ -345,10 +341,22 @@ public class TypeSystem {
          * - setup shallow Type instances to facilitate recursive type graphs
          */
         private void step1() throws AtlasException {
+            for (EnumTypeDefinition eDef : enumDefs) {
+                assert eDef.name != null;
+                if (types.containsKey(eDef.name)) {
+                    throw new AtlasException(String.format("Redefinition of type %s not supported", eDef.name));
+                }
+
+                EnumType eT = new EnumType(this, eDef.name, eDef.enumValues);
+                TypeSystem.this.types.put(eDef.name, eT);
+                typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.ENUM, eDef.name);
+                transientTypes.add(eDef.name);
+            }
+
             for (StructTypeDefinition sDef : structDefs) {
                 assert sDef.typeName != null;
                 if (dataType(sDef.typeName) != null) {
-                    throw new AtlasException(String.format("Cannot redefine type %s", sDef.typeName));
+                    throw new TypeExistsException(String.format("Cannot redefine type %s", sDef.typeName));
                 }
                 TypeSystem.this.types
                         .put(sDef.typeName, new StructType(this, sDef.typeName, sDef.attributeDefinitions.length));
@@ -359,7 +367,7 @@ public class TypeSystem {
             for (HierarchicalTypeDefinition<TraitType> traitDef : traitDefs) {
                 assert traitDef.typeName != null;
                 if (types.containsKey(traitDef.typeName)) {
-                    throw new AtlasException(String.format("Cannot redefine type %s", traitDef.typeName));
+                    throw new TypeExistsException(String.format("Cannot redefine type %s", traitDef.typeName));
                 }
 
                 TypeSystem.this.types.put(traitDef.typeName, new TraitType(this, traitDef.typeName, traitDef.superTypes,
@@ -371,7 +379,7 @@ public class TypeSystem {
             for (HierarchicalTypeDefinition<ClassType> classDef : classDefs) {
                 assert classDef.typeName != null;
                 if (types.containsKey(classDef.typeName)) {
-                    throw new AtlasException(String.format("Cannot redefine type %s", classDef.typeName));
+                    throw new TypeExistsException(String.format("Cannot redefine type %s", classDef.typeName));
                 }
 
                 TypeSystem.this.types.put(classDef.typeName, new ClassType(this, classDef.typeName, classDef.superTypes,
@@ -588,7 +596,8 @@ public class TypeSystem {
         }
 
         @Override
-        public Map<String, IDataType> defineTypes(ImmutableList<StructTypeDefinition> structDefs,
+        public Map<String, IDataType> defineTypes(ImmutableList<EnumTypeDefinition> enumDefs,
+                ImmutableList<StructTypeDefinition> structDefs,
                 ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
                 ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException {
             throw new AtlasException("Internal Error: define type called on TrasientTypeSystem");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ValueConversionException.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ValueConversionException.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/ValueConversionException.java
index 5cdee8d..7fe667a 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/ValueConversionException.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/ValueConversionException.java
@@ -43,10 +43,17 @@ public class ValueConversionException extends AtlasException {
         super(msg);
     }
 
+    protected ValueConversionException(String msg, Exception e) {
+        super(msg, e);
+    }
+
     public static class NullConversionException extends ValueConversionException {
         public NullConversionException(Multiplicity m) {
             super(String.format("Null value not allowed for multiplicty %s", m));
         }
 
+        public NullConversionException(String msg, Exception e) {
+            super(msg, e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/resources/application.properties
----------------------------------------------------------------------
diff --git a/typesystem/src/main/resources/application.properties b/typesystem/src/main/resources/application.properties
index f7e2774..341acec 100644
--- a/typesystem/src/main/resources/application.properties
+++ b/typesystem/src/main/resources/application.properties
@@ -41,6 +41,7 @@ atlas.graph.index.search.elasticsearch.create.sleep=2000
 atlas.graph.index.search.solr.mode=cloud
 atlas.graph.index.search.solr.zookeeper-url=${solr.zk.address}
 
+
 #########  Hive Lineage Configs  #########
 # This models reflects the base super types for Data and Process
 #atlas.lineage.hive.table.type.name=DataSet
@@ -53,8 +54,13 @@ atlas.lineage.hive.table.schema.query.hive_table=hive_table where name='%s'\, co
 
 #########  Notification Configs  #########
 atlas.notification.embedded=true
-atlas.notification.implementation=org.apache.atlas.kafka.KafkaNotification
-atlas.notification.kafka.data=target/data/kafka
+
+atlas.kafka.zookeeper.connect=localhost:9026
+atlas.kafka.bootstrap.servers=localhost:9027
+atlas.kafka.data=target/data/kafka
+atlas.kafka.zookeeper.session.timeout.ms=400
+atlas.kafka.zookeeper.sync.time.ms=20
+atlas.kafka.auto.commit.interval.ms=100
 
 #########  Security Properties  #########
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/resources/atlas-log4j.xml
----------------------------------------------------------------------
diff --git a/typesystem/src/main/resources/atlas-log4j.xml b/typesystem/src/main/resources/atlas-log4j.xml
new file mode 100755
index 0000000..d546e7f
--- /dev/null
+++ b/typesystem/src/main/resources/atlas-log4j.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+    <appender name="console" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="${atlas.log.dir}/audit.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
+
+    <logger name="org.apache.atlas" additivity="false">
+        <level value="debug"/>
+        <appender-ref ref="console"/>
+    </logger>
+
+    <logger name="AUDIT">
+        <level value="info"/>
+        <appender-ref ref="console"/>
+    </logger>
+
+    <root>
+        <priority value="warn"/>
+        <appender-ref ref="console"/>
+    </root>
+
+</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/main/scala/org/apache/atlas/typesystem/json/TypesSerialization.scala
----------------------------------------------------------------------
diff --git a/typesystem/src/main/scala/org/apache/atlas/typesystem/json/TypesSerialization.scala b/typesystem/src/main/scala/org/apache/atlas/typesystem/json/TypesSerialization.scala
index 7a681bf..6bdbc79 100755
--- a/typesystem/src/main/scala/org/apache/atlas/typesystem/json/TypesSerialization.scala
+++ b/typesystem/src/main/scala/org/apache/atlas/typesystem/json/TypesSerialization.scala
@@ -236,7 +236,7 @@ trait TypeHelpers {
 
     @throws(classOf[AtlasException])
     def defineClassType(ts: TypeSystem, classDef: HierarchicalTypeDefinition[ClassType]): ClassType = {
-        ts.defineTypes(ImmutableList.of[StructTypeDefinition],
+        ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
             ImmutableList.of[HierarchicalTypeDefinition[TraitType]],
             ImmutableList.of[HierarchicalTypeDefinition[ClassType]](classDef))
         return ts.getDataType(classOf[ClassType], classDef.typeName)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/java/org/apache/atlas/ApplicationPropertiesTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/ApplicationPropertiesTest.java b/typesystem/src/test/java/org/apache/atlas/ApplicationPropertiesTest.java
new file mode 100644
index 0000000..d8098ac
--- /dev/null
+++ b/typesystem/src/test/java/org/apache/atlas/ApplicationPropertiesTest.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas;
+
+import org.apache.commons.configuration.Configuration;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class ApplicationPropertiesTest {
+
+    @Test
+    public void testVariables() throws Exception {
+        Configuration properties = ApplicationProperties.get(ApplicationProperties.APPLICATION_PROPERTIES);
+
+        //plain property without variables
+        Assert.assertEquals(properties.getString("atlas.service"), "atlas");
+
+        //property containing system property
+        String data = "/var/data/" + System.getProperty("user.name") + "/atlas";
+        Assert.assertEquals(properties.getString("atlas.data"), data);
+
+        //property referencing other property
+        Assert.assertEquals(properties.getString("atlas.graph.data"), data + "/graph");
+
+        //invalid system property - not substituted
+        Assert.assertEquals(properties.getString("atlas.db"), "${atlasdb}");
+    }
+
+    @Test
+    //variable substitutions should work with subset configuration as well
+    public void testSubset() throws Exception {
+        Configuration configuration = ApplicationProperties.get(ApplicationProperties.APPLICATION_PROPERTIES);
+        Configuration subConfiguration = configuration.subset("atlas");
+
+        Assert.assertEquals(subConfiguration.getString("service"), "atlas");
+        String data = "/var/data/" + System.getProperty("user.name") + "/atlas";
+        Assert.assertEquals(subConfiguration.getString("data"), data);
+        Assert.assertEquals(subConfiguration.getString("graph.data"), data + "/graph");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/java/org/apache/atlas/typesystem/json/SerializationJavaTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/json/SerializationJavaTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/json/SerializationJavaTest.java
index a8c58bd..eb652f6 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/json/SerializationJavaTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/json/SerializationJavaTest.java
@@ -28,6 +28,7 @@ import org.apache.atlas.typesystem.types.AttributeDefinition;
 import org.apache.atlas.typesystem.types.BaseTest;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.EnumTypeDefinition;
 import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
 import org.apache.atlas.typesystem.types.Multiplicity;
 import org.apache.atlas.typesystem.types.StructTypeDefinition;
@@ -79,9 +80,9 @@ public class SerializationJavaTest extends BaseTest {
                 createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
                         createRequiredAttrDef("level", DataTypes.INT_TYPE));
 
-        ts.defineTypes(ImmutableList.<StructTypeDefinition>of(),
-                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(securityClearanceTypeDef),
-                ImmutableList.<HierarchicalTypeDefinition<ClassType>>of(deptTypeDef, personTypeDef, managerTypeDef));
+        ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(securityClearanceTypeDef),
+                ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef));
 
         Referenceable hrDept = new Referenceable("Department");
         Referenceable john = new Referenceable("Person");
@@ -147,8 +148,8 @@ public class SerializationJavaTest extends BaseTest {
                 createTraitTypeDef("SecurityClearance2", ImmutableList.<String>of(),
                         createRequiredAttrDef("level", DataTypes.INT_TYPE));
 
-        ts.defineTypes(ImmutableList.<StructTypeDefinition>of(),
-                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(securityClearanceTypeDef),
+        ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(securityClearanceTypeDef),
                 ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
 
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
index 1d80681..fced1bc 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
@@ -136,7 +136,8 @@ public abstract class BaseTest {
                 .createTraitTypeDef("SecurityClearance", ImmutableList.<String>of(),
                         TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE));
 
-        ts.defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList.of(securityClearanceTypeDef),
+        ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(securityClearanceTypeDef),
                 ImmutableList.of(deptTypeDef, personTypeDef, managerTypeDef));
 
         ImmutableList.of(ts.getDataType(HierarchicalType.class, "SecurityClearance"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/java/org/apache/atlas/typesystem/types/TypeSystemTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/TypeSystemTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/TypeSystemTest.java
index e8bbb0c..cedfa0b 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/TypeSystemTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/TypeSystemTest.java
@@ -80,9 +80,10 @@ public class TypeSystemTest extends BaseTest {
         HierarchicalTypeDefinition<TraitType> financeTrait =
                 TypesUtil.createTraitTypeDef("Finance", ImmutableList.<String>of());
 
-        getTypeSystem().defineTypes(ImmutableList.<StructTypeDefinition>of(), ImmutableList
-                        .of(classificationTraitDefinition, piiTrait, phiTrait, pciTrait, soxTrait, secTrait,
-                                financeTrait), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
+        getTypeSystem().defineTypes(ImmutableList.<EnumTypeDefinition>of(),
+                ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(classificationTraitDefinition, piiTrait, phiTrait, pciTrait, soxTrait, secTrait,
+                        financeTrait), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
 
         final ImmutableList<String> traitsNames = getTypeSystem().getTypeNamesByCategory(DataTypes.TypeCategory.TRAIT);
         Assert.assertEquals(traitsNames.size(), 7);
@@ -102,7 +103,6 @@ public class TypeSystemTest extends BaseTest {
         String enumType = random();
         EnumTypeDefinition orgLevelEnum =
                 new EnumTypeDefinition(enumType, new EnumValue(random(), 1), new EnumValue(random(), 2));
-        ts.defineEnumType(orgLevelEnum);
 
         String structName = random();
         String attrType = random();
@@ -117,7 +117,8 @@ public class TypeSystemTest extends BaseTest {
         HierarchicalTypeDefinition<TraitType> traitType = createTraitTypeDef(traitName, ImmutableList.<String>of(),
                 createRequiredAttrDef(attrType, DataTypes.INT_TYPE));
 
-        ts.defineTypes(ImmutableList.of(structType), ImmutableList.of(traitType), ImmutableList.of(classType));
+        ts.defineTypes(ImmutableList.of(orgLevelEnum), ImmutableList.of(structType),
+                ImmutableList.of(traitType), ImmutableList.of(classType));
     }
 
     @Test
@@ -127,7 +128,7 @@ public class TypeSystemTest extends BaseTest {
         HierarchicalTypeDefinition<ClassType> c = TypesUtil.createClassTypeDef("C", ImmutableList.of("B"));
 
         TypeSystem ts = getTypeSystem();
-        ts.defineTypes(ImmutableList.<StructTypeDefinition>of(),
+        ts.defineTypes(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
                 ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
                 ImmutableList.of(a, b, c));
         ClassType ac = ts.getDataType(ClassType.class, "a");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/resources/application.properties
----------------------------------------------------------------------
diff --git a/typesystem/src/test/resources/application.properties b/typesystem/src/test/resources/application.properties
new file mode 100644
index 0000000..dbd6002
--- /dev/null
+++ b/typesystem/src/test/resources/application.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+#system property
+atlas.data=/var/data/${sys:user.name}/atlas
+
+#re-use existing property
+atlas.graph.data=${atlas.data}/graph
+
+#plain property
+atlas.service=atlas
+
+#invalid system property
+atlas.db=${atlasdb}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/SerializationTest.scala
----------------------------------------------------------------------
diff --git a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/SerializationTest.scala b/typesystem/src/test/scala/org/apache/atlas/typesystem/json/SerializationTest.scala
index d14ac72..d315032 100755
--- a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/SerializationTest.scala
+++ b/typesystem/src/test/scala/org/apache/atlas/typesystem/json/SerializationTest.scala
@@ -146,7 +146,7 @@ class SerializationTest extends BaseTest {
       TypesUtil.createTraitTypeDef("SecurityClearance", ImmutableList.of[String],
         TypesUtil.createRequiredAttrDef("level", DataTypes.INT_TYPE))
 
-    ts.defineTypes(ImmutableList.of[StructTypeDefinition],
+    ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
       ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef),
       ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef)
     )

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/TypesSerializationTest.scala
----------------------------------------------------------------------
diff --git a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/TypesSerializationTest.scala b/typesystem/src/test/scala/org/apache/atlas/typesystem/json/TypesSerializationTest.scala
index ba4782c..3737bf8 100755
--- a/typesystem/src/test/scala/org/apache/atlas/typesystem/json/TypesSerializationTest.scala
+++ b/typesystem/src/test/scala/org/apache/atlas/typesystem/json/TypesSerializationTest.scala
@@ -45,7 +45,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
             optionalAttr("o", DataTypes.mapTypeName(DataTypes.STRING_TYPE, DataTypes.DOUBLE_TYPE)))
 
 
-        ts.defineTypes(ImmutableList.of[StructTypeDefinition](sDef),
+        ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition](sDef),
             ImmutableList.of[HierarchicalTypeDefinition[TraitType]],
             ImmutableList.of[HierarchicalTypeDefinition[ClassType]]
         )
@@ -123,7 +123,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
         val securityClearanceTypeDef: HierarchicalTypeDefinition[TraitType] = createTraitTypeDef("SecurityClearance", List(),
             requiredAttr("level", DataTypes.INT_TYPE)
         )
-        ts.defineTypes(ImmutableList.of[StructTypeDefinition],
+        ts.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.of[StructTypeDefinition],
             ImmutableList.of[HierarchicalTypeDefinition[TraitType]](securityClearanceTypeDef),
             ImmutableList.of[HierarchicalTypeDefinition[ClassType]](deptTypeDef, personTypeDef, managerTypeDef))
 
@@ -136,7 +136,7 @@ class TypesSerializationTest extends BaseTest with TypeHelpers {
 
         typesDef1.enumTypes.foreach(ts1.defineEnumType(_))
 
-        ts1.defineTypes(ImmutableList.copyOf(typesDef1.structTypes.toArray),
+        ts1.defineTypes(ImmutableList.of[EnumTypeDefinition], ImmutableList.copyOf(typesDef1.structTypes.toArray),
             ImmutableList.copyOf(typesDef1.traitTypes.toArray),
             ImmutableList.copyOf(typesDef1.classTypes.toArray)
         )

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/pom.xml
----------------------------------------------------------------------
diff --git a/webapp/pom.xml b/webapp/pom.xml
index 6085d68..b375c8f 100755
--- a/webapp/pom.xml
+++ b/webapp/pom.xml
@@ -188,6 +188,11 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-common</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.atlas</groupId>
             <artifactId>atlas-typesystem</artifactId>
         </dependency>
 
@@ -202,6 +207,11 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-notification</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
         </dependency>
@@ -413,6 +423,45 @@
                 </configuration>
             </plugin>
 
+            <!-- Running unit tests in pre-integration-test phase after war is built -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemProperties>
+                        <user.dir>${project.basedir}</user.dir>
+                        <projectBaseDir>${project.basedir}/..</projectBaseDir>
+                    </systemProperties>
+                    <!--<skipTests>true</skipTests>-->
+                    <forkMode>always</forkMode>
+                    <redirectTestOutputToFile>true</redirectTestOutputToFile>
+                    <argLine>-Djava.awt.headless=true -Dproject.version=${project.version}
+                        -Dhadoop.tmp.dir=${project.build.directory}/tmp-hadoop-${user.name}
+                        -Xmx1024m -XX:MaxPermSize=512m
+                    </argLine>
+                    <excludes>
+                        <exclude>**/*Base*</exclude>
+                    </excludes>
+                </configuration>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.maven.surefire</groupId>
+                        <artifactId>surefire-testng</artifactId>
+                        <version>2.18.1</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <id>default-test</id>
+                        <phase>pre-integration-test</phase>
+                        <goals>
+                            <goal>test</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
+
             <plugin>
                 <groupId>org.eclipse.jetty</groupId>
                 <artifactId>jetty-maven-plugin</artifactId>
@@ -435,6 +484,10 @@
                     <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>
@@ -453,7 +506,7 @@
                         </systemProperty>
                     </systemProperties>
                     <stopKey>atlas-stop</stopKey>
-                    <stopPort>41001</stopPort>
+                    <stopPort>21001</stopPort>
                     <daemon>${debug.jetty.daemon}</daemon>
                     <testClassesDirectory>${project.build.directory}/../../webapp/target/test-classes/</testClassesDirectory>
                     <useTestClasspath>true</useTestClasspath>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/Main.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/Main.java b/webapp/src/main/java/org/apache/atlas/Main.java
index 7b13f3d..d91680d 100755
--- a/webapp/src/main/java/org/apache/atlas/Main.java
+++ b/webapp/src/main/java/org/apache/atlas/Main.java
@@ -101,6 +101,7 @@ public final class Main {
         configuration.setProperty("atlas.enableTLS", String.valueOf(enableTLS));
 
         showStartupInfo(buildConfiguration, enableTLS, appPort);
+
         server = EmbeddedServer.newServer(appPort, appPath, enableTLS);
         server.start();
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java b/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
index b32b403..72b81cf 100755
--- a/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
+++ b/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
@@ -38,7 +38,6 @@ import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.TypeUtils;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.codehaus.jettison.json.JSONArray;
-import org.codehaus.jettison.json.JSONObject;
 
 import java.util.List;
 
@@ -241,12 +240,11 @@ public class QuickStart {
 
         String entityJSON = InstanceSerialization.toJson(referenceable, true);
         System.out.println("Submitting new entity= " + entityJSON);
-        JSONObject jsonObject = metadataServiceClient.createEntity(entityJSON);
-        String guid = jsonObject.getString(AtlasClient.GUID);
-        System.out.println("created instance for type " + typeName + ", guid: " + guid);
+        JSONArray guids = metadataServiceClient.createEntity(entityJSON);
+        System.out.println("created instance for type " + typeName + ", guid: " + guids);
 
         // return the Id for created instance with guid
-        return new Id(guid, referenceable.getId().getVersion(), referenceable.getTypeName());
+        return new Id(guids.getString(0), referenceable.getId().getVersion(), referenceable.getTypeName());
     }
 
     Id database(String name, String description, String owner, String locationUri, String... traitNames)
@@ -387,11 +385,9 @@ public class QuickStart {
 
     private void search() throws Exception {
         for (String dslQuery : getDSLQueries()) {
-            JSONObject response = metadataServiceClient.searchEntity(dslQuery);
-            JSONObject results = response.getJSONObject(AtlasClient.RESULTS);
-            if (!results.isNull("rows")) {
-                JSONArray rows = results.getJSONArray("rows");
-                System.out.println("query [" + dslQuery + "] returned [" + rows.length() + "] rows");
+            JSONArray results = metadataServiceClient.search(dslQuery);
+            if (results != null) {
+                System.out.println("query [" + dslQuery + "] returned [" + results.length() + "] rows");
             } else {
                 System.out.println("query [" + dslQuery + "] failed, results:" + results.toString());
             }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java b/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
index bcf115f..fd05d28 100755
--- a/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
+++ b/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
@@ -33,7 +33,9 @@ import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.notification.NotificationModule;
 import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.service.Services;
 import org.apache.atlas.web.filters.AtlasAuthenticationFilter;
 import org.apache.atlas.web.filters.AuditFilter;
 import org.apache.commons.configuration.Configuration;
@@ -64,7 +66,8 @@ public class GuiceServletConfig extends GuiceServletContextListener {
 		 * .html
 		 */
         if (injector == null) {
-            injector = Guice.createInjector(new RepositoryMetadataModule(), new JerseyServletModule() {
+            injector = Guice.createInjector(new RepositoryMetadataModule(), new NotificationModule(),
+                    new JerseyServletModule() {
                         @Override
                         protected void configureServlets() {
                             filter("/*").through(AuditFilter.class);
@@ -110,6 +113,14 @@ public class GuiceServletConfig extends GuiceServletContextListener {
         // perform login operations
         LoginProcessor loginProcessor = new LoginProcessor();
         loginProcessor.login();
+
+        startServices();
+    }
+
+    protected void startServices() {
+        LOG.debug("Starting services");
+        Services services = injector.getInstance(Services.class);
+        services.start();
     }
 
     /**
@@ -132,6 +143,15 @@ public class GuiceServletConfig extends GuiceServletContextListener {
             Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType));
             final Graph graph = graphProvider.get().get();
             graph.shutdown();
+
+            //stop services
+            stopServices();
         }
     }
+
+    protected void stopServices() {
+        LOG.debug("Stopping services");
+        Services services = injector.getInstance(Services.class);
+        services.stop();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/resources/EntitiesResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/EntitiesResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/EntitiesResource.java
new file mode 100644
index 0000000..8cff0c5
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/EntitiesResource.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.web.resources;
+
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.repository.EntityExistsException;
+import org.apache.atlas.services.MetadataService;
+import org.apache.atlas.typesystem.types.ValueConversionException;
+import org.apache.atlas.web.util.Servlets;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Singleton;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriInfo;
+import java.net.URI;
+import java.util.List;
+
+@Path("entities")
+@Singleton
+public class EntitiesResource {
+    private static final Logger LOG = LoggerFactory.getLogger(EntitiesResource.class);
+
+    @Inject
+    private MetadataService metadataService;
+
+    @Context
+    UriInfo uriInfo;
+
+    /**
+     * Submits the entity definitions (instances).
+     * The body contains the JSONArray of entity json. The service takes care of de-duping the entities based on any
+     * unique attribute for the give type.
+     */
+    @POST
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public Response submit(@Context HttpServletRequest request) {
+        try {
+            final String entities = Servlets.getRequestPayload(request);
+            LOG.debug("submitting entities {} ", AtlasClient.toString(new JSONArray(entities)));
+
+            final String guids = metadataService.createEntities(entities);
+
+            UriBuilder ub = uriInfo.getAbsolutePathBuilder();
+            URI locationURI = ub.path(guids).build();
+
+            JSONObject response = new JSONObject();
+            response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
+            response.put(AtlasClient.GUID, new JSONArray(guids));
+            response.put(AtlasClient.DEFINITION, metadataService.getEntityDefinition(new JSONArray(guids).getString(0)));
+
+            return Response.created(locationURI).entity(response).build();
+
+        } catch(EntityExistsException e) {
+            LOG.error("Unique constraint violation", e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.CONFLICT));
+        } catch (ValueConversionException ve) {
+            LOG.error("Unable to persist entity instance due to a desrialization error ", ve);
+            throw new WebApplicationException(Servlets.getErrorResponse(ve.getCause(), Response.Status.BAD_REQUEST));
+        } catch (AtlasException | IllegalArgumentException e) {
+            LOG.error("Unable to persist entity instance", e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+        } catch (Throwable e) {
+            LOG.error("Unable to persist entity instance", e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
+        }
+    }
+
+    /**
+     * Gets the list of entities for a given entity type.
+     *
+     * @param entityType name of a type which is unique
+     */
+    @GET
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public Response getEntityListByType(@QueryParam("type") String entityType) {
+        try {
+            Preconditions.checkNotNull(entityType, "Entity type cannot be null");
+
+            LOG.debug("Fetching entity list for type={} ", entityType);
+            final List<String> entityList = metadataService.getEntityList(entityType);
+
+            JSONObject response = new JSONObject();
+            response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
+            response.put(AtlasClient.TYPENAME, entityType);
+            response.put(AtlasClient.RESULTS, new JSONArray(entityList));
+            response.put(AtlasClient.COUNT, entityList.size());
+
+            return Response.ok(response).build();
+        } catch (NullPointerException e) {
+            LOG.error("Entity type cannot be null", e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+        } catch (AtlasException | IllegalArgumentException e) {
+            LOG.error("Unable to get entity list for type {}", entityType, e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+        } catch (Throwable e) {
+            LOG.error("Unable to get entity list for type {}", entityType, e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
index dcb460d..d8ffe9f 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
@@ -25,7 +25,6 @@ import org.apache.atlas.ParamChecker;
 import org.apache.atlas.TypeNotFoundException;
 import org.apache.atlas.repository.EntityNotFoundException;
 import org.apache.atlas.services.MetadataService;
-import org.apache.atlas.typesystem.types.ValueConversionException;
 import org.apache.atlas.web.util.Servlets;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONObject;
@@ -59,7 +58,7 @@ import java.util.List;
  * An entity is an "instance" of a Type.  Entities conform to the definition
  * of the Type they correspond with.
  */
-@Path("entities")
+@Path("entity")
 @Singleton
 public class EntityResource {
 
@@ -82,40 +81,6 @@ public class EntityResource {
         this.metadataService = metadataService;
     }
 
-    /**
-     * Submits an entity definition (instance) corresponding to a given type.
-     */
-    @POST
-    @Consumes(Servlets.JSON_MEDIA_TYPE)
-    @Produces(Servlets.JSON_MEDIA_TYPE)
-    public Response submit(@Context HttpServletRequest request) {
-        try {
-            final String entity = Servlets.getRequestPayload(request);
-            LOG.debug("submitting entity {} ", entity);
-
-            final String guid = metadataService.createEntity(entity);
-
-            UriBuilder ub = uriInfo.getAbsolutePathBuilder();
-            URI locationURI = ub.path(guid).build();
-
-            JSONObject response = new JSONObject();
-            response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
-            response.put(AtlasClient.GUID, guid);
-            response.put(AtlasClient.DEFINITION, metadataService.getEntityDefinition(guid));
-
-            return Response.created(locationURI).entity(response).build();
-
-        } catch (ValueConversionException ve) {
-            LOG.error("Unable to persist entity instance due to a desrialization error ", ve);
-            throw new WebApplicationException(Servlets.getErrorResponse(ve.getCause(), Response.Status.BAD_REQUEST));
-        } catch (AtlasException | IllegalArgumentException e) {
-            LOG.error("Unable to persist entity instance", e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
-        } catch (Throwable e) {
-            LOG.error("Unable to persist entity instance", e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
-        }
-    }
 
     /**
      * Fetch the complete definition of an entity given its GUID.
@@ -159,34 +124,47 @@ public class EntityResource {
     }
 
     /**
-     * Gets the list of entities for a given entity type.
+     * Fetch the complete definition of an entity given its qualified name.
      *
-     * @param entityType name of a type which is unique
+     * @param entityType
+     * @param attribute
+     * @param value
      */
     @GET
     @Produces(Servlets.JSON_MEDIA_TYPE)
-    public Response getEntityListByType(@QueryParam("type") String entityType) {
+    public Response getEntityDefinitionByAttribute(@QueryParam("type") String entityType,
+                                                   @QueryParam("property") String attribute,
+                                                   @QueryParam("value") String value) {
         try {
-            Preconditions.checkNotNull(entityType, "Entity type cannot be null");
+            LOG.debug("Fetching entity definition for type={}, qualified name={}", entityType, value);
+            ParamChecker.notEmpty(entityType, "type cannot be null");
+            ParamChecker.notEmpty(attribute, "attribute name cannot be null");
+            ParamChecker.notEmpty(value, "attribute value cannot be null");
 
-            LOG.debug("Fetching entity list for type={} ", entityType);
-            final List<String> entityList = metadataService.getEntityList(entityType);
+            final String entityDefinition = metadataService.getEntityDefinition(entityType, attribute, value);
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
-            response.put(AtlasClient.TYPENAME, entityType);
-            response.put(AtlasClient.RESULTS, new JSONArray(entityList));
-            response.put(AtlasClient.COUNT, entityList.size());
 
-            return Response.ok(response).build();
-        } catch (NullPointerException e) {
-            LOG.error("Entity type cannot be null", e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+            Response.Status status = Response.Status.NOT_FOUND;
+            if (entityDefinition != null) {
+                response.put(AtlasClient.DEFINITION, entityDefinition);
+                status = Response.Status.OK;
+            } else {
+                response.put(AtlasClient.ERROR, Servlets.escapeJsonString(String.format("An entity with type={%s}, " +
+                        "qualifiedName={%s} does not exist", entityType, value)));
+            }
+
+            return Response.status(status).entity(response).build();
+
+        } catch (EntityNotFoundException e) {
+            LOG.error("An entity with type={} and qualifiedName={} does not exist", entityType, value, e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
         } catch (AtlasException | IllegalArgumentException e) {
-            LOG.error("Unable to get entity list for type {}", entityType, e);
+            LOG.error("Bad type={}, qualifiedName={}", entityType, value, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
-            LOG.error("Unable to get entity list for type {}", entityType, e);
+            LOG.error("Unable to get instance definition for type={}, qualifiedName={}", entityType, value, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/resources/MetadataDiscoveryResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/MetadataDiscoveryResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/MetadataDiscoveryResource.java
index 9248979..355e491 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/MetadataDiscoveryResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/MetadataDiscoveryResource.java
@@ -88,28 +88,16 @@ public class MetadataDiscoveryResource {
 
             final String jsonResultStr = discoveryService.searchByDSL(query);
             response = new DSLJSONResponseBuilder().results(jsonResultStr).query(query).build();
+            return Response.ok(response).build();
 
         } catch (IllegalArgumentException e) {
             LOG.error("Unable to get entity list for empty query", e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+
         } catch (Throwable throwable) {
             LOG.error("Unable to get entity list for query {} using dsl", query, throwable);
-
-            try {   //fall back to full-text
-                final String jsonResultStr = discoveryService.searchByFullText(query);
-                response = new FullTextJSonResponseBuilder().results(jsonResultStr).query(query).build();
-
-            } catch (DiscoveryException | IllegalArgumentException e) {
-                LOG.error("Unable to get entity list for query {}", query, e);
-                throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
-            } catch (Throwable e) {
-                LOG.error("Unable to get entity list for query {}", query, e);
-                throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
-            }
+            return searchUsingFullText(query);
         }
-
-        return Response.ok(response).build();
-
     }
 
     /**
@@ -267,7 +255,8 @@ public class MetadataDiscoveryResource {
             count(rowsJsonArr.length());
             queryType(QUERY_TYPE_DSL);
             JSONObject response = super.build();
-            response.put(AtlasClient.RESULTS, dslResults);
+            response.put(AtlasClient.RESULTS, rowsJsonArr);
+            response.put(AtlasClient.DATATYPE, dslResults.get(AtlasClient.DATATYPE));
             return response;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
index d6d5915..e4e18f5 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
@@ -21,6 +21,7 @@ package org.apache.atlas.web.resources;
 import com.sun.jersey.api.client.ClientResponse;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
+import org.apache.atlas.TypeExistsException;
 import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.web.util.Servlets;
@@ -96,6 +97,9 @@ public class TypesResource {
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
             response.put(AtlasClient.TYPES, typesResponse);
             return Response.status(ClientResponse.Status.CREATED).entity(response).build();
+        } catch (TypeExistsException e) {
+            LOG.error("Type already exists", e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.CONFLICT));
         } catch (AtlasException | IllegalArgumentException e) {
             LOG.error("Unable to persist types", e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/java/org/apache/atlas/web/service/SecureEmbeddedServer.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/service/SecureEmbeddedServer.java b/webapp/src/main/java/org/apache/atlas/web/service/SecureEmbeddedServer.java
index 1b2192c..3e6ec26 100755
--- a/webapp/src/main/java/org/apache/atlas/web/service/SecureEmbeddedServer.java
+++ b/webapp/src/main/java/org/apache/atlas/web/service/SecureEmbeddedServer.java
@@ -20,8 +20,6 @@ package org.apache.atlas.web.service;
 
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasException;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/main/resources/atlas-log4j.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/resources/atlas-log4j.xml b/webapp/src/main/resources/atlas-log4j.xml
index 7827c1a..51aa411 100755
--- a/webapp/src/main/resources/atlas-log4j.xml
+++ b/webapp/src/main/resources/atlas-log4j.xml
@@ -50,7 +50,6 @@
         <appender-ref ref="FILE"/>
     </logger>
 
-
     <logger name="AUDIT">
         <level value="info"/>
         <appender-ref ref="AUDIT"/>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerIT.java b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerIT.java
new file mode 100644
index 0000000..e03f618
--- /dev/null
+++ b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerIT.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.notification;
+
+import com.google.inject.Inject;
+import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.json.InstanceSerialization;
+import org.apache.atlas.web.resources.BaseResourceIT;
+import org.codehaus.jettison.json.JSONArray;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+@Guice(modules = NotificationModule.class)
+public class NotificationHookConsumerIT extends BaseResourceIT{
+
+    @Inject
+    private NotificationInterface kafka;
+    private String dbName;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        super.setUp();
+        createTypeDefinitions();
+    }
+
+    @AfterClass
+    public void teardown() throws Exception {
+        kafka.close();
+    }
+
+    private void sendHookMessage(Referenceable entity) throws NotificationException {
+        String entityJson = InstanceSerialization.toJson(entity, true);
+        JSONArray jsonArray = new JSONArray();
+        jsonArray.put(entityJson);
+        kafka.send(NotificationInterface.NotificationType.HOOK, jsonArray.toString());
+    }
+
+    @Test
+    public void testConsumeHookMessage() throws Exception {
+        Referenceable entity = new Referenceable(DATABASE_TYPE);
+        dbName = "db" + randomString();
+        entity.set("name", dbName);
+        entity.set("description", randomString());
+
+        sendHookMessage(entity);
+
+        waitFor(1000, new Predicate() {
+            @Override
+            public boolean evaluate() throws Exception {
+                JSONArray results =
+                        serviceClient.searchByDSL(String.format("%s where name='%s'", DATABASE_TYPE, dbName));
+                return results.length() == 1;
+            }
+        });
+    }
+
+    @Test (dependsOnMethods = "testConsumeHookMessage")
+    public void testEnityDeduping() throws Exception {
+//        Referenceable db = serviceClient.getEntity(DATABASE_TYPE, "name", dbName);
+        Referenceable db = new Referenceable(DATABASE_TYPE);
+        db.set("name", dbName);
+        db.set("description", randomString());
+
+        Referenceable table = new Referenceable(HIVE_TABLE_TYPE);
+        final String tableName = randomString();
+        table.set("name", tableName);
+        table.set("db", db);
+
+        sendHookMessage(table);
+        waitFor(1000, new Predicate() {
+            @Override
+            public boolean evaluate() throws Exception {
+                JSONArray results =
+                        serviceClient.searchByDSL(String.format("%s where name='%s'", HIVE_TABLE_TYPE, tableName));
+                return results.length() == 1;
+            }
+        });
+
+        JSONArray results =
+                serviceClient.searchByDSL(String.format("%s where name='%s'", DATABASE_TYPE, dbName));
+        Assert.assertEquals(results.length(), 1);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/1bfda02a/webapp/src/test/java/org/apache/atlas/web/listeners/TestGuiceServletConfig.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/listeners/TestGuiceServletConfig.java b/webapp/src/test/java/org/apache/atlas/web/listeners/TestGuiceServletConfig.java
index 3ba6387..a4f8cce 100644
--- a/webapp/src/test/java/org/apache/atlas/web/listeners/TestGuiceServletConfig.java
+++ b/webapp/src/test/java/org/apache/atlas/web/listeners/TestGuiceServletConfig.java
@@ -21,8 +21,10 @@ import com.google.inject.Provider;
 import com.google.inject.TypeLiteral;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.util.TitanCleanup;
-import com.tinkerpop.blueprints.Graph;
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasException;
 import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.commons.configuration.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,6 +33,7 @@ import javax.servlet.ServletContextEvent;
 public class TestGuiceServletConfig extends GuiceServletConfig {
 
     private static final Logger LOG = LoggerFactory.getLogger(TestGuiceServletConfig.class);
+    private boolean servicesEnabled;
 
     @Override
     public void contextInitialized(ServletContextEvent servletContextEvent) {
@@ -40,6 +43,8 @@ public class TestGuiceServletConfig extends GuiceServletConfig {
 
     @Override
     public void contextDestroyed(ServletContextEvent servletContextEvent) {
+        super.contextDestroyed(servletContextEvent);
+
         if(injector != null) {
             TypeLiteral<GraphProvider<TitanGraph>> graphProviderType = new TypeLiteral<GraphProvider<TitanGraph>>() {};
             Provider<GraphProvider<TitanGraph>> graphProvider = injector.getProvider(Key.get(graphProviderType));
@@ -47,11 +52,30 @@ public class TestGuiceServletConfig extends GuiceServletConfig {
 
             LOG.info("Clearing graph store");
             try {
-                graph.shutdown();
                 TitanCleanup.clear(graph);
             } catch (Exception e) {
                 LOG.warn("Clearing graph store failed ", e);
             }
         }
     }
+
+    @Override
+    protected void startServices() {
+        try {
+            Configuration conf = ApplicationProperties.get();
+            servicesEnabled = conf.getBoolean("atlas.services.enabled", true);
+            if (servicesEnabled) {
+                super.startServices();
+            }
+        } catch (AtlasException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected void stopServices() {
+        if (servicesEnabled) {
+            super.stopServices();
+        }
+    }
 }