You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/02/14 20:46:17 UTC

incubator-atlas git commit: ATLAS-1557: IDBased resolver should attempt unique-attribute match as well

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 2a93a6aa4 -> c7540b38f


ATLAS-1557: IDBased resolver should attempt unique-attribute match as well


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

Branch: refs/heads/master
Commit: c7540b38f50718e0f8d488b25defa47c9c34f3c2
Parents: 2a93a6a
Author: Madhan Neethiraj <ma...@apache.org>
Authored: Tue Feb 14 03:19:31 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Tue Feb 14 11:29:02 2017 -0800

----------------------------------------------------------------------
 .../atlas/type/AtlasClassificationType.java     |   9 +-
 .../org/apache/atlas/type/AtlasEntityType.java  |  11 +-
 .../org/apache/atlas/type/AtlasStructType.java  |  39 +++++--
 .../graph/v1/AtlasEntityGraphDiscoveryV1.java   |   2 +-
 .../store/graph/v1/AtlasEntityStoreV1.java      |   9 +-
 .../store/graph/v1/AtlasGraphUtilsV1.java       | 114 ++++++++++++-------
 .../store/graph/v1/EntityGraphMapper.java       |  22 +++-
 .../store/graph/v1/EntityMutationContext.java   |   7 +-
 .../store/graph/v1/IDBasedEntityResolver.java   |  59 +++++-----
 .../graph/v1/UniqAttrBasedEntityResolver.java   |  83 ++------------
 .../store/graph/v1/AtlasEntityStoreV1Test.java  |   2 +
 11 files changed, 190 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java b/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
index f839328..318f2ed 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
@@ -87,10 +87,11 @@ public class AtlasClassificationType extends AtlasStructType {
             }
         }
 
-        this.superTypes    = Collections.unmodifiableList(s);
-        this.allSuperTypes = Collections.unmodifiableSet(allS);
-        this.allAttributes = Collections.unmodifiableMap(allA);
-        this.allSubTypes   = new HashSet<>(); // this will be populated in resolveReferencesPhase2()
+        this.superTypes     = Collections.unmodifiableList(s);
+        this.allSuperTypes  = Collections.unmodifiableSet(allS);
+        this.allAttributes  = Collections.unmodifiableMap(allA);
+        this.uniqAttributes = getUniqueAttributes(this.allAttributes);
+        this.allSubTypes    = new HashSet<>(); // this will be populated in resolveReferencesPhase2()
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java b/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
index a9a4a10..d997725 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
@@ -73,7 +73,7 @@ public class AtlasEntityType extends AtlasStructType {
 
         List<AtlasEntityType>       s    = new ArrayList<>();
         Set<String>                 allS = new HashSet<>();
-        Map<String, AtlasAttribute> allA    = new HashMap<>();
+        Map<String, AtlasAttribute> allA = new HashMap<>();
 
         getTypeHierarchyInfo(typeRegistry, allS, allA);
 
@@ -87,10 +87,11 @@ public class AtlasEntityType extends AtlasStructType {
             }
         }
 
-        this.superTypes    = Collections.unmodifiableList(s);
-        this.allSuperTypes = Collections.unmodifiableSet(allS);
-        this.allAttributes = Collections.unmodifiableMap(allA);
-        this.allSubTypes   = new HashSet<>();   // this will be populated in resolveReferencesPhase2()
+        this.superTypes     = Collections.unmodifiableList(s);
+        this.allSuperTypes  = Collections.unmodifiableSet(allS);
+        this.allAttributes  = Collections.unmodifiableMap(allA);
+        this.uniqAttributes = getUniqueAttributes(this.allAttributes);
+        this.allSubTypes    = new HashSet<>();   // this will be populated in resolveReferencesPhase2()
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java b/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
index 39442f1..3bc5f83 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
@@ -29,6 +29,7 @@ import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,7 +47,8 @@ public class AtlasStructType extends AtlasType {
 
     private final AtlasStructDef structDef;
 
-    protected Map<String, AtlasAttribute> allAttributes = Collections.emptyMap();
+    protected Map<String, AtlasAttribute> allAttributes  = Collections.emptyMap();
+    protected Map<String, AtlasAttribute> uniqAttributes = Collections.emptyMap();
 
     public AtlasStructType(AtlasStructDef structDef) {
         super(structDef);
@@ -103,7 +105,8 @@ public class AtlasStructType extends AtlasType {
 
         resolveConstraints(typeRegistry);
 
-        this.allAttributes = Collections.unmodifiableMap(a);
+        this.allAttributes  = Collections.unmodifiableMap(a);
+        this.uniqAttributes = getUniqueAttributes(this.allAttributes);
     }
 
     private void resolveConstraints(AtlasTypeRegistry typeRegistry) throws AtlasBaseException {
@@ -176,6 +179,10 @@ public class AtlasStructType extends AtlasType {
         return allAttributes;
     }
 
+    public Map<String, AtlasAttribute> getUniqAttributes() {
+        return uniqAttributes;
+    }
+
     public AtlasAttribute getAttribute(String attributeName) {
         return allAttributes.get(attributeName);
     }
@@ -403,6 +410,20 @@ public class AtlasStructType extends AtlasType {
         return type instanceof AtlasEntityType ? (AtlasEntityType)type : null;
     }
 
+    protected Map<String, AtlasAttribute> getUniqueAttributes(Map<String, AtlasAttribute> attributes) {
+        Map<String, AtlasAttribute> ret = new HashMap<>();
+
+        if (MapUtils.isNotEmpty(attributes)) {
+            for (AtlasAttribute attribute : attributes.values()) {
+                if (attribute.getAttributeDef().getIsUnique()) {
+                    ret.put(attribute.getName(), attribute);
+                }
+            }
+        }
+
+        return Collections.unmodifiableMap(ret);
+    }
+
     public static class AtlasAttribute {
         private final AtlasStructType   definedInType;
         private final AtlasType         attributeType;
@@ -466,12 +487,7 @@ public class AtlasStructType extends AtlasType {
 
         public String getInverseRefAttribute() { return inverseRefAttribute; }
 
-        private static String getQualifiedAttributeName(AtlasStructDef structDef, String attrName) {
-            final String typeName = structDef.getName();
-            return attrName.contains(".") ? attrName : String.format("%s.%s", typeName, attrName);
-        }
-
-        private static String encodePropertyKey(String key) {
+        public static String encodePropertyKey(String key) {
             if (StringUtils.isBlank(key)) {
                 return key;
             }
@@ -483,7 +499,7 @@ public class AtlasStructType extends AtlasType {
             return key;
         }
 
-        private static String decodePropertyKey(String key) {
+        public static String decodePropertyKey(String key) {
             if (StringUtils.isBlank(key)) {
                 return key;
             }
@@ -495,6 +511,11 @@ public class AtlasStructType extends AtlasType {
             return key;
         }
 
+        private static String getQualifiedAttributeName(AtlasStructDef structDef, String attrName) {
+            final String typeName = structDef.getName();
+            return attrName.contains(".") ? attrName : String.format("%s.%s", typeName, attrName);
+        }
+
         private static String[][] RESERVED_CHAR_ENCODE_MAP = new String[][] {
                 new String[] { "{",  "_o" },
                 new String[] { "}",  "_c" },

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityGraphDiscoveryV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityGraphDiscoveryV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityGraphDiscoveryV1.java
index d95b3f9..256c5f6 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityGraphDiscoveryV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityGraphDiscoveryV1.java
@@ -114,7 +114,7 @@ public class AtlasEntityGraphDiscoveryV1 implements EntityGraphDiscovery {
     }
 
     protected void resolveReferences() throws AtlasBaseException {
-        EntityResolver[] entityResolvers = new EntityResolver[] { new IDBasedEntityResolver(),
+        EntityResolver[] entityResolvers = new EntityResolver[] { new IDBasedEntityResolver(typeRegistry),
                                                                   new UniqAttrBasedEntityResolver(typeRegistry)
                                                                 };
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
index 1f4ad57..c6a7206 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
@@ -35,6 +35,7 @@ import org.apache.atlas.repository.store.graph.EntityGraphDiscovery;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscoveryContext;
 import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -208,7 +209,13 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
                 if (entity != null) {
                     AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entity.getTypeName());
 
-                    context.addUpdated(entity, entityType, vertex);
+                    String guidVertex = AtlasGraphUtilsV1.getIdFromVertex(vertex);
+
+                    if (!StringUtils.equals(guidVertex, guid)) { // if entity was found by unique attribute
+                        entity.setGuid(guidVertex);
+                    }
+
+                    context.addUpdated(guid, entity, entityType, vertex);
 
                     RequestContextV1.get().recordEntityUpdate(entity.getAtlasObjectId());
                 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
index b3fa370..efc50d3 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
@@ -17,8 +17,6 @@
  */
 package org.apache.atlas.repository.store.graph.v1;
 
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
 
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.exception.AtlasBaseException;
@@ -34,8 +32,9 @@ import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
 import org.apache.atlas.type.AtlasType;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.collections.MapUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,15 +50,6 @@ public class AtlasGraphUtilsV1 {
     public static final String SUPERTYPE_EDGE_LABEL = PROPERTY_PREFIX + ".supertype";
     public static final String VERTEX_TYPE          = "typeSystem";
 
-    public static final 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 getTypeDefPropertyKey(AtlasBaseTypeDef typeDef) {
         return getTypeDefPropertyKey(typeDef.getName());
@@ -120,25 +110,13 @@ public class AtlasGraphUtilsV1 {
     }
 
     public static String encodePropertyKey(String key) {
-        String ret = key;
-
-        if (StringUtils.isNotBlank(key)) {
-            for (String str : RESERVED_CHARS_ENCODE_MAP.keySet()) {
-                ret = ret.replace(str, RESERVED_CHARS_ENCODE_MAP.get(str));
-            }
-        }
+        String ret = AtlasStructType.AtlasAttribute.encodePropertyKey(key);
 
         return ret;
     }
 
     public static String decodePropertyKey(String key) {
-        String ret = key;
-
-        if (StringUtils.isNotBlank(key)) {
-            for (String encodedStr : RESERVED_CHARS_ENCODE_MAP.values()) {
-                ret = ret.replace(encodedStr, RESERVED_CHARS_ENCODE_MAP.inverse().get(encodedStr));
-            }
-        }
+        String ret = AtlasStructType.AtlasAttribute.decodePropertyKey(key);
 
         return ret;
     }
@@ -201,27 +179,81 @@ public class AtlasGraphUtilsV1 {
         return returnType.cast(property);
     }
 
-    public static AtlasVertex getVertexByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> uniqAttributes) throws AtlasBaseException {
-        AtlasGraphQuery query = AtlasGraphProvider.getGraphInstance().query();
-
-        for (Map.Entry<String, Object> e : uniqAttributes.entrySet()) {
-            String attrName = e.getKey();
-            Object attrValue = e.getValue();
+    public static AtlasVertex getVertexByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> attrValues) throws AtlasBaseException {
+        AtlasVertex vertex = findByUniqueAttributes(entityType, attrValues);
 
-            query = query.has(entityType.getQualifiedAttributeName(attrName), attrValue);
+        if (vertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND, entityType.getTypeName(),
+                                         attrValues.toString());
         }
 
-        Iterator<AtlasVertex> result = query.has(Constants.ENTITY_TYPE_PROPERTY_KEY, entityType.getTypeName())
-                                            .has(Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name())
-                                            .vertices().iterator();
-        AtlasVertex entityVertex = result.hasNext() ? result.next() : null;
+        return vertex;
+    }
 
-        if (entityVertex == null) {
-            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND, entityType.getTypeName(),
-                                         uniqAttributes.toString());
+    public static AtlasVertex findByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> attrValues) {
+        AtlasVertex vertex = null;
+
+        final Map<String, AtlasAttribute> uniqueAttributes = entityType.getUniqAttributes();
+
+        if (MapUtils.isNotEmpty(uniqueAttributes) && MapUtils.isNotEmpty(attrValues)) {
+            for (AtlasAttribute attribute : uniqueAttributes.values()) {
+                Object attrValue = attrValues.get(attribute.getName());
+
+                if (attrValue == null) {
+                    continue;
+                }
+
+                vertex = AtlasGraphUtilsV1.findByTypeAndPropertyName(entityType.getTypeName(), attribute.getVertexPropertyName(), attrValue);
+
+                if (vertex == null) {
+                    vertex = AtlasGraphUtilsV1.findBySuperTypeAndPropertyName(entityType.getTypeName(), attribute.getVertexPropertyName(), attrValue);
+                }
+
+                if (vertex != null) {
+                    break;
+                }
+            }
         }
 
-        return entityVertex;
+        return vertex;
+    }
+
+    public static AtlasVertex findByGuid(String guid) {
+        AtlasGraphQuery query = AtlasGraphProvider.getGraphInstance().query()
+                                                  .has(Constants.GUID_PROPERTY_KEY, guid)
+                                                  .has(Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name());
+
+        Iterator<AtlasVertex> results = query.vertices().iterator();
+
+        AtlasVertex vertex = results.hasNext() ? results.next() : null;
+
+        return vertex;
+    }
+
+    public static AtlasVertex findByTypeAndPropertyName(String typeName, String propertyName, Object attrVal) {
+        AtlasGraphQuery query = AtlasGraphProvider.getGraphInstance().query()
+                                                    .has(Constants.ENTITY_TYPE_PROPERTY_KEY, typeName)
+                                                    .has(Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name())
+                                                    .has(propertyName, attrVal);
+
+        Iterator<AtlasVertex> results = query.vertices().iterator();
+
+        AtlasVertex vertex = results.hasNext() ? results.next() : null;
+
+        return vertex;
+    }
+
+    public static AtlasVertex findBySuperTypeAndPropertyName(String typeName, String propertyName, Object attrVal) {
+        AtlasGraphQuery query = AtlasGraphProvider.getGraphInstance().query()
+                                                    .has(Constants.SUPER_TYPES_PROPERTY_KEY, typeName)
+                                                    .has(Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name())
+                                                    .has(propertyName, attrVal);
+
+        Iterator<AtlasVertex> results = query.vertices().iterator();
+
+        AtlasVertex vertex = results.hasNext() ? results.next() : null;
+
+        return vertex;
     }
 
     private static String toString(AtlasElement element) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphMapper.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphMapper.java
index d707bbd..072d10d 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphMapper.java
@@ -286,18 +286,30 @@ public class EntityGraphMapper {
         AtlasEdge ret = null;
 
         if (ctx.getCurrentEdge() != null) {
-            updateVertex((AtlasStruct) ctx.getValue(), ctx.getCurrentEdge().getInVertex(), context);
+            AtlasStruct structVal = null;
+            if (ctx.getValue() instanceof AtlasStruct) {
+                structVal = (AtlasStruct)ctx.getValue();
+            } else if (ctx.getValue() instanceof Map) {
+                structVal = new AtlasStruct(ctx.getAttrType().getTypeName(), (Map) AtlasTypeUtil.toStructAttributes((Map)ctx.getValue()));
+            }
+
+            if (structVal != null) {
+                updateVertex(structVal, ctx.getCurrentEdge().getInVertex(), context);
+            }
 
             ret = ctx.getCurrentEdge();
         } else if (ctx.getValue() != null) {
             String edgeLabel = AtlasGraphUtilsV1.getEdgeLabel(ctx.getVertexProperty());
 
+            AtlasStruct structVal = null;
             if (ctx.getValue() instanceof AtlasStruct) {
-                ret = createVertex((AtlasStruct) ctx.getValue(), ctx.getReferringVertex(), edgeLabel, context);
+                structVal = (AtlasStruct) ctx.getValue();
             } else if (ctx.getValue() instanceof Map) {
-                AtlasStruct stuct = new AtlasStruct(ctx.getAttrType().getTypeName(), (Map) AtlasTypeUtil.toStructAttributes((Map)ctx.getValue()));
+                structVal = new AtlasStruct(ctx.getAttrType().getTypeName(), (Map) AtlasTypeUtil.toStructAttributes((Map)ctx.getValue()));
+            }
 
-                ret = createVertex(stuct, ctx.getReferringVertex(), edgeLabel, context);
+            if (structVal != null) {
+                ret = createVertex(structVal, ctx.getReferringVertex(), edgeLabel, context);
             }
         }
 
@@ -422,7 +434,7 @@ public class EntityGraphMapper {
         if (CollectionUtils.isNotEmpty(newElements)) {
             for (int index = 0; index < newElements.size(); index++) {
                 AtlasEdge               existingEdge = getEdgeAt(currentElements, index, elementType);
-                AttributeMutationContext arrCtx      =  new AttributeMutationContext(ctx.getOp(), ctx.getReferringVertex(), ctx.getAttribute(), newElements.get(index),
+                AttributeMutationContext arrCtx      = new AttributeMutationContext(ctx.getOp(), ctx.getReferringVertex(), ctx.getAttribute(), newElements.get(index),
                                                                                      ctx.getVertexProperty(), elementType, existingEdge);
 
                 Object newEntry = mapCollectionElementsToVertex(arrCtx, context);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
index f24cab3..23e825e 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
@@ -51,10 +51,15 @@ public class EntityMutationContext {
         }
     }
 
-    public void addUpdated(AtlasEntity entity, AtlasEntityType type, AtlasVertex atlasVertex) {
+    public void addUpdated(String internalGuid, AtlasEntity entity, AtlasEntityType type, AtlasVertex atlasVertex) {
         entitiesUpdated.add(entity);
         entityVsType.put(entity.getGuid(), type);
         entityVsVertex.put(entity.getGuid(), atlasVertex);
+
+        if (!StringUtils.equals(internalGuid, entity.getGuid())) {
+            guidAssignments.put(internalGuid, entity.getGuid());
+            entityVsVertex.put(internalGuid, atlasVertex);
+        }
     }
 
     public EntityGraphDiscoveryContext getDiscoveryContext() {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/IDBasedEntityResolver.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/IDBasedEntityResolver.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/IDBasedEntityResolver.java
index 41ea75b..d5946f2 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/IDBasedEntityResolver.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/IDBasedEntityResolver.java
@@ -19,14 +19,14 @@ package org.apache.atlas.repository.store.graph.v1;
 
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
 import org.apache.atlas.model.instance.AtlasEntity;
-import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscoveryContext;
 import org.apache.atlas.repository.store.graph.EntityResolver;
-import org.apache.atlas.typesystem.exception.EntityNotFoundException;
-import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasTypeRegistry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,9 +34,12 @@ import org.slf4j.LoggerFactory;
 public class IDBasedEntityResolver implements EntityResolver {
     private static final Logger LOG = LoggerFactory.getLogger(IDBasedEntityResolver.class);
 
+    private final GraphHelper       graphHelper = GraphHelper.getInstance();
+    private final AtlasTypeRegistry typeRegistry;
 
-    private final GraphHelper graphHelper = GraphHelper.getInstance();
-
+    public IDBasedEntityResolver(AtlasTypeRegistry typeRegistry) {
+        this.typeRegistry = typeRegistry;
+    }
 
     public EntityGraphDiscoveryContext resolveEntityReferences(EntityGraphDiscoveryContext context) throws AtlasBaseException {
         if (context == null) {
@@ -46,34 +49,36 @@ public class IDBasedEntityResolver implements EntityResolver {
         EntityStream entityStream = context.getEntityStream();
 
         for (String guid : context.getReferencedGuids()) {
-            if (AtlasEntity.isAssigned(guid)) { // validate in graph repo that given guid exists
-                AtlasVertex vertex = resolveGuid(guid);
+            boolean     isAssignedGuid = AtlasEntity.isAssigned(guid);
+            AtlasVertex vertex         = isAssignedGuid ? AtlasGraphUtilsV1.findByGuid(guid) : null;
+
+            if (vertex == null) { // if not found in the store, look if the entity is present in the stream
+                AtlasEntity entity = entityStream.getByGuid(guid);
+
+                if (entity != null) { // look for the entity in the store using unique-attributes
+                    AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entity.getTypeName());
+
+                    if (entityType == null) {
+                        throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), entity.getTypeName());
+                    }
+
+                    vertex = AtlasGraphUtilsV1.findByUniqueAttributes(entityType, entity.getAttributes());
+                } else if (!isAssignedGuid) { // for local-guids, entity must be in the stream
+                    throw new AtlasBaseException(AtlasErrorCode.REFERENCED_ENTITY_NOT_FOUND, guid);
+                }
+            }
 
+            if (vertex != null) {
                 context.addResolvedGuid(guid, vertex);
-            } else  if (entityStream.getByGuid(guid) != null) { //check if entity stream have this reference id
-                context.addLocalGuidReference(guid);
             } else {
-                throw new AtlasBaseException(AtlasErrorCode.REFERENCED_ENTITY_NOT_FOUND, guid);
+                if (isAssignedGuid) {
+                    throw new AtlasBaseException(AtlasErrorCode.REFERENCED_ENTITY_NOT_FOUND, guid);
+                } else {
+                    context.addLocalGuidReference(guid);
+                }
             }
         }
 
         return context;
     }
-
-    private AtlasVertex resolveGuid(String guid) throws AtlasBaseException {
-        //validate in graph repo that given guid, typename exists
-        AtlasVertex vertex = null;
-        try {
-            vertex = graphHelper.findVertex(Constants.GUID_PROPERTY_KEY, guid,
-                                            Constants.STATE_PROPERTY_KEY, Id.EntityState.ACTIVE.name());
-        } catch (EntityNotFoundException e) {
-            //Ignore
-        }
-
-        if (vertex != null) {
-            return vertex;
-        } else {
-            throw new AtlasBaseException(AtlasErrorCode.REFERENCED_ENTITY_NOT_FOUND, guid);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/UniqAttrBasedEntityResolver.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/UniqAttrBasedEntityResolver.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/UniqAttrBasedEntityResolver.java
index 17b102a..c80639a 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/UniqAttrBasedEntityResolver.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/UniqAttrBasedEntityResolver.java
@@ -43,8 +43,9 @@ import java.util.Map;
 public class UniqAttrBasedEntityResolver implements EntityResolver {
     private static final Logger LOG = LoggerFactory.getLogger(UniqAttrBasedEntityResolver.class);
 
-    private final GraphHelper           graphHelper = GraphHelper.getInstance();
-    private final AtlasTypeRegistry     typeRegistry;
+    private final static GraphHelper graphHelper = GraphHelper.getInstance();
+
+    private final AtlasTypeRegistry typeRegistry;
 
     public UniqAttrBasedEntityResolver(AtlasTypeRegistry typeRegistry) {
         this.typeRegistry = typeRegistry;
@@ -61,7 +62,13 @@ public class UniqAttrBasedEntityResolver implements EntityResolver {
 
         for (AtlasObjectId objId : context.getReferencedByUniqAttribs()) {
             //query in graph repo that given unique attribute - check for deleted also?
-            AtlasVertex vertex = resolveByUniqueAttribute(objId);
+            AtlasEntityType entityType = typeRegistry.getEntityTypeByName(objId.getTypeName());
+
+            if (entityType == null) {
+                throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), objId.getTypeName());
+            }
+
+            AtlasVertex vertex = AtlasGraphUtilsV1.findByUniqueAttributes(entityType, objId.getUniqueAttributes());
 
             if (vertex != null) {
                 context.addResolvedIdByUniqAttribs(objId, vertex);
@@ -73,75 +80,5 @@ public class UniqAttrBasedEntityResolver implements EntityResolver {
 
         return context;
     }
-
-    private AtlasVertex resolveByUniqueAttribute(AtlasObjectId entityId) throws AtlasBaseException {
-        AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entityId.getTypeName());
-
-        if (entityType == null) {
-            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), entityId.getTypeName());
-        }
-
-        final Map<String, Object> uniqueAttributes = entityId.getUniqueAttributes();
-
-        if (MapUtils.isNotEmpty(uniqueAttributes)) {
-            for (Map.Entry<String, Object> e : uniqueAttributes.entrySet()) {
-                String         attrName  = e.getKey();
-                Object         attrValue = e.getValue();
-                AtlasAttribute attr      = entityType.getAttribute(attrName);
-
-                if (attr == null || !attr.getAttributeDef().getIsUnique() || attrValue == null) {
-                    continue;
-                }
-
-                AtlasVertex vertex = findByTypeAndQualifiedName(entityId.getTypeName(), attr.getQualifiedName(), attrValue);
-
-                if (vertex == null) {
-                    vertex = findBySuperTypeAndQualifiedName(entityId.getTypeName(), attr.getQualifiedName(), attrValue);
-                }
-
-                if (vertex != null) {
-                    return vertex;
-                }
-            }
-        }
-
-        throw new AtlasBaseException(AtlasErrorCode.REFERENCED_ENTITY_NOT_FOUND, entityId.toString());
-    }
-
-    private AtlasVertex findByTypeAndQualifiedName(String typeName, String qualifiedAttrName, Object attrVal) {
-        AtlasVertex vertex = null;
-
-        try {
-            vertex = graphHelper.findVertex(qualifiedAttrName, attrVal,
-                                            Constants.ENTITY_TYPE_PROPERTY_KEY, typeName,
-                                            Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name());
-
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Found vertex by unique attribute and type ({}={}), {} ", qualifiedAttrName, attrVal, typeName);
-            }
-        } catch (EntityNotFoundException e) {
-            //Ignore if not found
-        }
-
-        return vertex;
-    }
-
-    private AtlasVertex findBySuperTypeAndQualifiedName(String typeName, String qualifiedAttrName, Object attrVal) {
-        AtlasVertex vertex = null;
-
-        try {
-            vertex = graphHelper.findVertex(qualifiedAttrName, attrVal,
-                                            Constants.SUPER_TYPES_PROPERTY_KEY, typeName,
-                                            Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name());
-
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Found vertex by unique attribute and supertype ({}={}), {} ", qualifiedAttrName, attrVal, typeName);
-            }
-        } catch (EntityNotFoundException e) {
-            //Ignore if not found
-        }
-
-        return vertex;
-    }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/c7540b38/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java
index e493345..bb7de4a 100644
--- a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1Test.java
@@ -209,6 +209,7 @@ public class AtlasEntityStoreV1Test {
 
         columns.add(col3.getAtlasObjectId());
         columns.add(col4.getAtlasObjectId());
+        tableEntity.setAttribute(TestUtilsV2.COLUMNS_ATTR_NAME, columns);
 
         entitiesInfo.addReferredEntity(col3);
         entitiesInfo.addReferredEntity(col4);
@@ -223,6 +224,7 @@ public class AtlasEntityStoreV1Test {
         columns.clear();
         columns.add(col4.getAtlasObjectId());
         columns.add(col3.getAtlasObjectId());
+        tableEntity.setAttribute(TestUtilsV2.COLUMNS_ATTR_NAME, columns);
 
         init();
         response = entityStore.createOrUpdate(new AtlasEntityStream(entitiesInfo));