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/04 00:33:06 UTC

[2/3] incubator-atlas git commit: ATLAS-1522: V2 entity API changes to accept only AtlasObjectId for child references

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/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 76308a0..f21896d 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
@@ -23,7 +23,6 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.TypeCategory;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasObjectId;
-import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
@@ -33,21 +32,20 @@ import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasStructType;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
+import org.apache.commons.collections.MapUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.inject.Inject;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 public class UniqAttrBasedEntityResolver implements EntityResolver {
-
     private static final Logger LOG = LoggerFactory.getLogger(UniqAttrBasedEntityResolver.class);
 
-    private final AtlasTypeRegistry typeRegistry;
-
-    private final GraphHelper graphHelper = GraphHelper.getInstance();
-
+    private final GraphHelper           graphHelper = GraphHelper.getInstance();
+    private final AtlasTypeRegistry     typeRegistry;
     private EntityGraphDiscoveryContext context;
 
     @Inject
@@ -56,41 +54,41 @@ public class UniqAttrBasedEntityResolver implements EntityResolver {
     }
 
     @Override
-    public void init(EntityGraphDiscoveryContext entities) throws AtlasBaseException {
-        this.context = entities;
+    public void init(EntityGraphDiscoveryContext context) throws AtlasBaseException {
+        this.context = context;
     }
 
     @Override
     public EntityGraphDiscoveryContext resolveEntityReferences() throws AtlasBaseException {
-
-        if ( context == null) {
+        if (context == null) {
             throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, "Unique attribute based entity resolver not initialized");
         }
 
         //Resolve attribute references
-        List<AtlasEntity> resolvedReferences = new ArrayList<>();
+        List<AtlasObjectId> resolvedReferences = new ArrayList<>();
 
-        for (AtlasEntity entity : context.getUnResolvedEntityReferences()) {
+        for (AtlasObjectId entityId : context.getUnresolvedIdsByUniqAttribs()) {
             //query in graph repo that given unique attribute - check for deleted also?
-            Optional<AtlasVertex> vertex = resolveByUniqueAttribute(entity);
+            Optional<AtlasVertex> vertex = resolveByUniqueAttribute(entityId);
+
             if (vertex.isPresent()) {
-                context.addRepositoryResolvedReference(new AtlasObjectId(entity.getTypeName(), entity.getGuid()), vertex.get());
-                resolvedReferences.add(entity);
+                context.addResolvedId(entityId, vertex.get());
+                resolvedReferences.add(entityId);
             }
         }
 
-        context.removeUnResolvedEntityReferences(resolvedReferences);
-
-        if (context.getUnResolvedEntityReferences().size() > 0) {
-            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND, context.getUnResolvedEntityReferences().toString());
-        }
+        context.removeUnresolvedIdsByUniqAttribs(resolvedReferences);
 
         //Resolve root references
         for (AtlasEntity entity : context.getRootEntities()) {
-            if ( !context.isResolved(entity.getGuid()) ) {
+            AtlasObjectId entityId = entity.getAtlasObjectId();
+
+            if (!context.isResolvedId(entityId) ) {
                 Optional<AtlasVertex> vertex = resolveByUniqueAttribute(entity);
+
                 if (vertex.isPresent()) {
-                    context.addRepositoryResolvedReference(new AtlasObjectId(entity.getTypeName(), entity.getGuid()), vertex.get());
+                    context.addResolvedId(entityId, vertex.get());
+                    context.removeUnResolvedId(entityId);
                 }
             }
         }
@@ -108,23 +106,57 @@ public class UniqAttrBasedEntityResolver implements EntityResolver {
         for (AtlasStructType.AtlasAttribute attr : entityType.getAllAttributes().values()) {
             if (attr.getAttributeDef().getIsUnique()) {
                 Object attrVal = entity.getAttribute(attr.getName());
-                if (attrVal != null) {
-                    String qualifiedAttrName = attr.getQualifiedAttributeName();
-                    AtlasVertex vertex = null;
-                    try {
-                        vertex = graphHelper.findVertex(qualifiedAttrName, attrVal,
-                            Constants.ENTITY_TYPE_PROPERTY_KEY, entityType.getTypeName(),
-                            Constants.STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE
-                                .name());
-
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Found vertex by unique attribute : " + qualifiedAttrName + "=" + attrVal);
-                        }
-                        if (vertex != null) {
-                            return Optional.of(vertex);
-                        }
-                    } catch (EntityNotFoundException e) {
-                        //Ignore if not found
+
+                if (attrVal == null) {
+                    continue;
+                }
+
+                Optional<AtlasVertex> vertex = findByTypeAndQualifiedName(entityType.getTypeName(), attr.getQualifiedAttributeName(), attrVal);
+
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Found vertex by unique attribute : " + attr.getQualifiedAttributeName() + "=" + attrVal);
+                }
+
+                if (!vertex.isPresent()) {
+                    vertex = findBySuperTypeAndQualifiedName(entityType.getTypeName(), attr.getQualifiedAttributeName(), attrVal);
+                }
+
+                if (vertex.isPresent()) {
+                    return vertex;
+                }
+            }
+        }
+
+        return Optional.absent();
+    }
+
+    Optional<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 (String attrName : uniqueAttributes.keySet()) {
+                AtlasStructType.AtlasAttribute attr = entityType.getAttribute(attrName);
+
+                if (attr.getAttributeDef().getIsUnique()) {
+                    Object attrVal = uniqueAttributes.get(attr.getName());
+
+                    if (attrVal == null) {
+                        continue;
+                    }
+
+                    Optional<AtlasVertex> vertex = findByTypeAndQualifiedName(entityId.getTypeName(), attr.getQualifiedAttributeName(), attrVal);
+
+                    if (!vertex.isPresent()) {
+                        vertex = findBySuperTypeAndQualifiedName(entityId.getTypeName(), attr.getQualifiedAttributeName(), attrVal);
+                    }
+
+                    if (vertex.isPresent()) {
+                        return vertex;
                     }
                 }
             }
@@ -132,6 +164,46 @@ public class UniqAttrBasedEntityResolver implements EntityResolver {
         return Optional.absent();
     }
 
+    Optional<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);
+            }
+            if (vertex != null) {
+                return Optional.of(vertex);
+            }
+        } catch (EntityNotFoundException e) {
+            //Ignore if not found
+        }
+        return Optional.absent();
+    }
+
+    Optional<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);
+            }
+            if (vertex != null) {
+                return Optional.of(vertex);
+            }
+        } catch (EntityNotFoundException e) {
+            //Ignore if not found
+        }
+        return Optional.absent();
+    }
+
     @Override
     public void cleanUp() {
         //Nothing to cleanup

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/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 546cd0c..6cbb602 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
@@ -71,6 +71,8 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.atlas.TestUtils.COLUMNS_ATTR_NAME;
+import static org.apache.atlas.TestUtils.COLUMN_TYPE;
+import static org.apache.atlas.TestUtils.NAME;
 import static org.apache.atlas.TestUtils.TABLE_TYPE;
 import static org.apache.atlas.TestUtils.randomString;
 import static org.testng.Assert.assertEquals;
@@ -91,6 +93,10 @@ public class AtlasEntityStoreV1Test {
     @Inject
     MetadataService metadataService;
     
+    private Map<String, AtlasEntity> deptEntityMap;
+    private Map<String, AtlasEntity> dbEntityMap;
+    private Map<String, AtlasEntity> tableEntityMap;
+
     private AtlasEntity deptEntity;
     private AtlasEntity dbEntity;
     private AtlasEntity tableEntity;
@@ -105,9 +111,13 @@ public class AtlasEntityStoreV1Test {
         final AtlasTypesDef hiveTypesDef = TestUtilsV2.defineHiveTypes();
         typeDefStore.createTypesDef(hiveTypesDef);
         
-        deptEntity = TestUtilsV2.createDeptEg1();
-        dbEntity = TestUtilsV2.createDBEntity();
-        tableEntity = TestUtilsV2.createTableEntity(dbEntity.getGuid());
+        deptEntityMap = TestUtilsV2.createDeptEg1();
+        dbEntityMap = TestUtilsV2.createDBEntity();
+        tableEntityMap = TestUtilsV2.createTableEntity(dbEntityMap.keySet().iterator().next());
+
+        deptEntity = deptEntityMap.values().iterator().next();
+        dbEntity = dbEntityMap.values().iterator().next();
+        tableEntity = tableEntityMap.values().iterator().next();
     }
 
     @AfterClass
@@ -124,45 +134,46 @@ public class AtlasEntityStoreV1Test {
         ArrayVertexMapper arrVertexMapper = new ArrayVertexMapper(deleteHandler);
         MapVertexMapper mapVertexMapper = new MapVertexMapper(deleteHandler);
 
-        List<EntityResolver> entityResolvers = new ArrayList<>();
-        entityResolvers.add(new IDBasedEntityResolver());
-        entityResolvers.add(new UniqAttrBasedEntityResolver(typeRegistry));
-
-        EntityGraphDiscovery graphDiscovery = new AtlasEntityGraphDiscoveryV1(typeRegistry, entityResolvers);
 
         entityStore = new AtlasEntityStoreV1(new EntityGraphMapper(arrVertexMapper, mapVertexMapper, deleteHandler));
-        entityStore.init(typeRegistry, graphDiscovery);
+        entityStore.init(typeRegistry);
 
         RequestContextV1.clear();
     }
 
     @Test
     public void testCreate() throws Exception {
-        EntityMutationResponse response = entityStore.createOrUpdate(deptEntity);
+        EntityMutationResponse response = entityStore.createOrUpdate(deptEntityMap);
 
         validateMutationResponse(response, EntityMutations.EntityOperation.CREATE, 5);
-        AtlasEntityHeader deptEntity = response.getFirstEntityCreated();
+        AtlasEntityHeader deptEntity = response.getFirstCreatedEntityByTypeName(TestUtilsV2.DEPARTMENT_TYPE);
 
-        validateAttributes(deptEntity);
+        final Map<EntityMutations.EntityOperation, List<AtlasEntityHeader>> entitiesMutated = response.getEntitiesMutated();
+        List<AtlasEntityHeader> entitiesCreated = entitiesMutated.get(EntityMutations.EntityOperation.CREATE);
+
+        for (AtlasEntityHeader header : entitiesCreated) {
+            validateAttributes(deptEntityMap, header);
+        }
 
         //Create DB
-        EntityMutationResponse dbCreationResponse = entityStore.createOrUpdate(dbEntity);
+        EntityMutationResponse dbCreationResponse = entityStore.createOrUpdate(dbEntityMap);
         validateMutationResponse(dbCreationResponse, EntityMutations.EntityOperation.CREATE, 1);
 
-        AtlasEntityHeader dbEntity = dbCreationResponse.getFirstEntityCreated();
-        validateAttributes(dbEntity);
+        AtlasEntityHeader dbEntity = dbCreationResponse.getFirstCreatedEntityByTypeName(TestUtilsV2.DATABASE_TYPE);
+        validateAttributes(dbEntityMap, dbEntity);
 
         //Create Table
         //Update DB guid
         AtlasObjectId dbId = (AtlasObjectId) tableEntity.getAttribute("database");
         dbId.setGuid(dbEntity.getGuid());
+        tableEntityMap.put(dbId.getGuid(), dbEntityMap.values().iterator().next());
         tableEntity.setAttribute("database", dbId);
 
-        EntityMutationResponse tableCreationResponse = entityStore.createOrUpdate(tableEntity);
+        EntityMutationResponse tableCreationResponse = entityStore.createOrUpdate(tableEntityMap);
         validateMutationResponse(tableCreationResponse, EntityMutations.EntityOperation.CREATE, 1);
 
-        AtlasEntityHeader tableEntity = tableCreationResponse.getFirstEntityCreated();
-        validateAttributes(tableEntity);
+        AtlasEntityHeader tableEntity = tableCreationResponse.getFirstCreatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableEntity);
 
     }
 
@@ -171,74 +182,86 @@ public class AtlasEntityStoreV1Test {
         //clear state
         init();
 
-        AtlasEntity entityClone = new AtlasEntity(deptEntity);
+//        Map<String, AtlasEntity> entityCloneMap = new HashMap<>();
+//        AtlasEntity entityClone = new AtlasEntity(deptEntity);
+//        List<AtlasObjectId> employees = (List<AtlasObjectId>) entityClone.getAttribute("employees");
+//        AtlasEntity entityRemoved = clearSubOrdinates(employees, 1);
+//        entityClone.setAttribute("employees", employees);
+//        EntityMutationResponse response = entityStore.createOrUpdate(entityCloneMap);
+//
+//        validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 5);
+//        AtlasEntityHeader deptEntity = response.getFirstEntityUpdated();
+//        Assert.assertEquals(((List<AtlasEntity>)(((List<AtlasEntity>) deptEntity.getAttribute("employees")).get(1).getAttribute("subordinates"))).size(), 1);
+//
+//        init();
+//        //add  entity back
+//        addSubordinate(employees.get(1), entityRemoved);
+//        response = entityStore.createOrUpdate(entityCloneMap);
+//        validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 5);
+//        deptEntity = response.getFirstEntityUpdated();
+//        validateAttributes(deptEntity);
 
-        List<AtlasEntity> employees = (List<AtlasEntity>) entityClone.getAttribute("employees");
 
-        AtlasEntity entityRemoved = clearSubOrdinates(employees, 1);
-        entityClone.setAttribute("employees", employees);
-        EntityMutationResponse response = entityStore.createOrUpdate(entityClone);
-        
-        validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 5);
-        AtlasEntityHeader deptEntity = response.getFirstEntityUpdated();
-        Assert.assertEquals(((List<AtlasEntity>)(((List<AtlasEntity>) deptEntity.getAttribute("employees")).get(1).getAttribute("subordinates"))).size(), 1);
-
-        init();
-        //add  entity back
-        addSubordinate(employees.get(1), entityRemoved);
-        response = entityStore.createOrUpdate(entityClone);
-        validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 5);
-        deptEntity = response.getFirstEntityUpdated();
-        validateAttributes(deptEntity);
+        Map<String, AtlasEntity> tableUpdatedMap = new HashMap<>();
+        tableUpdatedMap.put(dbEntity.getGuid(), dbEntity);
 
         //test array of class with id
-        final List<AtlasEntity> columns = new ArrayList<>();
+        final List<AtlasObjectId> columns = new ArrayList<>();
 
         AtlasEntity col1 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
         col1.setAttribute(TestUtilsV2.NAME, "col1");
-        columns.add(col1);
+        columns.add(col1.getAtlasObjectId());
+        tableUpdatedMap.put(col1.getGuid(), col1);
+
         AtlasEntity tableUpdated = new AtlasEntity(tableEntity);
         tableUpdated.setAttribute(TestUtilsV2.COLUMNS_ATTR_NAME, columns);
+        tableUpdatedMap.put(tableUpdated.getGuid(), tableUpdated);
 
         init();
-        response = entityStore.createOrUpdate(tableUpdated);
-        AtlasEntityHeader updatedTable = response.getFirstEntityUpdated();
-        validateAttributes(updatedTable);
+        EntityMutationResponse response = entityStore.createOrUpdate(tableUpdatedMap);
+        AtlasEntityHeader updatedTable = response.getFirstUpdatedEntityByTypeName(tableUpdated.getTypeName());
+        validateAttributes(tableUpdatedMap, updatedTable);
 
         //Complete update. Add  array elements - col3,col4
         AtlasEntity col3 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
         col1.setAttribute(TestUtilsV2.NAME, "col3");
-        columns.add(col3);
+        columns.add(col3.getAtlasObjectId());
+        tableUpdatedMap.put(col3.getGuid(), col3);
 
         AtlasEntity col4 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
         col1.setAttribute(TestUtilsV2.NAME, "col4");
-        columns.add(col4);
+        columns.add(col4.getAtlasObjectId());
+        tableUpdatedMap.put(col4.getGuid(), col4);
 
         tableUpdated.setAttribute(COLUMNS_ATTR_NAME, columns);
         init();
-        response = entityStore.createOrUpdate(tableUpdated);
-        updatedTable = response.getFirstEntityUpdated();
-        validateAttributes(updatedTable);
+        response = entityStore.createOrUpdate(tableUpdatedMap);
+        updatedTable = response.getFirstUpdatedEntityByTypeName(tableUpdated.getTypeName());
+        validateAttributes(tableUpdatedMap, updatedTable);
 
         //Swap elements
+        tableUpdatedMap.clear();
         columns.clear();
-        columns.add(col4);
-        columns.add(col3);
-
         tableUpdated.setAttribute(COLUMNS_ATTR_NAME, columns);
+        tableUpdatedMap.put(tableUpdated.getGuid(), tableUpdated);
+        tableUpdatedMap.put(col3.getGuid(), col3);
+        tableUpdatedMap.put(col4.getGuid(), col4);
+        columns.add(col4.getAtlasObjectId());
+        columns.add(col3.getAtlasObjectId());
+
         init();
-        response = entityStore.createOrUpdate(tableUpdated);
-        updatedTable = response.getFirstEntityUpdated();
-        Assert.assertEquals(((List<AtlasEntity>) updatedTable.getAttribute(COLUMNS_ATTR_NAME)).size(), 2);
+        response = entityStore.createOrUpdate(tableUpdatedMap);
+        updatedTable = response.getFirstUpdatedEntityByTypeName(tableUpdated.getTypeName());
+        Assert.assertEquals(((List<AtlasObjectId>) updatedTable.getAttribute(COLUMNS_ATTR_NAME)).size(), 2);
 
         assertEquals(response.getEntitiesByOperation(EntityMutations.EntityOperation.DELETE).size(), 1);  //col1 is deleted
 
         //Update array column to null
         tableUpdated.setAttribute(COLUMNS_ATTR_NAME, null);
         init();
-        response = entityStore.createOrUpdate(tableUpdated);
-        updatedTable = response.getFirstEntityUpdated();
-        validateAttributes(updatedTable);
+        response = entityStore.createOrUpdate(tableUpdatedMap);
+        updatedTable = response.getFirstUpdatedEntityByTypeName(tableUpdated.getTypeName());
+        validateAttributes(tableUpdatedMap, updatedTable);
         assertEquals(response.getEntitiesByOperation(EntityMutations.EntityOperation.DELETE).size(), 2);
 
     }
@@ -246,6 +269,7 @@ public class AtlasEntityStoreV1Test {
     @Test(dependsOnMethods = "testCreate")
     public void testUpdateEntityWithMap() throws Exception {
 
+        final Map<String, AtlasEntity> tableCloneMap = new HashMap<>();
         final AtlasEntity tableClone = new AtlasEntity(tableEntity);
         final Map<String, AtlasStruct> partsMap = new HashMap<>();
         partsMap.put("part0", new AtlasStruct(TestUtils.PARTITION_STRUCT_TYPE,
@@ -255,11 +279,13 @@ public class AtlasEntityStoreV1Test {
 
         
         tableClone.setAttribute("partitionsMap", partsMap);
+        tableCloneMap.put(tableClone.getGuid(), tableClone);
+
 
         init();
-        EntityMutationResponse response = entityStore.createOrUpdate(tableClone);
-        final AtlasEntityHeader tableDefinition1 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition1);
+        EntityMutationResponse response = entityStore.createOrUpdate(tableCloneMap);
+        final AtlasEntityHeader tableDefinition1 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition1);
                 
         Assert.assertTrue(partsMap.get("part0").equals(((Map<String, AtlasStruct>) tableDefinition1.getAttribute("partitionsMap")).get("part0")));
 
@@ -271,9 +297,9 @@ public class AtlasEntityStoreV1Test {
         tableClone.setAttribute("partitionsMap", partsMap);
 
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition2 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition2);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition2 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition2);
 
         assertEquals(((Map<String, AtlasStruct>) tableDefinition2.getAttribute("partitionsMap")).size(), 2);
         Assert.assertTrue(partsMap.get("part1").equals(((Map<String, AtlasStruct>) tableDefinition2.getAttribute("partitionsMap")).get("part1")));
@@ -287,9 +313,9 @@ public class AtlasEntityStoreV1Test {
         tableClone.setAttribute("partitionsMap", partsMap);
 
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition3 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition3);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition3 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition3);
 
         assertEquals(((Map<String, AtlasStruct>) tableDefinition3.getAttribute("partitionsMap")).size(), 2);
         Assert.assertNull(((Map<String, AtlasStruct>) tableDefinition3.getAttribute("partitionsMap")).get("part0"));
@@ -299,9 +325,9 @@ public class AtlasEntityStoreV1Test {
         init();
         AtlasStruct partition2 = partsMap.get("part2");
         partition2.setAttribute(TestUtilsV2.NAME, "test2Updated");
-        response = entityStore.createOrUpdate(tableClone);
-        final AtlasEntityHeader tableDefinition4 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition4);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        final AtlasEntityHeader tableDefinition4 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition4);
 
         assertEquals(((Map<String, AtlasStruct>) tableDefinition4.getAttribute("partitionsMap")).size(), 2);
         Assert.assertNull(((Map<String, AtlasStruct>) tableDefinition4.getAttribute("partitionsMap")).get("part0"));
@@ -312,16 +338,23 @@ public class AtlasEntityStoreV1Test {
 
         //Test map pointing to a class
 
-        final Map<String, AtlasEntity> columnsMap = new HashMap<>();
+        final Map<String, AtlasObjectId> columnsMap = new HashMap<>();
+
+        Map<String, AtlasEntity> col0TypeMap = new HashMap<>();
         AtlasEntity col0Type = new AtlasEntity(TestUtilsV2.COLUMN_TYPE,
             new HashMap<String, Object>() {{
                 put(TestUtilsV2.NAME, "test1");
                 put("type", "string");
                 put("table", new AtlasObjectId(TABLE_TYPE, tableDefinition1.getGuid()));
             }});
+
+        col0TypeMap.put(col0Type.getGuid(), col0Type);
+
+
         init();
-        entityStore.createOrUpdate(col0Type);
+        entityStore.createOrUpdate(col0TypeMap);
 
+        Map<String, AtlasEntity> col1TypeMap = new HashMap<>();
         AtlasEntity col1Type = new AtlasEntity(TestUtils.COLUMN_TYPE,
             new HashMap<String, Object>() {{
                 put(TestUtilsV2.NAME, "test2");
@@ -330,43 +363,50 @@ public class AtlasEntityStoreV1Test {
             }});
 
         init();
-        entityStore.createOrUpdate(col1Type);
+        col1TypeMap.put(col1Type.getGuid(), col1Type);
+        entityStore.createOrUpdate(col1TypeMap);
+
+        AtlasObjectId col0Id = new AtlasObjectId(col0Type.getTypeName(), col0Type.getGuid());
+        AtlasObjectId col1Id = new AtlasObjectId(col1Type.getTypeName(), col1Type.getGuid());
+
+        columnsMap.put("col0", col0Id);
+        columnsMap.put("col1", col1Id);
+        tableCloneMap.put(col0Type.getGuid(), col0Type);
+        tableCloneMap.put(col1Type.getGuid(), col1Type);
 
-        columnsMap.put("col0", col0Type);
-        columnsMap.put("col1", col1Type);
         tableClone.setAttribute(TestUtils.COLUMNS_MAP, columnsMap);
 
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition5 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition5);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition5 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition5);
 
         //Swap elements
         columnsMap.clear();
-        columnsMap.put("col0", col1Type);
-        columnsMap.put("col1", col0Type);
+        columnsMap.put("col0", col1Id);
+        columnsMap.put("col1", col0Id);
 
         tableClone.setAttribute(TestUtils.COLUMNS_MAP, columnsMap);
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition6 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition6);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition6 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition6);
 
         //Drop the first key and change the class type as well to col0
         columnsMap.clear();
-        columnsMap.put("col0", col0Type);
+        columnsMap.put("col0", col0Id);
 
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition7 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition7);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition7 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition7);
 
         //Clear state
         tableClone.setAttribute(TestUtils.COLUMNS_MAP, null);
         init();
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition8 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition8);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition8 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition8);
     }
 
     @Test(dependsOnMethods = "testCreate")
@@ -374,17 +414,19 @@ public class AtlasEntityStoreV1Test {
         //clear state
         init();
 
+        Map<String, AtlasEntity> entityCloneMap = new HashMap<>();
         AtlasEntity entityClone = new AtlasEntity(tableEntity);
+        entityCloneMap.put(entityClone.getGuid(), entityClone);
 
         //Add a new entry
         Map<String, String> paramsMap = (Map<String, String>) entityClone.getAttribute("parametersMap");
         paramsMap.put("newParam", "value");
         entityClone.setAttribute("parametersMap", paramsMap);
 
-        EntityMutationResponse response = entityStore.createOrUpdate(entityClone);
+        EntityMutationResponse response = entityStore.createOrUpdate(entityCloneMap);
         validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 1);
-        AtlasEntityHeader tableEntity = response.getFirstEntityUpdated();
-        validateAttributes(tableEntity);
+        AtlasEntityHeader tableEntity = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(entityCloneMap, tableEntity);
 
         //clear state
         init();
@@ -393,10 +435,10 @@ public class AtlasEntityStoreV1Test {
         paramsMap.remove("key1");
         entityClone.setAttribute("parametersMap", paramsMap);
 
-        response = entityStore.createOrUpdate(entityClone);
+        response = entityStore.createOrUpdate(entityCloneMap);
         validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 1);
-        tableEntity = response.getFirstEntityUpdated();
-        validateAttributes(tableEntity);
+        tableEntity = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(entityCloneMap, tableEntity);
     }
 
     @Test(dependsOnMethods = "testCreate")
@@ -412,10 +454,10 @@ public class AtlasEntityStoreV1Test {
         List<AtlasStruct> partitions = new ArrayList<AtlasStruct>(){{ add(partition1); add(partition2); }};
         tableEntity.setAttribute("partitions", partitions);
 
-        EntityMutationResponse response = entityStore.createOrUpdate(tableEntity);
-        AtlasEntityHeader tableDefinition = response.getFirstEntityUpdated();
+        EntityMutationResponse response = entityStore.createOrUpdate(tableEntityMap);
+        AtlasEntityHeader tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
 
-        validateAttributes(tableDefinition);
+        validateAttributes(tableEntityMap, tableDefinition);
 
         //add a new element to array of struct
         init();
@@ -423,25 +465,25 @@ public class AtlasEntityStoreV1Test {
         partition3.setAttribute(TestUtilsV2.NAME, "part3");
         partitions.add(partition3);
         tableEntity.setAttribute("partitions", partitions);
-        response = entityStore.createOrUpdate(tableEntity);
-        tableDefinition = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition);
+        response = entityStore.createOrUpdate(tableEntityMap);
+        tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableDefinition);
 
         //remove one of the struct values
         init();
         partitions.remove(1);
         tableEntity.setAttribute("partitions", partitions);
-        response = entityStore.createOrUpdate(tableEntity);
-        tableDefinition = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition);
+        response = entityStore.createOrUpdate(tableEntityMap);
+        tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableDefinition);
 
         //Update struct value within array of struct
         init();
         partitions.get(0).setAttribute(TestUtilsV2.NAME, "part4");
         tableEntity.setAttribute("partitions", partitions);
-        response = entityStore.createOrUpdate(tableEntity);
-        tableDefinition = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition);
+        response = entityStore.createOrUpdate(tableEntityMap);
+        tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableDefinition);
 
 
         //add a repeated element to array of struct
@@ -450,17 +492,17 @@ public class AtlasEntityStoreV1Test {
         partition4.setAttribute(TestUtilsV2.NAME, "part4");
         partitions.add(partition4);
         tableEntity.setAttribute("partitions", partitions);
-        response = entityStore.createOrUpdate(tableEntity);
-        tableDefinition = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition);
+        response = entityStore.createOrUpdate(tableEntityMap);
+        tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableDefinition);
 
         // Remove all elements. Should set array attribute to null
         init();
         partitions.clear();
         tableEntity.setAttribute("partitions", partitions);
-        response = entityStore.createOrUpdate(tableEntity);
-        tableDefinition = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition);
+        response = entityStore.createOrUpdate(tableEntityMap);
+        tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableEntityMap, tableDefinition);
     }
 
 
@@ -468,53 +510,57 @@ public class AtlasEntityStoreV1Test {
     public void testStructs() throws Exception {
         init();
 
+        Map<String, AtlasEntity> tableCloneMap = new HashMap<>();
         AtlasEntity tableClone = new AtlasEntity(tableEntity);
         AtlasStruct serdeInstance = new AtlasStruct(TestUtils.SERDE_TYPE);
         serdeInstance.setAttribute(TestUtilsV2.NAME, "serde1Name");
         serdeInstance.setAttribute("serde", "test");
         serdeInstance.setAttribute("description", "testDesc");
         tableClone.setAttribute("serde1", serdeInstance);
-        tableClone.setAttribute("database", dbEntity);
+        tableClone.setAttribute("database", new AtlasObjectId(dbEntity.getTypeName(), new HashMap<String, Object>() {{
+            put(TestUtilsV2.NAME, dbEntity.getAttribute(TestUtilsV2.NAME));
+        }}));
+
+        tableCloneMap.put(tableClone.getGuid(), tableClone);
 
-        EntityMutationResponse response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition1 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition1);
+        EntityMutationResponse response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition1 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition1);
 
         //update struct attribute
         init();
         serdeInstance.setAttribute("serde", "testUpdated");
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition2 = response.getFirstEntityUpdated();
-        validateAttributes(tableDefinition2);
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition2 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
+        validateAttributes(tableCloneMap, tableDefinition2);
 
         //set to null
         init();
         tableClone.setAttribute("description", null);
-        response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition3 = response.getFirstEntityUpdated();
+        response = entityStore.createOrUpdate(tableCloneMap);
+        AtlasEntityHeader tableDefinition3 = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
         Assert.assertNull(tableDefinition3.getAttribute("description"));
-        validateAttributes(tableDefinition3);
+        validateAttributes(tableCloneMap, tableDefinition3);
     }
 
-    private AtlasEntity clearSubOrdinates(List<AtlasEntity> employees, int index) {
-
-        AtlasEntity ret = null;
-        List<AtlasEntity> subOrdinates = (List<AtlasEntity>) employees.get(index).getAttribute("subordinates");
-        List<AtlasEntity> subOrdClone = new ArrayList<>(subOrdinates);
-        ret = subOrdClone.remove(index);
-
-        employees.get(index).setAttribute("subordinates", subOrdClone);
-        return ret;
-    }
-
-    private int addSubordinate(AtlasEntity manager, AtlasEntity employee) {
-        List<AtlasEntity> subOrdinates = (List<AtlasEntity>) manager.getAttribute("subordinates");
-        subOrdinates.add(employee);
-
-        manager.setAttribute("subordinates", subOrdinates);
-
-        return subOrdinates.size() - 1;
-    }
+//    private AtlasEntity clearSubOrdinates(List<AtlasObjectId> employees, int index) {
+//
+//        AtlasEntity ret = null;
+//        AtlasObjectId employee = employees.get(index);
+//        AtlasEntity subOrdClone = new ArrayList<>(subOrdinates);
+//        ret = subOrdClone.remove(index);
+//
+//        employees.get(index).setAttribute("subordinates", subOrdClone);
+//        return ret;
+//    }
+//
+//    private int addSubordinate(AtlasEntity manager, AtlasEntity employee) {
+//        List<AtlasEntity> subOrdinates = (List<AtlasEntity>) manager.getAttribute("subordinates");
+//        subOrdinates.add(employee);
+//
+//        manager.setAttribute("subordinates", subOrdinates);
+//        return subOrdinates.size() - 1;
+//    }
 
     private void validateMutationResponse(EntityMutationResponse response, EntityMutations.EntityOperation op, int expectedNumCreated) {
         List<AtlasEntityHeader> entitiesCreated = response.getEntitiesByOperation(op);
@@ -522,13 +568,13 @@ public class AtlasEntityStoreV1Test {
         Assert.assertEquals(entitiesCreated.size(), expectedNumCreated);
     }
 
-    private void validateAttributes(AtlasEntityHeader entity) throws AtlasBaseException, AtlasException {
+    private void validateAttributes(Map<String, AtlasEntity> entityMap, AtlasEntityHeader entity) throws AtlasBaseException, AtlasException {
         //TODO : Use the older API for get until new instance API is ready and validated
         ITypedReferenceableInstance instance = metadataService.getEntityDefinition(entity.getGuid());
-        assertAttributes(entity, instance);
+        assertAttributes(entityMap, entity, instance);
     }
 
-    private void assertAttributes(AtlasStruct entity, IInstance instance) throws AtlasBaseException, AtlasException {
+    private void assertAttributes(Map<String, AtlasEntity> entityMap, AtlasStruct entity, IInstance instance) throws AtlasBaseException, AtlasException {
         LOG.debug("Asserting type : " + entity.getTypeName());
         AtlasStructType entityType = (AtlasStructType) typeRegistry.getType(instance.getTypeName());
         for (String attrName : entity.getAttributes().keySet()) {
@@ -536,11 +582,11 @@ public class AtlasEntityStoreV1Test {
             Object expected = instance.get(attrName);
 
             AtlasType attrType = entityType.getAttributeType(attrName);
-            assertAttribute(actual, expected, attrType, attrName);
+            assertAttribute(entityMap, actual, expected, attrType, attrName);
         }
     }
 
-    private void assertAttribute(Object actual, Object expected, AtlasType attributeType, String attrName) throws AtlasBaseException, AtlasException {
+    private void assertAttribute(Map<String, AtlasEntity> actualEntityMap, Object actual, Object expected, AtlasType attributeType, String attrName) throws AtlasBaseException, AtlasException {
         LOG.debug("Asserting attribute : " + attrName);
 
         switch(attributeType.getTypeCategory()) {
@@ -550,9 +596,16 @@ public class AtlasEntityStoreV1Test {
                 Assert.assertTrue(AtlasEntity.isAssigned(guid));
             } else {
                 ReferenceableInstance expectedInstance = (ReferenceableInstance) expected;
-                AtlasEntity actualInstance = (AtlasEntity) actual;
-                if ( actualInstance != null) {
-                    assertAttributes(actualInstance, expectedInstance);
+                if (actual instanceof AtlasObjectId) {
+                    AtlasEntity actualEntity = actualEntityMap.get(((AtlasObjectId) actual).getGuid());
+                    if (actualEntity != null) {
+                        assertAttributes(actualEntityMap, actualEntity, expectedInstance);
+                    }
+                } else {
+                    AtlasEntity actualInstance = (AtlasEntity) actual;
+                    if (actualInstance != null) {
+                        assertAttributes(actualEntityMap , actualInstance, expectedInstance);
+                    }
                 }
             }
             break;
@@ -575,7 +628,7 @@ public class AtlasEntityStoreV1Test {
             if (expectedMap != null && actualMap != null) {
                 Assert.assertEquals(actualMap.size(), expectedMap.size());
                 for (Object key : actualMap.keySet()) {
-                    assertAttribute(actualMap.get(key), expectedMap.get(key), valueType, attrName);
+                    assertAttribute(actualEntityMap, actualMap.get(key), expectedMap.get(key), valueType, attrName);
                 }
             }
             break;
@@ -588,14 +641,14 @@ public class AtlasEntityStoreV1Test {
             if (CollectionUtils.isNotEmpty(actualList)) {
                 //actual list could have deleted entities . Hence size may not match.
                 for (int i = 0; i < actualList.size(); i++) {
-                    assertAttribute(actualList.get(i), expectedList.get(i), elemType, attrName);
+                    assertAttribute(actualEntityMap, actualList.get(i), expectedList.get(i), elemType, attrName);
                 }
             }
             break;
         case STRUCT:
             StructInstance structInstance = (StructInstance) expected;
             AtlasStruct newStructVal = (AtlasStruct) actual;
-            assertAttributes(newStructVal, structInstance);
+            assertAttributes(actualEntityMap, newStructVal, structInstance);
             break;
         default:
             Assert.fail("Unknown type category");
@@ -607,17 +660,21 @@ public class AtlasEntityStoreV1Test {
 
         init();
         //Create new db instance
-        final AtlasEntity databaseInstance = TestUtilsV2.createDBEntity();
+        final Map<String, AtlasEntity> databaseInstance = TestUtilsV2.createDBEntity();
 
         EntityMutationResponse response = entityStore.createOrUpdate(databaseInstance);
-        final AtlasEntityHeader dbCreated = response.getFirstEntityCreated();
+        final AtlasEntityHeader dbCreated = response.getFirstCreatedEntityByTypeName(TestUtilsV2.DATABASE_TYPE);
 
         init();
+        Map<String, AtlasEntity> tableCloneMap = new HashMap<>();
         AtlasEntity tableClone = new AtlasEntity(tableEntity);
         tableClone.setAttribute("database", new AtlasObjectId(TestUtils.DATABASE_TYPE, dbCreated.getGuid()));
-        response = entityStore.createOrUpdate(tableClone);
-        final AtlasEntityHeader tableDefinition = response.getFirstEntityUpdated();
 
+        tableCloneMap.put(dbCreated.getGuid(), databaseInstance.values().iterator().next());
+        tableCloneMap.put(tableClone.getGuid(), tableClone);
+
+        response = entityStore.createOrUpdate(tableCloneMap);
+        final AtlasEntityHeader tableDefinition = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.TABLE_TYPE);
         Assert.assertNotNull(tableDefinition.getAttribute("database"));
         Assert.assertEquals(((AtlasObjectId) tableDefinition.getAttribute("database")).getGuid(), dbCreated.getGuid());
     }
@@ -625,9 +682,9 @@ public class AtlasEntityStoreV1Test {
     @Test
     public void testCheckOptionalAttrValueRetention() throws Exception {
 
-        AtlasEntity dbEntity = TestUtilsV2.createDBEntity();
-        EntityMutationResponse response = entityStore.createOrUpdate(dbEntity);
-        AtlasEntityHeader firstEntityCreated = response.getFirstEntityCreated();
+        Map<String, AtlasEntity> dbEntityMap = TestUtilsV2.createDBEntity();
+        EntityMutationResponse response = entityStore.createOrUpdate(dbEntityMap);
+        AtlasEntityHeader firstEntityCreated = response.getFirstCreatedEntityByTypeName(TestUtilsV2.DATABASE_TYPE);
 
         //The optional boolean attribute should have a non-null value
         final String isReplicatedAttr = "isReplicated";
@@ -638,13 +695,16 @@ public class AtlasEntityStoreV1Test {
 
         //Update to true
         init();
+        AtlasEntity dbEntity = dbEntityMap.values().iterator().next();
         dbEntity.setAttribute(isReplicatedAttr, Boolean.TRUE);
         //Update array
         final HashMap<String, String> params = new HashMap<String, String>() {{ put("param1", "val1"); put("param2", "val2"); }};
         dbEntity.setAttribute(paramsAttr, params);
         //Complete update
-        response = entityStore.createOrUpdate(dbEntity);
-        AtlasEntityHeader firstEntityUpdated = response.getFirstEntityUpdated();
+
+        dbEntityMap.put(dbEntity.getGuid(), dbEntity);
+        response = entityStore.createOrUpdate(dbEntityMap);
+        AtlasEntityHeader firstEntityUpdated = response.getFirstUpdatedEntityByTypeName(TestUtilsV2.DATABASE_TYPE);
 
         Assert.assertNotNull(firstEntityUpdated.getAttribute(isReplicatedAttr));
         Assert.assertEquals(firstEntityUpdated.getAttribute(isReplicatedAttr), Boolean.TRUE);
@@ -684,6 +744,8 @@ public class AtlasEntityStoreV1Test {
         typeDefStore.createEntityDef(typeDefinition);
 
         //verify that entity can be created with reserved characters in string value, array value and map key and value
+        Map<String, AtlasEntity> entityCloneMap = new HashMap<>();
+
         AtlasEntity entity = new AtlasEntity();
         entity.setAttribute(strAttrName, randomStrWithReservedChars());
         entity.setAttribute(arrayAttrName, new String[]{randomStrWithReservedChars()});
@@ -691,9 +753,10 @@ public class AtlasEntityStoreV1Test {
             put(randomStrWithReservedChars(), randomStrWithReservedChars());
         }});
 
-        final EntityMutationResponse response = entityStore.createOrUpdate(entity);
+        entityCloneMap.put(entity.getGuid(), entity);
+        final EntityMutationResponse response = entityStore.createOrUpdate(entityCloneMap);
         final AtlasEntityHeader firstEntityCreated = response.getFirstEntityCreated();
-        validateAttributes(firstEntityCreated);
+        validateAttributes(entityCloneMap, firstEntityCreated);
 
 
         //Verify that search with reserved characters works - for string attribute
@@ -711,11 +774,12 @@ public class AtlasEntityStoreV1Test {
     @Test(expectedExceptions = AtlasBaseException.class)
     public void testCreateRequiredAttrNull() throws Exception {
         //Update required attribute
-
+        Map<String, AtlasEntity> tableCloneMap = new HashMap<>();
         AtlasEntity tableEntity = new AtlasEntity(TestUtilsV2.TABLE_TYPE);
         tableEntity.setAttribute(TestUtilsV2.NAME, "table_" + TestUtils.randomString());
+        tableCloneMap.put(tableEntity.getGuid(), tableEntity);
 
-        entityStore.createOrUpdate(tableEntity);
+        entityStore.createOrUpdate(tableCloneMap);
         Assert.fail("Expected exception while creating with required attribute null");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
index e3b4efa..aa14aff 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
@@ -38,7 +38,7 @@ public class AtlasArrayFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Collection fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Collection fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         Collection ret = null;
 
         if (v1Obj != null) {
@@ -57,7 +57,7 @@ public class AtlasArrayFormatConverter extends AtlasAbstractFormatConverter {
             Collection           v1List        = (Collection) v1Obj;
 
             for (Object v1Elem : v1List) {
-                Object convertedVal = elemConverter.fromV1ToV2(v1Elem, elemType);
+                Object convertedVal = elemConverter.fromV1ToV2(v1Elem, elemType, ctx);
 
                 ret.add(convertedVal);
             }
@@ -67,7 +67,7 @@ public class AtlasArrayFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Collection fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Collection fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         Collection ret = null;
 
         if (v2Obj != null) {
@@ -86,7 +86,7 @@ public class AtlasArrayFormatConverter extends AtlasAbstractFormatConverter {
             Collection           v2List        = (Collection) v2Obj;
 
             for (Object v2Elem : v2List) {
-                Object convertedVal = elemConverter.fromV2ToV1(v2Elem, elemType);
+                Object convertedVal = elemConverter.fromV2ToV1(v2Elem, elemType, ctx);
 
                 ret.add(convertedVal);
             }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasClassificationFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasClassificationFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasClassificationFormatConverter.java
index da71c31..dc740f5 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasClassificationFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasClassificationFormatConverter.java
@@ -40,7 +40,7 @@ public class AtlasClassificationFormatConverter extends AtlasStructFormatConvert
     }
 
     @Override
-    public AtlasClassification fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public AtlasClassification fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         AtlasClassification ret = null;
 
         if (v1Obj != null) {
@@ -51,7 +51,7 @@ public class AtlasClassificationFormatConverter extends AtlasStructFormatConvert
                 final Map v1Attribs = (Map) v1Map.get(ATTRIBUTES_PROPERTY_KEY);
 
                 if (MapUtils.isNotEmpty(v1Attribs)) {
-                    ret = new AtlasClassification(type.getTypeName(), fromV1ToV2(classificationType, v1Attribs));
+                    ret = new AtlasClassification(type.getTypeName(), fromV1ToV2(classificationType, v1Attribs, ctx));
                 } else {
                     ret = new AtlasClassification(type.getTypeName());
                 }
@@ -65,7 +65,7 @@ public class AtlasClassificationFormatConverter extends AtlasStructFormatConvert
                     LOG.error("IStruct.getValuesMap() failed", excp);
                 }
 
-                ret = new AtlasClassification(type.getTypeName(), fromV1ToV2(classificationType, v1Attribs));
+                ret = new AtlasClassification(type.getTypeName(), fromV1ToV2(classificationType, v1Attribs, ctx));
             } else {
                 throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Map or IStruct",
                                              v1Obj.getClass().getCanonicalName());

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEntityFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEntityFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEntityFormatConverter.java
index bbfcc59..8f85052 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEntityFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEntityFormatConverter.java
@@ -53,7 +53,7 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
     }
 
     @Override
-    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext context) throws AtlasBaseException {
         Object ret = null;
 
         if (v1Obj != null) {
@@ -67,13 +67,15 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
                 IReferenceableInstance entity    = (IReferenceableInstance) v1Obj;
                 Map<String, Object>    v1Attribs = null;
 
+                ret = new AtlasObjectId(entity.getTypeName(), entity.getId()._getId());
+
                 try {
                     v1Attribs = entity.getValuesMap();
                 } catch (AtlasException excp) {
                     LOG.error("IReferenceableInstance.getValuesMap() failed", excp);
                 }
 
-                AtlasEntityWithAssociations ret1 =  new AtlasEntityWithAssociations(entity.getTypeName(), super.fromV1ToV2(entityType, v1Attribs));
+                AtlasEntityWithAssociations ret1 =  new AtlasEntityWithAssociations(entity.getTypeName(), super.fromV1ToV2(entityType, v1Attribs, context));
                 ret1.setGuid(entity.getId()._getId());
                 ret1.setStatus(convertState(entity.getId().getState()));
                 AtlasSystemAttributes systemAttributes = entity.getSystemAttributes();
@@ -90,16 +92,17 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
                     for (String traitName : entity.getTraits()) {
                         IStruct             trait          = entity.getTrait(traitName);
                         AtlasType           classifiType   = typeRegistry.getType(traitName);
-                        AtlasClassification classification = (AtlasClassification) traitConverter.fromV1ToV2(trait, classifiType);
+                        AtlasClassification classification = (AtlasClassification) traitConverter.fromV1ToV2(trait, classifiType, context);
 
                         classifications.add(classification);
                     }
 
                     ret1.setClassifications(classifications);
                 }
-                ret = ret1;
+
+                context.addEntity(ret1);
             } else {
-                throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Id or IReferenceableInstance",
+                throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "IReferenceableInstance",
                                              v1Obj.getClass().getCanonicalName());
             }
         }
@@ -116,7 +119,7 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
     }
 
     @Override
-    public Object fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext context) throws AtlasBaseException {
         Object ret = null;
 
         if (v2Obj != null) {
@@ -136,15 +139,21 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
                 if (MapUtils.isEmpty(v2Attribs)) {
                     ret = new Id(idStr, 0, typeName);
                 } else {
-                    ret = new Referenceable(idStr, typeName, super.fromV2ToV1(entityType, v2Attribs));
+                    ret = new Referenceable(idStr, typeName, super.fromV2ToV1(entityType, v2Attribs, context));
                 }
             } else if (v2Obj instanceof AtlasEntity) {
                 AtlasEntity entity = (AtlasEntity) v2Obj;
 
                 ret = new Referenceable(entity.getGuid(), entity.getTypeName(),
-                                        fromV2ToV1(entityType, entity.getAttributes()));
+                                        fromV2ToV1(entityType, entity.getAttributes(), context));
+
             } else if (v2Obj instanceof AtlasObjectId) { // transient-id
-                ret = new Referenceable(((AtlasObjectId) v2Obj).getGuid(), type.getTypeName(), null);
+                AtlasEntity entity = context.getById(((AtlasObjectId) v2Obj).getGuid());
+                if ( entity == null) {
+                    throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "Could not find entity ",
+                        v2Obj.toString());
+                }
+                ret = this.fromV2ToV1(entity, typeRegistry.getType(((AtlasObjectId) v2Obj).getTypeName()), context);
             } else {
                 throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Map or AtlasEntity or String",
                                              v2Obj.getClass().getCanonicalName());

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
index 4f22437..6d8e3ae 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
@@ -30,12 +30,12 @@ public class AtlasEnumFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         return type.getNormalizedValue(v1Obj);
     }
 
     @Override
-    public Object fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         return type.getNormalizedValue(v2Obj);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverter.java
index 079f3be..1272543 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverter.java
@@ -20,12 +20,62 @@ package org.apache.atlas.web.adapters;
 
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntityWithAssociations;
 import org.apache.atlas.type.AtlasType;
 
+import java.util.HashMap;
+import java.util.Map;
+
 public interface AtlasFormatConverter {
-    Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException;
+    Object fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext context) throws AtlasBaseException;
 
-    Object fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException;
+    Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext context) throws AtlasBaseException;
 
     TypeCategory getTypeCategory();
+
+    public static class ConverterContext {
+
+        private Map<String, AtlasEntityWithAssociations> entities = null;
+
+        public void addEntity(AtlasEntityWithAssociations entity) {
+            if (entities == null) {
+                entities = new HashMap<>();
+            }
+            entities.put(entity.getGuid(), entity);
+        }
+
+        public void addEntity(AtlasEntity entity) {
+            if (entities == null) {
+                entities = new HashMap<>();
+            }
+            entities.put(entity.getGuid(), new AtlasEntityWithAssociations(entity));
+        }
+
+        public boolean exists(AtlasEntityWithAssociations entity) {
+            return entities != null ? entities.containsKey(entity.getGuid()) : false;
+        }
+
+        public AtlasEntity getById(String guid) {
+
+            if( entities != null) {
+                return entities.get(guid);
+            }
+
+            return null;
+        }
+
+        public Map<String, AtlasEntityWithAssociations> getEntities() {
+            return entities;
+        }
+
+        public void addEntities(Map<String, AtlasEntity> entities) {
+            if (this.entities == null) {
+                this.entities = new HashMap<>(entities.size());
+            }
+            for (String entityId : entities.keySet()) {
+                this.entities.put(entityId, new AtlasEntityWithAssociations(entities.get(entityId)));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
index 7368c72..692f917 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
@@ -47,7 +47,11 @@ import org.apache.atlas.typesystem.exception.TypeNotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 @Singleton
 public class AtlasInstanceRestAdapters {
@@ -63,19 +67,24 @@ public class AtlasInstanceRestAdapters {
     @Inject
     private MetadataService metadataService;
 
-    public ITypedReferenceableInstance[] getITypedReferenceables(List<AtlasEntity> entities) throws AtlasBaseException {
+    public ITypedReferenceableInstance[] getITypedReferenceables(Collection<AtlasEntity> entities) throws AtlasBaseException {
         ITypedReferenceableInstance[] entitiesInOldFormat = new ITypedReferenceableInstance[entities.size()];
 
+        AtlasFormatConverter.ConverterContext ctx = new AtlasFormatConverter.ConverterContext();
+        for(Iterator<AtlasEntity> i = entities.iterator(); i.hasNext(); ) {
+            ctx.addEntity(i.next());
+        }
+
+        Iterator<AtlasEntity> entityIterator = entities.iterator();
         for (int i = 0; i < entities.size(); i++) {
-            ITypedReferenceableInstance typedInstance = getITypedReferenceable(entities.get(i));
+            ITypedReferenceableInstance typedInstance = getITypedReferenceable(entityIterator.next(), ctx);
             entitiesInOldFormat[i] = typedInstance;
         }
-
         return entitiesInOldFormat;
     }
 
-    public ITypedReferenceableInstance getITypedReferenceable(AtlasEntity entity) throws AtlasBaseException {
-        Referenceable ref = getReferenceable(entity);
+    public ITypedReferenceableInstance getITypedReferenceable(AtlasEntity entity, AtlasFormatConverter.ConverterContext ctx) throws AtlasBaseException {
+        Referenceable ref = getReferenceable(entity, ctx);
 
         try {
             return metadataService.getTypedReferenceableInstance(ref);
@@ -85,10 +94,10 @@ public class AtlasInstanceRestAdapters {
         }
     }
 
-    public Referenceable getReferenceable(AtlasEntity entity) throws AtlasBaseException {
+    public Referenceable getReferenceable(AtlasEntity entity, final AtlasFormatConverter.ConverterContext ctx) throws AtlasBaseException {
         AtlasFormatConverter converter  = instanceFormatters.getConverter(TypeCategory.ENTITY);
         AtlasType            entityType = typeRegistry.getType(entity.getTypeName());
-        Referenceable        ref        = (Referenceable)converter.fromV2ToV1(entity, entityType);
+        Referenceable        ref        = (Referenceable)converter.fromV2ToV1(entity, entityType, ctx);
 
         return ref;
     }
@@ -96,7 +105,7 @@ public class AtlasInstanceRestAdapters {
     public ITypedStruct getTrait(AtlasClassification classification) throws AtlasBaseException {
         AtlasFormatConverter converter          = instanceFormatters.getConverter(TypeCategory.CLASSIFICATION);
         AtlasType            classificationType = typeRegistry.getType(classification.getTypeName());
-        Struct               trait               = (Struct)converter.fromV2ToV1(classification, classificationType);
+        Struct               trait               = (Struct)converter.fromV2ToV1(classification, classificationType, new AtlasFormatConverter.ConverterContext());
 
         try {
             return metadataService.createTraitInstance(trait);
@@ -112,21 +121,22 @@ public class AtlasInstanceRestAdapters {
         if (classificationType == null) {
             throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.CLASSIFICATION.name(), classification.getTypeName());
         }
-        AtlasClassification  ret                = (AtlasClassification)converter.fromV1ToV2(classification, classificationType);
+        AtlasClassification  ret                = (AtlasClassification)converter.fromV1ToV2(classification, classificationType, new AtlasFormatConverter.ConverterContext());
 
         return ret;
     }
 
-    public AtlasEntityWithAssociations getAtlasEntity(IReferenceableInstance referenceable) throws AtlasBaseException {
+    public Map<String, AtlasEntityWithAssociations> getAtlasEntity(IReferenceableInstance referenceable) throws AtlasBaseException {
+
         AtlasFormatConverter converter  = instanceFormatters.getConverter(TypeCategory.ENTITY);
         AtlasEntityType entityType = typeRegistry.getEntityTypeByName(referenceable.getTypeName());
         if (entityType == null) {
             throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), referenceable.getTypeName());
         }
 
-        AtlasEntityWithAssociations ret = (AtlasEntityWithAssociations)converter.fromV1ToV2(referenceable, entityType);
-
-        return ret;
+        AtlasFormatConverter.ConverterContext ctx = new AtlasFormatConverter.ConverterContext();
+        converter.fromV1ToV2(referenceable, entityType, ctx);
+        return ctx.getEntities();
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
index f390e82..6967c4f 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
@@ -35,7 +35,7 @@ public class AtlasMapFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Map fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Map fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         Map ret = null;
 
         if (v1Obj != null) {
@@ -52,8 +52,8 @@ public class AtlasMapFormatConverter extends AtlasAbstractFormatConverter {
                 for (Object key : v1Map.keySet()) {
                     Object value = v1Map.get(key);
 
-                    Object v2Key   = keyConverter.fromV1ToV2(key, keyType);
-                    Object v2Value = valueConverter.fromV1ToV2(value, valueType);
+                    Object v2Key   = keyConverter.fromV1ToV2(key, keyType, ctx);
+                    Object v2Value = valueConverter.fromV1ToV2(value, valueType, ctx);
 
                     ret.put(v2Key, v2Value);
                 }
@@ -67,7 +67,7 @@ public class AtlasMapFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Map fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Map fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         Map ret = null;
 
         if (v2Obj != null) {
@@ -84,8 +84,8 @@ public class AtlasMapFormatConverter extends AtlasAbstractFormatConverter {
                 for (Object key : v1Map.keySet()) {
                     Object value = v1Map.get(key);
 
-                    Object v2Key   = keyConverter.fromV2ToV1(key, keyType);
-                    Object v2Value = valueConverter.fromV2ToV1(value, valueType);
+                    Object v2Key   = keyConverter.fromV2ToV1(key, keyType, ctx);
+                    Object v2Value = valueConverter.fromV2ToV1(value, valueType, ctx);
 
                     ret.put(v2Key, v2Value);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasObjectIdConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasObjectIdConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasObjectIdConverter.java
new file mode 100644
index 0000000..b2de802
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasObjectIdConverter.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas.web.adapters;
+
+
+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.model.instance.AtlasObjectId;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Map;
+
+public class
+AtlasObjectIdConverter extends  AtlasAbstractFormatConverter {
+
+    public AtlasObjectIdConverter(AtlasFormatConverters registry, AtlasTypeRegistry typeRegistry) {
+        this(registry, typeRegistry, TypeCategory.OBJECT_ID_TYPE);
+    }
+
+    protected AtlasObjectIdConverter(AtlasFormatConverters registry, AtlasTypeRegistry typeRegistry, TypeCategory typeCategory) {
+        super(registry, typeRegistry, typeCategory);
+    }
+
+    @Override
+    public Object fromV1ToV2(Object v1Obj, AtlasType type, AtlasFormatConverter.ConverterContext converterContext) throws AtlasBaseException {
+        Object ret = null;
+
+        if (v1Obj != null) {
+            if (v1Obj instanceof Id) {
+                Id id = (Id) v1Obj;
+                ret = new AtlasObjectId(id.getTypeName(), id._getId());
+            } else if (v1Obj instanceof IReferenceableInstance) {
+                IReferenceableInstance entity = (IReferenceableInstance) v1Obj;
+                ret = new AtlasObjectId(entity.getTypeName(), entity.getId()._getId());
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext converterContext) throws AtlasBaseException {
+        Id ret = null;
+
+        if (v2Obj != null) {
+
+            if (v2Obj instanceof Map) {
+                Map    v2Map    = (Map) v2Obj;
+                String idStr    = (String)v2Map.get(AtlasObjectId.KEY_GUID);
+                String typeName = type.getTypeName();
+
+                if (StringUtils.isEmpty(idStr)) {
+                    throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND);
+                }
+
+                ret = new Id(idStr, 0, typeName);
+            } else if (v2Obj instanceof AtlasObjectId) { // transient-id
+                ret = new Id(((AtlasObjectId) v2Obj).getGuid(), 0, type.getTypeName());
+            } else if (v2Obj instanceof AtlasEntity) {
+                AtlasEntity entity = (AtlasEntity) v2Obj;
+                ret = new Id(((AtlasObjectId) v2Obj).getGuid(), 0, type.getTypeName());
+            } else {
+                throw new AtlasBaseException(AtlasErrorCode.TYPE_CATEGORY_INVALID, type.getTypeCategory().name());
+            }
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
index 382d1ef..2b70c5e 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
@@ -30,12 +30,12 @@ public class AtlasPrimitiveFormatConverter extends AtlasAbstractFormatConverter
     }
 
     @Override
-    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         return type.getNormalizedValue(v1Obj);
     }
 
     @Override
-    public Object fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext ctx) throws AtlasBaseException {
         return type.getNormalizedValue(v2Obj);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasStructFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasStructFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasStructFormatConverter.java
index 4b3dd4c..4d2e123 100644
--- a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasStructFormatConverter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasStructFormatConverter.java
@@ -50,7 +50,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV1ToV2(Object v1Obj, AtlasType type, ConverterContext converterContext) throws AtlasBaseException {
         AtlasStruct ret = null;
 
         if (v1Obj != null) {
@@ -61,7 +61,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
                 final Map v1Attribs = (Map) v1Map.get(ATTRIBUTES_PROPERTY_KEY);
 
                 if (MapUtils.isNotEmpty(v1Attribs)) {
-                    ret = new AtlasStruct(type.getTypeName(), fromV1ToV2(structType, v1Attribs));
+                    ret = new AtlasStruct(type.getTypeName(), fromV1ToV2(structType, v1Attribs, converterContext));
                 } else {
                     ret = new AtlasStruct(type.getTypeName());
                 }
@@ -75,7 +75,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
                     LOG.error("IStruct.getValuesMap() failed", excp);
                 }
 
-                ret = new AtlasStruct(type.getTypeName(), fromV1ToV2(structType, v1Attribs));
+                ret = new AtlasStruct(type.getTypeName(), fromV1ToV2(structType, v1Attribs, converterContext));
             } else {
                 throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Map or IStruct", v1Obj.getClass().getCanonicalName());
             }
@@ -85,7 +85,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
     }
 
     @Override
-    public Object fromV2ToV1(Object v2Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV2ToV1(Object v2Obj, AtlasType type, ConverterContext converterContext) throws AtlasBaseException {
         Struct ret = null;
 
         if (v2Obj != null) {
@@ -102,14 +102,14 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
                 }
 
                 if (MapUtils.isNotEmpty(v2Attribs)) {
-                    ret = new Struct(type.getTypeName(), fromV2ToV1(structType, v2Attribs));
+                    ret = new Struct(type.getTypeName(), fromV2ToV1(structType, v2Attribs, converterContext));
                 } else {
                     ret = new Struct(type.getTypeName());
                 }
             } else if (v2Obj instanceof AtlasStruct) {
                 AtlasStruct struct = (AtlasStruct) v2Obj;
 
-                ret = new Struct(type.getTypeName(), fromV2ToV1(structType, struct.getAttributes()));
+                ret = new Struct(type.getTypeName(), fromV2ToV1(structType, struct.getAttributes(), converterContext));
             } else {
                 throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Map or AtlasStruct", v2Obj.getClass().getCanonicalName());
             }
@@ -118,7 +118,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
         return ret;
     }
 
-    protected Map<String, Object> fromV2ToV1(AtlasStructType structType, Map attributes) throws AtlasBaseException {
+    protected Map<String, Object> fromV2ToV1(AtlasStructType structType, Map attributes, ConverterContext context) throws AtlasBaseException {
         Map<String, Object> ret = null;
 
         if (MapUtils.isNotEmpty(attributes)) {
@@ -132,11 +132,17 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
                     continue;
                 }
 
-                AtlasFormatConverter attrConverter = converterRegistry.getConverter(attrType.getTypeCategory());
-
                 Object v2Value = attributes.get(attr.getName());
-                Object v1Value = attrConverter.fromV2ToV1(v2Value, attrType);
+                Object v1Value = null;
 
+                AtlasFormatConverter attrConverter = null;
+                if (attrType.getTypeCategory() == TypeCategory.ENTITY && !attr.isContainedAttribute()) {
+                    attrConverter = new AtlasObjectIdConverter(converterRegistry, typeRegistry);
+                    v1Value = attrConverter.fromV2ToV1(v2Value, attrType, context);
+                } else {
+                    attrConverter = converterRegistry.getConverter(attrType.getTypeCategory());
+                    v1Value = attrConverter.fromV2ToV1(v2Value, attrType, context);
+                }
                 ret.put(attr.getName(), v1Value);
             }
         }
@@ -144,7 +150,7 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
         return ret;
     }
 
-    protected Map<String, Object> fromV1ToV2(AtlasStructType structType, Map attributes) throws AtlasBaseException {
+    protected Map<String, Object> fromV1ToV2(AtlasStructType structType, Map attributes, ConverterContext context) throws AtlasBaseException {
         Map<String, Object> ret = null;
 
         if (MapUtils.isNotEmpty(attributes)) {
@@ -152,11 +158,23 @@ public class AtlasStructFormatConverter extends AtlasAbstractFormatConverter {
 
             for (AtlasStructType.AtlasAttribute attr : structType.getAllAttributes().values()) {
                 AtlasType            attrType      = attr.getAttributeType();
-                AtlasFormatConverter attrConverter = converterRegistry.getConverter(attrType.getTypeCategory());
+
+                if (attrType == null) {
+                    LOG.warn("ignored attribute {}.{}: failed to find AtlasType", structType.getTypeName(), attr.getName());
+                    continue;
+                }
 
                 Object v1Value = attributes.get(attr.getName());
-                Object v2Value = attrConverter.fromV1ToV2(v1Value, attrType);
+                Object v2Value = null;
 
+                AtlasFormatConverter attrConverter = null;
+                if (attrType.getTypeCategory() == TypeCategory.ENTITY && !attr.isContainedAttribute()) {
+                    attrConverter = new AtlasObjectIdConverter(converterRegistry, typeRegistry);
+                    v2Value = attrConverter.fromV1ToV2(v1Value, attrType, context);
+                } else {
+                    attrConverter = converterRegistry.getConverter(attrType.getTypeCategory());
+                    v2Value = attrConverter.fromV1ToV2(v1Value, attrType, context);
+                }
                 ret.put(attr.getAttributeDef().getName(), v2Value);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d8c2a10e/webapp/src/main/java/org/apache/atlas/web/rest/EntitiesREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/EntitiesREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/EntitiesREST.java
index 5107767..af0377e 100644
--- a/webapp/src/main/java/org/apache/atlas/web/rest/EntitiesREST.java
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/EntitiesREST.java
@@ -25,6 +25,7 @@ import org.apache.atlas.model.SearchFilter;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasEntityWithAssociations;
 import org.apache.atlas.model.instance.ClassificationAssociateRequest;
 import org.apache.atlas.model.instance.EntityMutationResponse;
 import org.apache.atlas.repository.store.graph.AtlasEntityStore;
@@ -53,6 +54,7 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -92,9 +94,10 @@ public class EntitiesREST {
     @POST
     @Consumes(Servlets.JSON_MEDIA_TYPE)
     @Produces(Servlets.JSON_MEDIA_TYPE)
-    public EntityMutationResponse createOrUpdate(List<AtlasEntity> entities) throws AtlasBaseException {
+    public EntityMutationResponse createOrUpdate(Map<String, AtlasEntity> entities) throws AtlasBaseException {
         EntityMutationResponse response = null;
-        ITypedReferenceableInstance[] entitiesInOldFormat = restAdapters.getITypedReferenceables(entities);
+
+        ITypedReferenceableInstance[] entitiesInOldFormat = restAdapters.getITypedReferenceables(entities.values());
 
         try {
             final AtlasClient.EntityResult result = metadataService.updateEntities(entitiesInOldFormat);
@@ -114,7 +117,7 @@ public class EntitiesREST {
     @PUT
     @Consumes(Servlets.JSON_MEDIA_TYPE)
     @Produces(Servlets.JSON_MEDIA_TYPE)
-    public EntityMutationResponse update(List<AtlasEntity> entities) throws AtlasBaseException {
+    public EntityMutationResponse update(Map<String, AtlasEntity> entities) throws AtlasBaseException {
        return createOrUpdate(entities);
     }
 
@@ -135,8 +138,10 @@ public class EntitiesREST {
         for (String guid : guids) {
             try {
                ITypedReferenceableInstance ref = metadataService.getEntityDefinition(guid);
-               AtlasEntity entity = restAdapters.getAtlasEntity(ref);
-               entityList.add(entity);
+               Map<String, AtlasEntityWithAssociations> entityRet = restAdapters.getAtlasEntity(ref);
+
+               addToEntityList(entityList, entityRet.values());
+
             } catch (AtlasException e) {
                 throw toAtlasBaseException(e);
             }
@@ -146,6 +151,14 @@ public class EntitiesREST {
         return entities;
     }
 
+    private void addToEntityList(final List<AtlasEntity> entityList, final Collection<AtlasEntityWithAssociations> values) {
+        for (AtlasEntityWithAssociations val : values) {
+            if ( !entityList.contains(val)) {
+                entityList.add(val);
+            }
+        }
+    }
+
     /*******
      * Entity Delete
      *******/
@@ -168,28 +181,6 @@ public class EntitiesREST {
     }
 
     /**
-     * Bulk retrieval API for searching on entities by certain predefined attributes ( typeName, superType, name, qualifiedName etc) + optional user defined attributes
-     *
-     * @throws AtlasBaseException
-     */
-    @GET
-    @Produces(Servlets.JSON_MEDIA_TYPE)
-    public AtlasEntityHeader.AtlasEntityHeaders searchEntities() throws AtlasBaseException {
-        SearchFilter searchFilter = getSearchFilter();
-        AtlasEntity.AtlasEntities atlasEntities = entitiesStore.searchEntities(searchFilter);
-        AtlasEntityHeader.AtlasEntityHeaders entityHeaders = new AtlasEntityHeader.AtlasEntityHeaders();
-        entityHeaders.setList(new LinkedList<AtlasEntityHeader>());
-
-        if (atlasEntities != null) {
-            for (AtlasEntity atlasEntity : atlasEntities.getList()) {
-                entityHeaders.getList().add(new AtlasEntityHeader(atlasEntity.getTypeName(), atlasEntity.getAttributes()));
-            }
-        }
-
-        return entityHeaders;
-    }
-
-    /**
      * Bulk API to associate a tag to multiple entities
      *
      */
@@ -219,15 +210,4 @@ public class EntitiesREST {
             throw toAtlasBaseException(e);
         }
     }
-
-    private SearchFilter getSearchFilter() {
-        SearchFilter searchFilter = new SearchFilter();
-        if (null != httpServletRequest && null != httpServletRequest.getParameterMap()) {
-            for (Map.Entry<String, String[]> entry : httpServletRequest.getParameterMap().entrySet()) {
-                searchFilter.setParam(entry.getKey(), Arrays.asList(entry.getValue()));
-            }
-        }
-        return searchFilter;
-    }
-
 }