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 2016/07/19 06:35:23 UTC

incubator-atlas git commit: ATLAS-957 Atlas is not capturing topologies that have $ in the data payload (shwethags)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 7cc347134 -> d64112d6a


ATLAS-957 Atlas is not capturing topologies that have $ in the data payload (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/d64112d6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/d64112d6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/d64112d6

Branch: refs/heads/master
Commit: d64112d6a95e6279fd54f2164e89857124450b67
Parents: 7cc3471
Author: Shwetha GS <ss...@hortonworks.com>
Authored: Tue Jul 19 11:51:28 2016 +0530
Committer: Shwetha GS <ss...@hortonworks.com>
Committed: Tue Jul 19 11:51:28 2016 +0530

----------------------------------------------------------------------
 release-log.txt                                 |   1 +
 .../graph/DefaultGraphPersistenceStrategy.java  |   6 +-
 .../graph/GraphBackedDiscoveryService.java      |   5 +-
 .../atlas/repository/graph/DeleteHandler.java   |   8 +-
 .../atlas/repository/graph/FullTextMapper.java  |   3 +-
 .../graph/GraphBackedMetadataRepository.java    |   6 +-
 .../graph/GraphBackedSearchIndexer.java         |   2 +-
 .../atlas/repository/graph/GraphHelper.java     |  73 +++++++-
 .../graph/GraphToTypedInstanceMapper.java       |  58 +++---
 .../graph/TypedInstanceToGraphMapper.java       |  10 +-
 .../typestore/GraphBackedTypeStore.java         |  36 ++--
 .../org/apache/atlas/query/GremlinQuery.scala   |  33 ++--
 .../atlas/repository/graph/GraphHelperTest.java |  51 ++++++
 .../service/DefaultMetadataServiceTest.java     | 175 ++++++++++++-------
 14 files changed, 312 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 3b2338e..965cafe 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -6,6 +6,7 @@ INCOMPATIBLE CHANGES:
 
 
 ALL CHANGES:
+ATLAS-957 Atlas is not capturing topologies that have $ in the data payload (shwethags)
 ATLAS-1032 Atlas hook package should not include libraries already present in host component - like log4j (mneethiraj via sumasai)
 ATLAS-1027 Atlas hooks should use properties from atlas-application.properties, instead of component's configuration (mneethiraj via sumasai)
 ATLAS-1030 Add instrumentation to measure performance: REST API (mneethiraj via sumasai)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java b/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
index 8ddbe1b..e07a54e 100755
--- a/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
@@ -140,9 +140,9 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
                 TypeSystem.IdType idType = TypeSystem.getInstance().getIdType();
 
                 if (dataType.getName().equals(idType.getName())) {
-                    structInstance.set(idType.typeNameAttrName(), structVertex.getProperty(typeAttributeName()));
-                    structInstance.set(idType.idAttrName(), structVertex.getProperty(idAttributeName()));
-                    structInstance.set(idType.stateAttrName(), structVertex.getProperty(stateAttributeName()));
+                    structInstance.set(idType.typeNameAttrName(), GraphHelper.getProperty(structVertex, typeAttributeName()));
+                    structInstance.set(idType.idAttrName(), GraphHelper.getProperty(structVertex, idAttributeName()));
+                    structInstance.set(idType.stateAttrName(), GraphHelper.getProperty(structVertex, stateAttributeName()));
                 } else {
                     metadataRepository.getGraphToInstanceMapper()
                         .mapVertexToInstance(structVertex, structInstance, structType.fieldMapping().fields);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java b/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
index f97b83d..6def78b 100755
--- a/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
@@ -36,6 +36,7 @@ import org.apache.atlas.query.QueryParser;
 import org.apache.atlas.query.QueryProcessor;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.MetadataRepository;
+import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graph.GraphProvider;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
@@ -94,11 +95,11 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
             Vertex vertex = result.getElement();
 
             JSONObject row = new JSONObject();
-            String guid = vertex.getProperty(Constants.GUID_PROPERTY_KEY);
+            String guid = GraphHelper.getIdFromVertex(vertex);
             if (guid != null) { //Filter non-class entities
                 try {
                     row.put("guid", guid);
-                    row.put(AtlasClient.TYPENAME, vertex.<String>getProperty(Constants.ENTITY_TYPE_PROPERTY_KEY));
+                    row.put(AtlasClient.TYPENAME, GraphHelper.getTypeName(vertex));
                     row.put(SCORE, result.getScore());
                 } catch (JSONException e) {
                     LOG.error("Unable to create response", e);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
index 91f9bd0..e74e57f 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
@@ -154,7 +154,7 @@ public abstract class DeleteHandler {
 
                 if (valueTypeCategory == DataTypes.TypeCategory.STRUCT ||
                         valueTypeCategory == DataTypes.TypeCategory.CLASS) {
-                    List<String> keys = instanceVertex.getProperty(propertyName);
+                    List<String> keys = GraphHelper.getProperty(instanceVertex, propertyName);
                     if (keys != null) {
                         for (String key : keys) {
                             String mapEdgeLabel = GraphHelper.getQualifiedNameForMapKey(edgeLabel, key);
@@ -286,7 +286,7 @@ public abstract class DeleteHandler {
 
         case ARRAY:
             //If its array attribute, find the right edge between the two vertices and update array property
-            List<String> elements = outVertex.getProperty(propertyName);
+            List<String> elements = GraphHelper.getProperty(outVertex, propertyName);
             if (elements != null) {
                 elements = new ArrayList<>(elements);   //Make a copy, else list.remove reflects on titan.getProperty()
                 for (String elementEdgeId : elements) {
@@ -327,12 +327,12 @@ public abstract class DeleteHandler {
 
         case MAP:
             //If its map attribute, find the right edge between two vertices and update map property
-            List<String> keys = outVertex.getProperty(propertyName);
+            List<String> keys = GraphHelper.getProperty(outVertex, propertyName);
             if (keys != null) {
                 keys = new ArrayList<>(keys);   //Make a copy, else list.remove reflects on titan.getProperty()
                 for (String key : keys) {
                     String keyPropertyName = GraphHelper.getQualifiedNameForMapKey(propertyName, key);
-                    String mapEdgeId = outVertex.getProperty(keyPropertyName);
+                    String mapEdgeId = GraphHelper.getProperty(outVertex, keyPropertyName);
                     Edge mapEdge = graphHelper.getEdgeByEdgeId(outVertex, keyPropertyName, mapEdgeId);
                     Vertex mapVertex = mapEdge.getVertex(Direction.IN);
                     if (mapVertex.getId().toString().equals(inVertex.getId().toString())) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
index 37cacb0..b342e27 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
@@ -19,7 +19,6 @@ package org.apache.atlas.repository.graph;
 
 import com.tinkerpop.blueprints.Vertex;
 import org.apache.atlas.AtlasException;
-import org.apache.atlas.repository.Constants;
 import org.apache.atlas.typesystem.ITypedInstance;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.types.AttributeInfo;
@@ -51,7 +50,7 @@ public class FullTextMapper {
     }
 
     public String mapRecursive(Vertex instanceVertex, boolean followReferences) throws AtlasException {
-        String guid = instanceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+        String guid = GraphHelper.getIdFromVertex(instanceVertex);
         ITypedReferenceableInstance typedReference;
         if (instanceCache.containsKey(guid)) {
             typedReference = instanceCache.get(guid);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
index e5c0e46..73445cb 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
@@ -115,7 +115,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         if (aInfo.name.startsWith(Constants.INTERNAL_PROPERTY_KEY_PREFIX)) {
             return aInfo.name;
         }
-        return GraphHelper.getQualifiedFieldName(dataType, aInfo.name);
+        return GraphHelper.encodePropertyKey(GraphHelper.getQualifiedFieldName(dataType, aInfo.name));
     }
 
     public String getFieldNameInVertex(IDataType<?> dataType, String attrName) throws AtlasException {
@@ -168,7 +168,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
                 Constants.ENTITY_TYPE_PROPERTY_KEY, entityType,
                 Constants.STATE_PROPERTY_KEY, Id.EntityState.ACTIVE.name());
 
-        String guid = instanceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+        String guid = GraphHelper.getIdFromVertex(instanceVertex);
         return graphToInstanceMapper.mapGraphToTypedInstance(guid, instanceVertex);
     }
 
@@ -185,7 +185,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         ArrayList<String> entityList = new ArrayList<>();
         while (results.hasNext()) {
             Vertex vertex = results.next();
-            entityList.add(vertex.<String>getProperty(Constants.GUID_PROPERTY_KEY));
+            entityList.add(GraphHelper.getIdFromVertex(vertex));
         }
 
         return entityList;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
index 8c8134f..e960c2f 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
@@ -227,7 +227,7 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
     }
 
     private void createIndexForAttribute(TitanManagement management, String typeName, AttributeInfo field) {
-        final String propertyName = typeName + "." + field.name;
+        final String propertyName = GraphHelper.encodePropertyKey(typeName + "." + field.name);
         switch (field.dataType().getTypeCategory()) {
         case PRIMITIVE:
             Cardinality cardinality = getCardinality(field.multiplicity);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
index 4f531e2..b3711b5 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
@@ -18,6 +18,9 @@
 
 package org.apache.atlas.repository.graph;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.TitanProperty;
 import com.thinkaurelius.titan.core.TitanVertex;
@@ -41,11 +44,13 @@ import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.HierarchicalType;
 import org.apache.atlas.typesystem.types.IDataType;
 import org.apache.atlas.typesystem.types.TypeSystem;
+import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
@@ -215,7 +220,7 @@ public final class GraphHelper {
                 LOG.debug("Found {}", string(edge));
                 return edge;
             } else {
-                Long modificationTime = edge.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
+                Long modificationTime = getProperty(edge, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
                 if (modificationTime != null && modificationTime >= latestDeletedEdgeTime) {
                     latestDeletedEdgeTime = modificationTime;
                     latestDeletedEdge = edge;
@@ -244,21 +249,36 @@ public final class GraphHelper {
 
     public static <T extends Element> void setProperty(T element, String propertyName, Object value) {
         String elementStr = string(element);
-        LOG.debug("Setting property {} = \"{}\" to {}", propertyName, value, elementStr);
-        Object existValue = element.getProperty(propertyName);
+        String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
+        LOG.debug("Setting property {} = \"{}\" to {}", actualPropertyName, value, elementStr);
+        Object existValue = element.getProperty(actualPropertyName);
         if(value == null || (value instanceof Collection && ((Collection) value).isEmpty())) {
             if(existValue != null) {
-                LOG.info("Removing property - {} value from {}", propertyName, elementStr);
-                element.removeProperty(propertyName);
+                LOG.info("Removing property - {} value from {}", actualPropertyName, elementStr);
+                element.removeProperty(actualPropertyName);
             }
         } else {
             if (!value.equals(existValue)) {
-                element.setProperty(propertyName, value);
-                LOG.debug("Set property {} = \"{}\" to {}", propertyName, value, elementStr);
+                element.setProperty(actualPropertyName, value);
+                LOG.debug("Set property {} = \"{}\" to {}", actualPropertyName, value, elementStr);
             }
         }
     }
 
+    public static <T extends Element, O> O getProperty(T element, String propertyName) {
+        String elementStr = string(element);
+        String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
+        LOG.debug("Reading property {} from {}", actualPropertyName, elementStr);
+        return element.getProperty(actualPropertyName);
+    }
+
+    public static Iterable<TitanProperty> getProperties(TitanVertex vertex, String propertyName) {
+        String elementStr = string(vertex);
+        String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
+        LOG.debug("Reading property {} from {}", actualPropertyName, elementStr);
+        return vertex.getProperties(actualPropertyName);
+    }
+
     private static <T extends Element> String string(T element) {
         if (element instanceof Vertex) {
             return string((Vertex) element);
@@ -339,8 +359,8 @@ public final class GraphHelper {
     }
 
     public static Id getIdFromVertex(String dataTypeName, Vertex vertex) {
-        return new Id(vertex.<String>getProperty(Constants.GUID_PROPERTY_KEY),
-            vertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), dataTypeName);
+        return new Id(getIdFromVertex(vertex),
+            vertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), dataTypeName, getStateAsString(vertex));
     }
 
     public static String getIdFromVertex(Vertex vertex) {
@@ -425,4 +445,39 @@ public final class GraphHelper {
             return String.format("edge[id=%s]", edge.getId().toString());
         }
     }
+
+    @VisibleForTesting
+    //Keys copied from com.thinkaurelius.titan.graphdb.types.StandardRelationTypeMaker
+    //Titan checks that these chars are not part of any keys. So, encoding...
+    public static BiMap<String, String> RESERVED_CHARS_ENCODE_MAP =
+            HashBiMap.create(new HashMap<String, String>() {{
+                put("{", "_o");
+                put("}", "_c");
+                put("\"", "_q");
+                put("$", "_d");
+                put("%", "_p");
+            }});
+
+
+    public static String encodePropertyKey(String key) {
+        if (StringUtils.isBlank(key)) {
+            return key;
+        }
+
+        for (String str : RESERVED_CHARS_ENCODE_MAP.keySet()) {
+            key = key.replace(str, RESERVED_CHARS_ENCODE_MAP.get(str));
+        }
+        return key;
+    }
+
+    public static String decodePropertyKey(String key) {
+        if (StringUtils.isBlank(key)) {
+            return key;
+        }
+
+        for (String encodedStr : RESERVED_CHARS_ENCODE_MAP.values()) {
+            key = key.replace(encodedStr, RESERVED_CHARS_ENCODE_MAP.inverse().get(encodedStr));
+        }
+        return key;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
index e240fb6..69223f5 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
@@ -47,6 +47,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.atlas.repository.graph.GraphHelper.getIdFromVertex;
 import static org.apache.atlas.repository.graph.GraphHelper.string;
 
 @Singleton
@@ -66,11 +67,12 @@ public final class GraphToTypedInstanceMapper {
         throws AtlasException {
 
         LOG.debug("Mapping graph root vertex {} to typed instance for guid {}", instanceVertex, guid);
-        String typeName = instanceVertex.getProperty(Constants.ENTITY_TYPE_PROPERTY_KEY);
+        String typeName = GraphHelper.getProperty(instanceVertex, Constants.ENTITY_TYPE_PROPERTY_KEY);
         List<String> traits = GraphHelper.getTraitNames(instanceVertex);
         String state = GraphHelper.getStateAsString(instanceVertex);
 
-        Id id = new Id(guid, instanceVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), typeName, state);
+        Id id = new Id(guid, (Integer) GraphHelper.getProperty(instanceVertex, Constants.VERSION_PROPERTY_KEY),
+                typeName, state);
         LOG.debug("Created id {} for instance type {}", id, typeName);
 
         ClassType classType = typeSystem.getDataType(ClassType.class, typeName);
@@ -115,13 +117,12 @@ public final class GraphToTypedInstanceMapper {
             break;  // add only if vertex has this attribute
 
         case ENUM:
-            if (instanceVertex.getProperty(vertexPropertyName) == null) {
+            Object propertyValue = GraphHelper.getProperty(instanceVertex, vertexPropertyName);
+            if (propertyValue == null) {
                 return;
             }
 
-            typedInstance.set(attributeInfo.name,
-                dataType.convert(instanceVertex.<String>getProperty(vertexPropertyName),
-                    Multiplicity.REQUIRED));
+            typedInstance.set(attributeInfo.name, dataType.convert(propertyValue, Multiplicity.REQUIRED));
             break;
 
         case ARRAY:
@@ -168,17 +169,14 @@ public final class GraphToTypedInstanceMapper {
 
         if (edge != null) {
             final Vertex referenceVertex = edge.getVertex(Direction.IN);
-            final String guid = referenceVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+            final String guid = GraphHelper.getIdFromVertex(referenceVertex);
             LOG.debug("Found vertex {} for label {} with guid {}", referenceVertex, relationshipLabel, guid);
             if (attributeInfo.isComposite) {
                 //Also, when you retrieve a type's instance, you get the complete object graph of the composites
                 LOG.debug("Found composite, mapping vertex to instance");
                 return mapGraphToTypedInstance(guid, referenceVertex);
             } else {
-                String state = GraphHelper.getStateAsString(referenceVertex);
-                Id referenceId =
-                        new Id(guid, referenceVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY),
-                                dataType.getName(), state);
+                Id referenceId = getIdFromVertex(dataType.getName(), referenceVertex);
                 LOG.debug("Found non-composite, adding id {} ", referenceId);
                 return referenceId;
             }
@@ -191,7 +189,7 @@ public final class GraphToTypedInstanceMapper {
     private void mapVertexToArrayInstance(Vertex instanceVertex, ITypedInstance typedInstance,
         AttributeInfo attributeInfo, String propertyName) throws AtlasException {
         LOG.debug("mapping vertex {} to array {}", instanceVertex, attributeInfo.name);
-        List list = instanceVertex.getProperty(propertyName);
+        List list = GraphHelper.getProperty(instanceVertex, propertyName);
         if (list == null || list.size() == 0) {
             return;
         }
@@ -240,7 +238,7 @@ public final class GraphToTypedInstanceMapper {
     private void mapVertexToMapInstance(Vertex instanceVertex, ITypedInstance typedInstance,
         AttributeInfo attributeInfo, final String propertyName) throws AtlasException {
         LOG.debug("mapping vertex {} to array {}", instanceVertex, attributeInfo.name);
-        List<String> keys = instanceVertex.getProperty(propertyName);
+        List<String> keys = GraphHelper.getProperty(instanceVertex, propertyName);
         if (keys == null || keys.size() == 0) {
             return;
         }
@@ -251,7 +249,7 @@ public final class GraphToTypedInstanceMapper {
         for (String key : keys) {
             final String keyPropertyName = propertyName + "." + key;
             final String edgeLabel = GraphHelper.EDGE_LABEL_PREFIX + keyPropertyName;
-            final Object keyValue = instanceVertex.getProperty(keyPropertyName);
+            final Object keyValue = GraphHelper.getProperty(instanceVertex, keyPropertyName);
             Object mapValue = mapVertexToCollectionEntry(instanceVertex, attributeInfo, valueType, keyValue, edgeLabel);
             if (mapValue != null) {
                 values.put(key, mapValue);
@@ -312,33 +310,33 @@ public final class GraphToTypedInstanceMapper {
         AttributeInfo attributeInfo) throws AtlasException {
         LOG.debug("Adding primitive {} from vertex {}", attributeInfo, instanceVertex);
         final String vertexPropertyName = GraphHelper.getQualifiedFieldName(typedInstance, attributeInfo);
-        if (instanceVertex.getProperty(vertexPropertyName) == null) {
+        Object propertyValue = GraphHelper.getProperty(instanceVertex, vertexPropertyName);
+        if (propertyValue == null) {
             return;
         }
 
         if (attributeInfo.dataType() == DataTypes.STRING_TYPE) {
-            typedInstance.setString(attributeInfo.name, instanceVertex.<String>getProperty(vertexPropertyName));
+            typedInstance.setString(attributeInfo.name, (String) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.SHORT_TYPE) {
-            typedInstance.setShort(attributeInfo.name, instanceVertex.<Short>getProperty(vertexPropertyName));
+            typedInstance.setShort(attributeInfo.name, (Short) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.INT_TYPE) {
-            typedInstance.setInt(attributeInfo.name, instanceVertex.<Integer>getProperty(vertexPropertyName));
+            typedInstance.setInt(attributeInfo.name, (Integer) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.BIGINTEGER_TYPE) {
-            typedInstance.setBigInt(attributeInfo.name, instanceVertex.<BigInteger>getProperty(vertexPropertyName));
+            typedInstance.setBigInt(attributeInfo.name, (BigInteger) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.BOOLEAN_TYPE) {
-            typedInstance.setBoolean(attributeInfo.name, instanceVertex.<Boolean>getProperty(vertexPropertyName));
+            typedInstance.setBoolean(attributeInfo.name, (Boolean) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.BYTE_TYPE) {
-            typedInstance.setByte(attributeInfo.name, instanceVertex.<Byte>getProperty(vertexPropertyName));
+            typedInstance.setByte(attributeInfo.name, (Byte) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.LONG_TYPE) {
-            typedInstance.setLong(attributeInfo.name, instanceVertex.<Long>getProperty(vertexPropertyName));
+            typedInstance.setLong(attributeInfo.name, (Long) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.FLOAT_TYPE) {
-            typedInstance.setFloat(attributeInfo.name, instanceVertex.<Float>getProperty(vertexPropertyName));
+            typedInstance.setFloat(attributeInfo.name, (Float) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.DOUBLE_TYPE) {
-            typedInstance.setDouble(attributeInfo.name, instanceVertex.<Double>getProperty(vertexPropertyName));
+            typedInstance.setDouble(attributeInfo.name, (Double) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.BIGDECIMAL_TYPE) {
-            typedInstance
-                .setBigDecimal(attributeInfo.name, instanceVertex.<BigDecimal>getProperty(vertexPropertyName));
+            typedInstance.setBigDecimal(attributeInfo.name, (BigDecimal) propertyValue);
         } else if (attributeInfo.dataType() == DataTypes.DATE_TYPE) {
-            final Long dateVal = instanceVertex.<Long>getProperty(vertexPropertyName);
+            final Long dateVal = (Long) propertyValue;
             typedInstance.setDate(attributeInfo.name, new Date(dateVal));
         }
     }
@@ -359,11 +357,7 @@ public final class GraphToTypedInstanceMapper {
                     return instance;
                 case CLASS:
                     //TODO isComposite handling for class loads
-                    final String guid = referredVertex.getProperty(Constants.GUID_PROPERTY_KEY);
-                    Id referenceId =
-                        new Id(guid, referredVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY),
-                            referredType.getName());
-                    return referenceId;
+                    return GraphHelper.getIdFromVertex(referredType.getName(), referredVertex);
                 default:
                     throw new UnsupportedOperationException("Loading " + referredType.getTypeCategory() + " is not supported");
                 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
index 4c1f559..f4f9ec1 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
@@ -324,7 +324,7 @@ public final class TypedInstanceToGraphMapper {
         }
 
         String propertyName = GraphHelper.getQualifiedFieldName(typedInstance, attributeInfo);
-        List<String> currentElements = instanceVertex.getProperty(propertyName);
+        List<String> currentElements = GraphHelper.getProperty(instanceVertex, propertyName);
         IDataType elementType = ((DataTypes.ArrayType) attributeInfo.dataType()).getElemType();
         List<Object> newElementsCreated = new ArrayList<>();
 
@@ -403,11 +403,11 @@ public final class TypedInstanceToGraphMapper {
         Map<String, String> currentMap = new HashMap<>();
         Map<String, Object> newMap = new HashMap<>();
 
-        List<String> currentKeys = instanceVertex.getProperty(propertyName);
+        List<String> currentKeys = GraphHelper.getProperty(instanceVertex, propertyName);
         if (currentKeys != null && !currentKeys.isEmpty()) {
             for (String key : currentKeys) {
                 String propertyNameForKey = GraphHelper.getQualifiedNameForMapKey(propertyName, key);
-                String propertyValueForKey = instanceVertex.getProperty(propertyNameForKey).toString();
+                String propertyValueForKey = GraphHelper.getProperty(instanceVertex, propertyNameForKey).toString();
                 currentMap.put(key, propertyValueForKey);
             }
         }
@@ -562,7 +562,7 @@ public final class TypedInstanceToGraphMapper {
         // Update attributes
         final MessageDigest digester = MD5Utils.getDigester();
         String newSignature = newAttributeValue.getSignatureHash(digester);
-        String curSignature = structInstanceVertex.getProperty(SIGNATURE_HASH_PROPERTY_KEY);
+        String curSignature = GraphHelper.getProperty(structInstanceVertex, SIGNATURE_HASH_PROPERTY_KEY);
 
         if (!newSignature.equals(curSignature)) {
             //Update struct vertex instance only if there is a change
@@ -622,7 +622,7 @@ public final class TypedInstanceToGraphMapper {
 
         if (id.isUnassigned()) {
             Vertex classVertex = idToVertexMap.get(id);
-            String guid = classVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+            String guid = GraphHelper.getIdFromVertex(classVertex);
             id = new Id(guid, 0, typedReference.getTypeName());
         }
         return id;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java b/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
index 4503899..58ef675 100755
--- a/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
+++ b/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
@@ -26,7 +26,6 @@ import com.thinkaurelius.titan.core.TitanGraph;
 import com.tinkerpop.blueprints.Direction;
 import com.tinkerpop.blueprints.Edge;
 import com.tinkerpop.blueprints.Vertex;
-
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.repository.Constants;
@@ -59,6 +58,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.atlas.repository.graph.GraphHelper.setProperty;
+
 @Singleton
 public class GraphBackedTypeStore implements ITypeStore {
     public static final String VERTEX_TYPE = "typeSystem";
@@ -106,20 +107,15 @@ public class GraphBackedTypeStore implements ITypeStore {
         }
     }
 
-    private void addProperty(Vertex vertex, String propertyName, Object value) {
-        LOG.debug("Setting property {} = \"{}\" to vertex {}", propertyName, value, vertex);
-        vertex.setProperty(propertyName, value);
-    }
-
     private void storeInGraph(EnumType dataType) {
         Vertex vertex = createVertex(dataType.getTypeCategory(), dataType.getName(), dataType.getDescription());
         List<String> values = new ArrayList<>(dataType.values().size());
         for (EnumValue enumValue : dataType.values()) {
             String key = getPropertyKey(dataType.getName(), enumValue.value);
-            addProperty(vertex, key, enumValue.ordinal);
+            setProperty(vertex, key, enumValue.ordinal);
             values.add(enumValue.value);
         }
-        addProperty(vertex, getPropertyKey(dataType.getName()), values);
+        setProperty(vertex, getPropertyKey(dataType.getName()), values);
     }
 
     private String getPropertyKey(String name) {
@@ -142,7 +138,7 @@ public class GraphBackedTypeStore implements ITypeStore {
             for (AttributeInfo attribute : attributes) {
                 String propertyKey = getPropertyKey(typeName, attribute.name);
                 try {
-                    addProperty(vertex, propertyKey, attribute.toJson());
+                    setProperty(vertex, propertyKey, attribute.toJson());
                 } catch (JSONException e) {
                     throw new StorageException(typeName, e);
                 }
@@ -150,7 +146,7 @@ public class GraphBackedTypeStore implements ITypeStore {
                 addReferencesForAttribute(typeSystem, vertex, attribute);
             }
         }
-        addProperty(vertex, getPropertyKey(typeName), attrNames);
+        setProperty(vertex, getPropertyKey(typeName), attrNames);
 
         //Add edges for hierarchy
         if (superTypes != null) {
@@ -272,10 +268,10 @@ public class GraphBackedTypeStore implements ITypeStore {
         String typeName = vertex.getProperty(Constants.TYPENAME_PROPERTY_KEY);
         String typeDescription = vertex.getProperty(Constants.TYPEDESCRIPTION_PROPERTY_KEY);
         List<EnumValue> enumValues = new ArrayList<>();
-        List<String> values = vertex.getProperty(getPropertyKey(typeName));
+        List<String> values = graphHelper.getProperty(vertex, getPropertyKey(typeName));
         for (String value : values) {
             String valueProperty = getPropertyKey(typeName, value);
-            enumValues.add(new EnumValue(value, vertex.<Integer>getProperty(valueProperty)));
+            enumValues.add(new EnumValue(value, (Integer) graphHelper.getProperty(vertex, valueProperty)));
         }
         return new EnumTypeDefinition(typeName, typeDescription, enumValues.toArray(new EnumValue[enumValues.size()]));
     }
@@ -292,12 +288,12 @@ public class GraphBackedTypeStore implements ITypeStore {
 
     private AttributeDefinition[] getAttributes(Vertex vertex, String typeName) throws AtlasException {
         List<AttributeDefinition> attributes = new ArrayList<>();
-        List<String> attrNames = vertex.getProperty(getPropertyKey(typeName));
+        List<String> attrNames = graphHelper.getProperty(vertex, getPropertyKey(typeName));
         if (attrNames != null) {
             for (String attrName : attrNames) {
                 try {
                     String propertyKey = getPropertyKey(typeName, attrName);
-                    attributes.add(AttributeInfo.fromJson((String) vertex.getProperty(propertyKey)));
+                    attributes.add(AttributeInfo.fromJson((String) graphHelper.getProperty(vertex, propertyKey)));
                 } catch (JSONException e) {
                     throw new AtlasException(e);
                 }
@@ -306,10 +302,6 @@ public class GraphBackedTypeStore implements ITypeStore {
         return attributes.toArray(new AttributeDefinition[attributes.size()]);
     }
 
-    private String toString(Vertex vertex) {
-        return PROPERTY_PREFIX + vertex.getProperty(Constants.TYPENAME_PROPERTY_KEY);
-    }
-
     /**
      * Find vertex for the given type category and name, else create new vertex
      * @param category
@@ -333,14 +325,14 @@ public class GraphBackedTypeStore implements ITypeStore {
         if (vertex == null) {
             LOG.debug("Adding vertex {}{}", PROPERTY_PREFIX, typeName);
             vertex = titanGraph.addVertex(null);
-            addProperty(vertex, Constants.VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE); // Mark as type vertex
-            addProperty(vertex, Constants.TYPE_CATEGORY_PROPERTY_KEY, category);
-            addProperty(vertex, Constants.TYPENAME_PROPERTY_KEY, typeName);
+            setProperty(vertex, Constants.VERTEX_TYPE_PROPERTY_KEY, VERTEX_TYPE); // Mark as type vertex
+            setProperty(vertex, Constants.TYPE_CATEGORY_PROPERTY_KEY, category);
+            setProperty(vertex, Constants.TYPENAME_PROPERTY_KEY, typeName);
         }
         if (typeDescription != null) {
             String oldDescription = getPropertyKey(Constants.TYPEDESCRIPTION_PROPERTY_KEY);
             if (!typeDescription.equals(oldDescription)) {
-                addProperty(vertex, Constants.TYPEDESCRIPTION_PROPERTY_KEY, typeDescription);
+                setProperty(vertex, Constants.TYPEDESCRIPTION_PROPERTY_KEY, typeDescription);
             }
         } else {
             LOG.debug(" type description is null ");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/main/scala/org/apache/atlas/query/GremlinQuery.scala
----------------------------------------------------------------------
diff --git a/repository/src/main/scala/org/apache/atlas/query/GremlinQuery.scala b/repository/src/main/scala/org/apache/atlas/query/GremlinQuery.scala
index 8add6c8..d76b173 100755
--- a/repository/src/main/scala/org/apache/atlas/query/GremlinQuery.scala
+++ b/repository/src/main/scala/org/apache/atlas/query/GremlinQuery.scala
@@ -18,15 +18,16 @@
 
 package org.apache.atlas.query
 
-import org.apache.atlas.query.TypeUtils.FieldInfo;
+import org.apache.atlas.query.TypeUtils.FieldInfo
 import org.apache.atlas.query.Expressions._
-import org.apache.atlas.typesystem.types.{TypeSystem, DataTypes}
+import org.apache.atlas.typesystem.types.{DataTypes, TypeSystem}
 import org.apache.atlas.typesystem.types.DataTypes.TypeCategory
 import org.joda.time.format.ISODateTimeFormat
 
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 import org.apache.atlas.typesystem.types.IDataType
+import org.apache.commons.lang.StringEscapeUtils
 
 trait IntSequence {
     def next: Int
@@ -111,7 +112,8 @@ trait SelectExpressionHandling {
     /**
      * For each Output Column in the SelectExpression compute the ArrayList(Src) this maps to and the position within
      * this list.
-     * @param sel
+      *
+      * @param sel
      * @return
      */
     def buildResultMapping(sel: SelectExpression): Map[String, (String, Int)] = {
@@ -200,13 +202,19 @@ class GremlinTranslator(expr: Expression,
     }
 
     def typeTestExpression(typeName : String) : String = {
-        val stats = gPersistenceBehavior.typeTestExpression(typeName, counter)
+        val stats = gPersistenceBehavior.typeTestExpression(escape(typeName), counter)
         preStatements ++= stats.init
         stats.last
     }
 
+  def escape(str: String): String = {
+    if (str != null) {
+      return str.replace("\"", "\\\"").replace("$", "\\$");
+    }
+    str
+  }
 
-    private def genQuery(expr: Expression, inSelect: Boolean): String = expr match {
+  private def genQuery(expr: Expression, inSelect: Boolean): String = expr match {
         case ClassExpression(clsName) =>
             typeTestExpression(clsName)
         case TraitExpression(clsName) =>
@@ -239,9 +247,9 @@ class GremlinTranslator(expr: Expression,
             }
         }
         case c@ComparisonExpression(symb, f@FieldExpression(fieldName, fInfo, ch), l) => {
-           val qualifiedPropertyName = s"${gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo)}";
-           val persistentExprValue = translateValueToPersistentForm(fInfo, l);
-           return generateAndPrependExpr(ch, inSelect, s"""has("${qualifiedPropertyName}", ${gPersistenceBehavior.gremlinCompOp(c)}, $persistentExprValue)""");
+           val qualifiedPropertyName = s"${gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo)}"
+           val persistentExprValue = translateValueToPersistentForm(fInfo, l)
+           return generateAndPrependExpr(ch, inSelect, s"""has("${qualifiedPropertyName}", ${gPersistenceBehavior.gremlinCompOp(c)}, $persistentExprValue)""")
         }
         case fil@FilterExpression(child, condExpr) => {
             s"${genQuery(child, inSelect)}.${genQuery(condExpr, inSelect)}"
@@ -329,9 +337,9 @@ class GremlinTranslator(expr: Expression,
     def translateValueToPersistentForm(fInfo: FieldInfo, l: Expression): Any =  {
 
         val dataType = fInfo.attrInfo.dataType;
+      val QUOTE = "\"";
 
          if (dataType == DataTypes.DATE_TYPE) {
-              val QUOTE = "\"";
               try {
                   //Accepts both date, datetime formats
                   val dateStr = l.toString.stripPrefix(QUOTE).stripSuffix(QUOTE)
@@ -360,9 +368,10 @@ class GremlinTranslator(expr: Expression,
          }
          else if(dataType == DataTypes.DOUBLE_TYPE) {
              return s"""${l}d"""
-         }
-         else {
-             return l
+         } else if(dataType == DataTypes.STRING_TYPE) {
+           return string(escape(l.toString.stripPrefix(QUOTE).stripSuffix(QUOTE)));
+         } else {
+           l
          }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
new file mode 100644
index 0000000..8a0a6bc
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
@@ -0,0 +1,51 @@
+/**
+ * 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.repository.graph;
+
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+public class GraphHelperTest {
+    @DataProvider(name = "encodeDecodeTestData")
+    private Object[][] createTestData() {
+        return new Object[][]{
+                {"hivedb$", "hivedb_d"},
+                {"hivedb", "hivedb"},
+                {"{hivedb}", "_ohivedb_c"},
+                {"%hivedb}", "_phivedb_c"},
+                {"\"hivedb\"", "_qhivedb_q"},
+                {"\"$%{}", "_q_d_p_o_c"},
+                {"", ""},
+                {"  ", "  "},
+                {"\n\r", "\n\r"},
+                {null, null}
+        };
+    }
+
+    @Test(dataProvider = "encodeDecodeTestData")
+    public void testEncodeDecode(String str, String expectedEncodedStr) throws Exception {
+        String encodedStr = GraphHelper.encodePropertyKey(str);
+        assertEquals(encodedStr, expectedEncodedStr);
+
+        String decodedStr = GraphHelper.decodePropertyKey(encodedStr);
+        assertEquals(decodedStr, str);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d64112d6/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
index 443dd4a..f2a5b50 100644
--- a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
@@ -24,12 +24,12 @@ import com.google.inject.Inject;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.util.TitanCleanup;
 import org.apache.atlas.AtlasClient;
-import org.apache.atlas.AtlasConstants;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.EntityAuditEvent;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.RequestContext;
 import org.apache.atlas.TestUtils;
+import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
 import org.apache.atlas.listener.EntityChangeListener;
 import org.apache.atlas.repository.audit.EntityAuditRepository;
 import org.apache.atlas.repository.audit.HBaseBasedAuditRepository;
@@ -47,10 +47,12 @@ import org.apache.atlas.typesystem.exception.TypeNotFoundException;
 import org.apache.atlas.typesystem.json.InstanceSerialization;
 import org.apache.atlas.typesystem.json.TypesSerialization;
 import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.typesystem.types.AttributeDefinition;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.EnumValue;
 import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
+import org.apache.atlas.typesystem.types.Multiplicity;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.ValueConversionException;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
@@ -78,7 +80,11 @@ import static org.apache.atlas.TestUtils.PII;
 import static org.apache.atlas.TestUtils.TABLE_TYPE;
 import static org.apache.atlas.TestUtils.createColumnEntity;
 import static org.apache.atlas.TestUtils.createDBEntity;
+import static org.apache.atlas.TestUtils.createInstance;
 import static org.apache.atlas.TestUtils.createTableEntity;
+import static org.apache.atlas.TestUtils.randomString;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertNull;
@@ -96,6 +102,9 @@ public class DefaultMetadataServiceTest {
     @Inject
     private EntityAuditRepository auditRepository;
 
+    @Inject
+    private GraphBackedDiscoveryService discoveryService;
+
     private Referenceable db = createDBEntity();
 
     private Referenceable table;
@@ -121,9 +130,9 @@ public class DefaultMetadataServiceTest {
             metadataService.createType(TypesSerialization.toJson(typesDef));
         }
 
-        String dbGUid = createInstance(db);
+        String dbGUid = TestUtils.createInstance(metadataService, db);
         table = createTableEntity(dbGUid);
-        String tableGuid = createInstance(table);
+        String tableGuid = TestUtils.createInstance(metadataService, table);
         String tableDefinitionJson =
                 metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         table = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
@@ -151,19 +160,6 @@ public class DefaultMetadataServiceTest {
         }
     }
 
-    private String createInstance(Referenceable entity) throws Exception {
-        RequestContext.createContext();
-
-        String entityjson = InstanceSerialization.toJson(entity, true);
-        JSONArray entitiesJson = new JSONArray();
-        entitiesJson.put(entityjson);
-        List<String> guids = metadataService.createEntities(entitiesJson.toString());
-        if (guids != null && guids.size() > 0) {
-            return guids.get(guids.size() - 1);
-        }
-        return null;
-    }
-
     private AtlasClient.EntityResult updateInstance(Referenceable entity) throws Exception {
         RequestContext.createContext();
         ParamChecker.notNull(entity, "Entity");
@@ -180,7 +176,7 @@ public class DefaultMetadataServiceTest {
         String dbName = RandomStringUtils.randomAlphanumeric(10);
         entity.set(NAME, dbName);
         entity.set("description", "us db");
-        createInstance(entity);
+        TestUtils.createInstance(metadataService, entity);
         Assert.fail(TypeNotFoundException.class.getSimpleName() + " was expected but none thrown.");
     }
 
@@ -188,23 +184,79 @@ public class DefaultMetadataServiceTest {
     public void testCreateEntityWithUniqueAttribute() throws Exception {
         //name is the unique attribute
         Referenceable entity = createDBEntity();
-        String id = createInstance(entity);
+        String id = TestUtils.createInstance(metadataService, entity);
         assertAuditEvents(id, EntityAuditEvent.EntityAuditAction.ENTITY_CREATE);
 
         //using the same name should succeed, but not create another entity
-        String newId = createInstance(entity);
+        String newId = TestUtils.createInstance(metadataService, entity);
         assertNull(newId);
 
         //Same entity, but different qualified name should succeed
         entity.set(NAME, TestUtils.randomString());
-        newId = createInstance(entity);
+        newId = TestUtils.createInstance(metadataService, entity);
         Assert.assertNotEquals(newId, id);
     }
 
     @Test
+    //Titan doesn't allow some reserved chars in property keys. Verify that atlas encodes these
+    //See GraphHelper.encodePropertyKey()
+    public void testSpecialCharacters() throws Exception {
+        //Verify that type can be created with reserved characters in typename, attribute name
+        String strAttrName = randomStrWithReservedChars();
+        String arrayAttrName = randomStrWithReservedChars();
+        String mapAttrName = randomStrWithReservedChars();
+        HierarchicalTypeDefinition<ClassType> typeDefinition =
+                createClassTypeDef(randomStrWithReservedChars(), ImmutableSet.<String>of(),
+                        createOptionalAttrDef(strAttrName, DataTypes.STRING_TYPE),
+                        new AttributeDefinition(arrayAttrName, DataTypes.arrayTypeName(DataTypes.STRING_TYPE.getName()),
+                                Multiplicity.OPTIONAL, false, null),
+                        new AttributeDefinition(mapAttrName,
+                                DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(), DataTypes.STRING_TYPE.getName()),
+                                Multiplicity.OPTIONAL, false, null));
+        metadataService.createType(TypesSerialization.toJson(typeDefinition, false));
+
+        //verify that entity can be created with reserved characters in string value, array value and map key and value
+        Referenceable entity = new Referenceable(typeDefinition.typeName);
+        entity.set(strAttrName, randomStrWithReservedChars());
+        entity.set(arrayAttrName, new String[]{randomStrWithReservedChars()});
+        entity.set(mapAttrName, new HashMap<String, String>() {{
+            put(randomStrWithReservedChars(), randomStrWithReservedChars());
+        }});
+        String id = createInstance(metadataService, entity);
+
+        //Verify that get entity definition returns actual values with reserved characters
+        Referenceable instance =
+                InstanceSerialization.fromJsonReferenceable(metadataService.getEntityDefinition(id), true);
+        assertReferenceableEquals(instance, entity);
+
+        //Verify that search with reserved characters works - for string attribute
+        String responseJson = discoveryService.searchByDSL(
+                String.format("`%s` where `%s` = '%s'", typeDefinition.typeName, strAttrName, entity.get(strAttrName)));
+        JSONObject response = new JSONObject(responseJson);
+        assertEquals(response.getJSONArray("rows").length(), 1);
+    }
+
+    //equals excluding the id
+    private void assertReferenceableEquals(Referenceable actual, Referenceable expected) {
+        List<String> traits = actual.getTraits();
+        Map<String, IStruct> traitsMap = new HashMap<>();
+        for (String trait : traits) {
+            traitsMap.put(trait, actual.getTrait(trait));
+        }
+
+        Referenceable newActual = new Referenceable(expected.getId(), actual.getTypeName(), actual.getValuesMap(),
+                traits, traitsMap);
+        assertEquals(InstanceSerialization.toJson(newActual, true), InstanceSerialization.toJson(expected, true));
+    }
+
+    private String randomStrWithReservedChars() {
+        return randomString() + "\"${}%";
+    }
+
+    @Test
     public void testAddDeleteTrait() throws Exception {
         Referenceable entity = createDBEntity();
-        String id = createInstance(entity);
+        String id = TestUtils.createInstance(metadataService, entity);
 
         //add trait
         Struct tag = new Struct(TestUtils.PII);
@@ -218,7 +270,7 @@ public class DefaultMetadataServiceTest {
         String traitDefinition = metadataService.getTraitDefinition(id, PII);
         Struct traitResult = InstanceSerialization.fromJsonStruct(traitDefinition, true);
         Assert.assertNotNull(traitResult);
-        Assert.assertEquals(traitResult.getValuesMap().size(), 0);
+        assertEquals(traitResult.getValuesMap().size(), 0);
 
         //delete trait
         metadataService.deleteTrait(id, PII);
@@ -237,7 +289,7 @@ public class DefaultMetadataServiceTest {
     public void testEntityAudit() throws Exception {
         //create entity
         Referenceable entity = createDBEntity();
-        String id = createInstance(entity);
+        String id = TestUtils.createInstance(metadataService, entity);
         assertAuditEvents(id, EntityAuditEvent.EntityAuditAction.ENTITY_CREATE);
 
         Struct tag = new Struct(TestUtils.PII);
@@ -279,7 +331,7 @@ public class DefaultMetadataServiceTest {
     @Test
     public void testCreateEntityWithUniqueAttributeWithReference() throws Exception {
         Referenceable db = createDBEntity();
-        String dbId = createInstance(db);
+        String dbId = TestUtils.createInstance(metadataService, db);
 
         //Assert that there is just 1 audit events and thats for entity create
         assertAuditEvents(dbId, 1);
@@ -292,14 +344,14 @@ public class DefaultMetadataServiceTest {
         table.set("tableType", "MANAGED");
         table.set("database", new Id(dbId, 0, TestUtils.DATABASE_TYPE));
         table.set("databaseComposite", db);
-        createInstance(table);
+        TestUtils.createInstance(metadataService, table);
 
         //table create should re-use the db instance created earlier
         String tableDefinitionJson =
                 metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         Referenceable actualDb = (Referenceable) tableDefinition.get("databaseComposite");
-        Assert.assertEquals(actualDb.getId().id, dbId);
+        assertEquals(actualDb.getId().id, dbId);
 
         //Assert that as part table create, db is not created and audit event is not added to db
         assertAuditEvents(dbId, 1);
@@ -318,7 +370,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         List<String> actualColumns = (List) tableDefinition.get("columnNames");
-        Assert.assertEquals(actualColumns, colNameList);
+        assertEquals(actualColumns, colNameList);
     }
 
     @Test
@@ -350,7 +402,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
-        Assert.assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
+        assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
         Assert.assertTrue(partsMap.get("part1").equalsContents(((Map<String, Struct>)tableDefinition.get("partitionsMap")).get("part1")));
 
         //update map - remove a key and add another key
@@ -366,7 +418,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
-        Assert.assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
+        assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
         Assert.assertNull(((Map<String, Struct>)tableDefinition.get("partitionsMap")).get("part0"));
         Assert.assertTrue(partsMap.get("part2").equalsContents(((Map<String, Struct>)tableDefinition.get("partitionsMap")).get("part2")));
 
@@ -378,7 +430,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
-        Assert.assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
+        assertEquals(((Map<String, Struct>)tableDefinition.get("partitionsMap")).size(), 2);
         Assert.assertNull(((Map<String, Struct>)tableDefinition.get("partitionsMap")).get("part0"));
         Assert.assertTrue(partsMap.get("part2").equalsContents(((Map<String, Struct>)tableDefinition.get("partitionsMap")).get("part2")));
 
@@ -471,7 +523,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         List<String> actualColumns = (List) tableDefinition.get("columnNames");
-        Assert.assertEquals(actualColumns, colNameList);
+        assertEquals(actualColumns, colNameList);
 
         //update array of primitives
         final List<String> updatedColNameList = ImmutableList.of("col2", "col3");
@@ -484,7 +536,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         actualColumns = (List) tableDefinition.get("columnNames");
-        Assert.assertEquals(actualColumns, updatedColNameList);
+        assertEquals(actualColumns, updatedColNameList);
     }
 
     private AtlasClient.EntityResult updateEntityPartial(String guid, Referenceable entity) throws AtlasException {
@@ -625,7 +677,7 @@ public class DefaultMetadataServiceTest {
         table.set("serde1", serdeInstance);
 
         String newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         String tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -659,7 +711,7 @@ public class DefaultMetadataServiceTest {
         sdReferenceable.set(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, TestUtils.randomString());
             sdReferenceable.set("compressed", "false");
             sdReferenceable.set("location", "hdfs://tmp/hive-user");
-        String sdGuid = createInstance(sdReferenceable);
+        String sdGuid = TestUtils.createInstance(metadataService, sdReferenceable);
 
         Referenceable sdRef2 = new Referenceable(sdGuid, TestUtils.STORAGE_DESC_TYPE, null);
 
@@ -669,7 +721,7 @@ public class DefaultMetadataServiceTest {
         partRef.set("table", table);
         partRef.set("sd", sdRef2);
 
-        String partGuid = createInstance(partRef);
+        String partGuid = TestUtils.createInstance(metadataService, partRef);
         Assert.assertNotNull(partGuid);
     }
 
@@ -680,7 +732,7 @@ public class DefaultMetadataServiceTest {
         databaseInstance.set(NAME, TestUtils.randomString());
         databaseInstance.set("description", "new database");
 
-        String dbId = createInstance(databaseInstance);
+        String dbId = TestUtils.createInstance(metadataService, databaseInstance);
 
         /*Update reference property with Id */
         metadataService.updateEntityAttributeByGuid(tableId._getId(), "database", dbId);
@@ -689,7 +741,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(tableId._getId());
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
-        Assert.assertEquals(dbId, (((Id) tableDefinition.get("database"))._getId()));
+        assertEquals(dbId, (((Id) tableDefinition.get("database"))._getId()));
 
         /* Update with referenceable - TODO - Fails . Need to fix this */
         /*final String dbName = TestUtils.randomString();
@@ -729,7 +781,7 @@ public class DefaultMetadataServiceTest {
         table.set("partitions", partitions);
 
         String newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         String tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -745,7 +797,7 @@ public class DefaultMetadataServiceTest {
         partitions.add(partition3);
         table.set("partitions", partitions);
         newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -759,7 +811,7 @@ public class DefaultMetadataServiceTest {
         partitions.remove(1);
         table.set("partitions", partitions);
         newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -772,7 +824,7 @@ public class DefaultMetadataServiceTest {
         //Update struct value within array of struct
         partitions.get(0).set(NAME, "part4");
         newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -788,7 +840,7 @@ public class DefaultMetadataServiceTest {
         partitions.add(partition4);
         table.set("partitions", partitions);
         newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -802,7 +854,7 @@ public class DefaultMetadataServiceTest {
         // Remove all elements. Should set array attribute to null
         partitions.clear();
         newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -826,7 +878,7 @@ public class DefaultMetadataServiceTest {
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
-        Assert.assertEquals(tableDefinition.get("description"), "random table");
+        assertEquals(tableDefinition.get("description"), "random table");
         table.setNull("description");
 
         updateInstance(table);
@@ -845,7 +897,7 @@ public class DefaultMetadataServiceTest {
             table.setNull("created");
 
         String newtableId = updateInstance(table).getUpdateEntities().get(0);
-        Assert.assertEquals(newtableId, tableId._getId());
+        assertEquals(newtableId, tableId._getId());
 
         tableDefinitionJson =
             metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, NAME, (String) table.get(NAME));
@@ -860,20 +912,20 @@ public class DefaultMetadataServiceTest {
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         EnumValue tableType = (EnumValue) tableDefinition.get("tableType");
 
-        Assert.assertEquals(tableType, new EnumValue("MANAGED", 1));
+        assertEquals(tableType, new EnumValue("MANAGED", 1));
     }
 
     @Test
     public void testGetEntityByUniqueAttribute() throws Exception {
         Referenceable entity = createDBEntity();
-        createInstance(entity);
+        TestUtils.createInstance(metadataService, entity);
 
         //get entity by valid qualified name
         String entityJson = metadataService.getEntityDefinition(TestUtils.DATABASE_TYPE, NAME,
                 (String) entity.get(NAME));
         Assert.assertNotNull(entityJson);
         Referenceable referenceable = InstanceSerialization.fromJsonReferenceable(entityJson, true);
-        Assert.assertEquals(referenceable.get(NAME), entity.get(NAME));
+        assertEquals(referenceable.get(NAME), entity.get(NAME));
 
         //get entity by invalid qualified name
         try {
@@ -897,13 +949,13 @@ public class DefaultMetadataServiceTest {
     public void testDeleteEntities() throws Exception {
         // Create a table entity, with 3 composite column entities
         Referenceable dbEntity = createDBEntity();
-        String dbGuid = createInstance(dbEntity);
+        String dbGuid = TestUtils.createInstance(metadataService, dbEntity);
         Referenceable table1Entity = createTableEntity(dbGuid);
         Referenceable col1 = createColumnEntity();
         Referenceable col2 = createColumnEntity();
         Referenceable col3 = createColumnEntity();
         table1Entity.set(COLUMNS_ATTR_NAME, ImmutableList.of(col1, col2, col3));
-        createInstance(table1Entity);
+        TestUtils.createInstance(metadataService, table1Entity);
 
         // Retrieve the table entities from the repository,
         // to get their guids and the composite column guids.
@@ -946,7 +998,7 @@ public class DefaultMetadataServiceTest {
         // Verify that the listener was notified about the deleted entities.
         List<String> deletedEntitiesFromListener = listener.getDeletedEntities();
         Assert.assertNotNull(deletedEntitiesFromListener);
-        Assert.assertEquals(deletedEntitiesFromListener.size(), entityResult.getDeletedEntities().size());
+        assertEquals(deletedEntitiesFromListener.size(), entityResult.getDeletedEntities().size());
         Assert.assertTrue(deletedEntitiesFromListener.containsAll(entityResult.getDeletedEntities()));
     }
 
@@ -964,13 +1016,13 @@ public class DefaultMetadataServiceTest {
     public void testDeleteEntityByUniqueAttribute() throws Exception {
         // Create a table entity, with 3 composite column entities
         Referenceable dbEntity = createDBEntity();
-        String dbGuid = createInstance(dbEntity);
+        String dbGuid = TestUtils.createInstance(metadataService, dbEntity);
         Referenceable table1Entity = createTableEntity(dbGuid);
         Referenceable col1 = createColumnEntity();
         Referenceable col2 = createColumnEntity();
         Referenceable col3 = createColumnEntity();
         table1Entity.set(COLUMNS_ATTR_NAME, ImmutableList.of(col1, col2, col3));
-        createInstance(table1Entity);
+        TestUtils.createInstance(metadataService, table1Entity);
 
         // to get their guids and the composite column guids.
         String entityJson = metadataService.getEntityDefinition(TestUtils.TABLE_TYPE,
@@ -1005,12 +1057,12 @@ public class DefaultMetadataServiceTest {
         // Verify that the listener was notified about the deleted entities.
         List<String> deletedEntitiesFromListener = listener.getDeletedEntities();
         Assert.assertNotNull(deletedEntitiesFromListener);
-        Assert.assertEquals(deletedEntitiesFromListener.size(), deletedGuids.size());
+        assertEquals(deletedEntitiesFromListener.size(), deletedGuids.size());
         Assert.assertTrue(deletedEntitiesFromListener.containsAll(deletedGuids));
     }
 
     @Test
-    public void testTypeUpdateWithReservedAttributes() throws AtlasException, JSONException {
+    public void testTypeUpdateFailureShouldRollBack() throws AtlasException, JSONException {
         String typeName = "test_type_"+ RandomStringUtils.randomAlphanumeric(10);
         HierarchicalTypeDefinition<ClassType> typeDef = TypesUtil.createClassTypeDef(
                 typeName, ImmutableSet.<String>of(),
@@ -1022,17 +1074,20 @@ public class DefaultMetadataServiceTest {
         HierarchicalTypeDefinition<ClassType> updatedTypeDef = TypesUtil.createClassTypeDef(
             typeName, ImmutableSet.<String>of(),
             TypesUtil.createUniqueRequiredAttrDef("test_type_attribute", DataTypes.STRING_TYPE),
-            TypesUtil.createOptionalAttrDef("test_type_invalid_attribute$", DataTypes.STRING_TYPE));
+            TypesUtil.createRequiredAttrDef("test_type_invalid_attribute$", DataTypes.STRING_TYPE));
         TypesDef updatedTypesDef = new TypesDef(updatedTypeDef, false);
 
         try {
             metadataService.updateType(TypesSerialization.toJson(updatedTypesDef));
-            Assert.fail("Should not be able to update type with reserved character");
-        } catch (AtlasException ae) {
-            // pass.. expected
+            fail("Expected AtlasException");
+        } catch (AtlasException e) {
+            //expected
         }
+
+        //type definition should reflect old type
         String typeDefinition = metadataService.getTypeDefinition(typeName);
-        Assert.assertNotNull(typeDefinition);
+        typesDef = TypesSerialization.fromJson(typeDefinition);
+        assertEquals(typesDef.classTypes().head().attributeDefinitions.length, 1);
     }
 
     @Test