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/01 21:43:13 UTC

incubator-atlas git commit: ATLAS-1509: fixed issues with deletion during updates

Repository: incubator-atlas
Updated Branches:
  refs/heads/master aa74c73d0 -> 4367c4915


ATLAS-1509: fixed issues with deletion during updates

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 4367c491579e4de90770d95e4c746e289a6cf930
Parents: aa74c73
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Wed Feb 1 12:43:32 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Wed Feb 1 13:43:02 2017 -0800

----------------------------------------------------------------------
 .../test/java/org/apache/atlas/TestUtilsV2.java | 42 +++++++++---
 .../store/graph/v1/AtlasEntityStoreV1.java      |  5 +-
 .../store/graph/v1/DeleteHandlerV1.java         | 19 +++---
 .../store/graph/v1/EntityGraphMapper.java       | 28 ++++++--
 .../store/graph/v1/StructVertexMapper.java      | 19 +++++-
 .../store/graph/AtlasTypeDefGraphStoreTest.java |  1 -
 .../store/graph/v1/AtlasEntityStoreV1Test.java  | 72 +++++++++++++++-----
 .../atlas/services/MetricsServiceTest.java      |  3 +-
 .../java/org/apache/atlas/RequestContextV1.java | 25 ++++---
 .../adapters/AtlasEntityFormatConverter.java    | 28 ++++----
 .../adapters/AtlasStructFormatConverter.java    |  2 +-
 .../atlas/web/adapters/TestEntitiesREST.java    |  7 +-
 12 files changed, 170 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
index e0bf251..0756937 100755
--- a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
+++ b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
@@ -465,7 +465,17 @@ public final class TestUtilsV2 {
                 AtlasTypeUtil.createClassTypeDef(COLUMN_TYPE, COLUMN_TYPE + "_description",
                         ImmutableSet.<String>of(),
                         AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("type", "string")
+                        AtlasTypeUtil.createRequiredAttrDef("type", "string"),
+                    new AtlasAttributeDef("table", TABLE_TYPE,
+                        false,
+                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        false, false,
+                        new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
+                            add(new AtlasStructDef.AtlasConstraintDef(
+                                AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_FOREIGN_KEY, new HashMap<String, Object>() {{
+                                put(AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_ON_DELETE, AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_VAL_CASCADE);
+                            }}));
+                        }})
                         );
 
         AtlasStructDef partitionDefinition = new AtlasStructDef("partition_struct_type", "partition_struct_type" + _description, "1.0",
@@ -564,7 +574,13 @@ public final class TestUtilsV2 {
                                 true,
                                 AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
                                 false, false,
-                                Collections.<AtlasConstraintDef>emptyList()),
+                                new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
+                                    add(new AtlasStructDef.AtlasConstraintDef(
+                                            AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_MAPPED_FROM_REF,
+                                            new HashMap<String, Object>() {{
+                                                put(AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_REF_ATTRIBUTE, "table");
+                                        }}));
+                                }}),
                         // array of structs
                         new AtlasAttributeDef("partitions", String.format("array<%s>", "partition_struct_type"),
                                 true,
@@ -583,12 +599,19 @@ public final class TestUtilsV2 {
                                 true,
                                 AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
                                 false, false,
-                                Collections.<AtlasConstraintDef>emptyList()),
-
-                      // new ArrayList<AtlasConstraintDef>() {{
-                     //add(new AtlasConstraintDef(
-                       // AtlasConstraintDef.CONSTRAINT_TYPE_MAPPED_FROM_REF, new HashMap<String, Object>()));
-                       //}}),
+                                Collections.<AtlasStructDef.AtlasConstraintDef>emptyList()
+                                /* TODO - Fix map validation in type store and enable this
+                                 *
+                                new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
+                                    add(new AtlasStructDef.AtlasConstraintDef(
+                                            AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_MAPPED_FROM_REF,
+                                            new HashMap<String, Object>() {{
+                                                put(AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_REF_ATTRIBUTE, "table");
+                                        }}));
+                                    }}
+                                  *
+                                  */
+                                ),
                         //map of structs
                         new AtlasAttributeDef("partitionsMap",
                                 String.format("map<%s,%s>", "string", "partition_struct_type"),
@@ -668,10 +691,11 @@ public final class TestUtilsV2 {
         return entity;
     }
 
-    public static AtlasEntity createColumnEntity() {
+    public static AtlasEntity createColumnEntity(String tableId) {
         AtlasEntity entity = new AtlasEntity(COLUMN_TYPE);
         entity.setAttribute(NAME, RandomStringUtils.randomAlphanumeric(10));
         entity.setAttribute("type", "VARCHAR(32)");
+        entity.setAttribute("table", new AtlasObjectId(TABLE_TYPE, tableId));
         return entity;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/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 48bb77e..1590aee 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
@@ -37,7 +37,6 @@ import org.apache.atlas.repository.store.graph.AtlasEntityStore;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscoveryContext;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscovery;
 import org.apache.atlas.type.AtlasEntityType;
-import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -190,7 +189,7 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
                 context.addUpdated(entity, entityType, vertex);
 
                 String guid = AtlasGraphUtilsV1.getIdFromVertex(vertex);
-                RequestContextV1.get().recordEntityUpdate(guid);
+                RequestContextV1.get().recordEntityUpdate(new AtlasObjectId(entityType.getTypeName(), guid));
             } else {
                 //Create vertices which do not exist in the repository
                 vertex = graphMapper.createVertexTemplate(entity, entityType);
@@ -198,7 +197,7 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
                 discoveredEntities.addRepositoryResolvedReference(new AtlasObjectId(entityType.getTypeName(), entity.getGuid()), vertex);
 
                 String guid = AtlasGraphUtilsV1.getIdFromVertex(vertex);
-                RequestContextV1.get().recordEntityCreate(guid);
+                RequestContextV1.get().recordEntityCreate(new AtlasObjectId(entityType.getTypeName(), guid));
             }
 
             if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/DeleteHandlerV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/DeleteHandlerV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/DeleteHandlerV1.java
index 4a0eb20..08361ea 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/DeleteHandlerV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/DeleteHandlerV1.java
@@ -24,6 +24,7 @@ import org.apache.atlas.RequestContextV1;
 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.AtlasAttributeDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.AtlasEdgeLabel;
@@ -96,7 +97,7 @@ public abstract class DeleteHandlerV1 {
             // Record all deletion candidate GUIDs in RequestContext
             // and gather deletion candidate vertices.
             for (GraphHelper.VertexInfo vertexInfo : compositeVertices) {
-                requestContext.recordEntityDelete(vertexInfo.getGuid());
+                requestContext.recordEntityDelete(new AtlasObjectId(vertexInfo.getTypeName(), vertexInfo.getGuid()));
                 deletionCandidateVertices.add(vertexInfo.getVertex());
             }
         }
@@ -206,9 +207,9 @@ public abstract class DeleteHandlerV1 {
         boolean forceDeleteStructTrait) throws AtlasBaseException {
         LOG.debug("Deleting {}", string(edge));
         boolean forceDelete =
-            (AtlasGraphUtilsV1.isReference(typeCategory))
-                ? forceDeleteStructTrait : false;
-        if (AtlasGraphUtilsV1.isReference(typeCategory) && isComposite) {
+            (typeCategory == TypeCategory.STRUCT || typeCategory == TypeCategory.CLASSIFICATION) && forceDeleteStructTrait;
+        if (typeCategory == TypeCategory.STRUCT || typeCategory == TypeCategory.CLASSIFICATION
+            || (typeCategory == TypeCategory.ENTITY && isComposite)) {
             //If the vertex is of type struct/trait, delete the edge and then the reference vertex as the vertex is not shared by any other entities.
             //If the vertex is of type class, and its composite attribute, this reference vertex' lifecycle is controlled
             //through this delete, hence delete the edge and the reference vertex.
@@ -237,14 +238,14 @@ public abstract class DeleteHandlerV1 {
 
             if (parentType instanceof AtlasStructType) {
                 AtlasStructType parentStructType = (AtlasStructType) parentType;
-                if (parentStructType.isForeignKeyAttribute(atlasEdgeLabel.getAttributeName())) {
-                    deleteEdgeBetweenVertices(edge.getInVertex(), edge.getOutVertex(), atlasEdgeLabel.getAttributeName());
-                }
+                //TODO - Fix this later
+//                if (parentStructType.isForeignKeyAttribute(atlasEdgeLabel.getAttributeName())) {
+//                    deleteEdgeBetweenVertices(edge.getInVertex(), edge.getOutVertex(), atlasEdgeLabel.getAttributeName());
+//                }
             }
         }
 
         deleteEdge(edge, force);
-
     }
 
 
@@ -508,7 +509,7 @@ public abstract class DeleteHandlerV1 {
             GraphHelper.setProperty(outVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY,
                 requestContext.getRequestTime());
             GraphHelper.setProperty(outVertex, Constants.MODIFIED_BY_KEY, requestContext.getUser());
-            requestContext.recordEntityUpdate(outId);
+            requestContext.recordEntityUpdate(new AtlasObjectId(typeName, outId));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/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 f88661a..e534d4f 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
@@ -20,6 +20,7 @@ package org.apache.atlas.repository.store.graph.v1;
 
 import com.google.inject.Inject;
 import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.TypeCategory;
 import org.apache.atlas.model.instance.AtlasEntity;
@@ -54,8 +55,8 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
     protected final StructVertexMapper structVertexMapper;
 
     @Inject
-    public EntityGraphMapper(ArrayVertexMapper arrayVertexMapper, MapVertexMapper mapVertexMapper) {
-        this.structVertexMapper = new StructVertexMapper(arrayVertexMapper, mapVertexMapper);
+    public EntityGraphMapper(ArrayVertexMapper arrayVertexMapper, MapVertexMapper mapVertexMapper, DeleteHandlerV1 deleteHandler) {
+        this.structVertexMapper = new StructVertexMapper(arrayVertexMapper, mapVertexMapper, deleteHandler);
         arrayVertexMapper.init(structVertexMapper);
         mapVertexMapper.init(structVertexMapper);
     }
@@ -90,8 +91,7 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
         String guid = getId(ctx.getValue());
         AtlasVertex entityVertex = context.getDiscoveryContext().getResolvedReference(guid);
         if ( ctx.getCurrentEdge().isPresent() ) {
-            updateEdge(ctx.getAttributeDef(), ctx.getValue(), ctx.getCurrentEdge().get(), entityVertex);
-            result = ctx.getCurrentEdge().get();
+            result = updateEdge(ctx.getAttributeDef(), ctx.getValue(), ctx.getCurrentEdge().get(), entityVertex);
         } else if (ctx.getValue() != null) {
             String edgeLabel = AtlasGraphUtilsV1.getEdgeLabel(ctx.getVertexPropertyKey());
             try {
@@ -115,13 +115,14 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
 
         AtlasVertex currentVertex = currentEdge.getInVertex();
         String currentEntityId = AtlasGraphUtilsV1.getIdFromVertex(currentVertex);
-        String newEntityId = getId(value);
+        String newEntityId = AtlasGraphUtilsV1.getIdFromVertex(entityVertex);
+
         AtlasEdge newEdge = currentEdge;
         if (!currentEntityId.equals(newEntityId)) {
             // add an edge to the class vertex from the instance
             if (entityVertex != null) {
                 try {
-                    newEdge = graphHelper.getOrCreateEdge(currentEdge.getInVertex(), entityVertex, currentEdge.getLabel());
+                    newEdge = graphHelper.getOrCreateEdge(currentEdge.getOutVertex(), entityVertex, currentEdge.getLabel());
                 } catch (RepositoryException e) {
                     throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, e);
                 }
@@ -131,7 +132,8 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
         return newEdge;
     }
 
-    public EntityMutationResponse mapAttributes(EntityMutationContext ctx) throws AtlasBaseException {
+    public EntityMutationResponse
+    mapAttributes(EntityMutationContext ctx) throws AtlasBaseException {
 
         this.context = ctx;
         structVertexMapper.init(this);
@@ -155,6 +157,11 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
             }
         }
 
+        RequestContextV1 req = RequestContextV1.get();
+        for (AtlasObjectId id : req.getDeletedEntityIds()) {
+            resp.addEntity(EntityMutations.EntityOperation.DELETE, constructHeader(id));
+        }
+
         return resp;
     }
 
@@ -192,6 +199,13 @@ public class EntityGraphMapper implements InstanceGraphMapper<AtlasEdge> {
         return header;
     }
 
+    private AtlasEntityHeader constructHeader(AtlasObjectId id) {
+        AtlasEntityHeader entity = new AtlasEntityHeader(id.getTypeName());
+        entity.setGuid(id.getGuid());
+
+        return entity;
+    }
+
     public EntityMutationContext getContext() {
         return context;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/StructVertexMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/StructVertexMapper.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/StructVertexMapper.java
index ff1a702..2e9e5e4 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/StructVertexMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/StructVertexMapper.java
@@ -53,12 +53,15 @@ public class StructVertexMapper implements InstanceGraphMapper<AtlasEdge> {
 
     private EntityGraphMapper entityVertexMapper;
 
+    private DeleteHandlerV1 deleteHandler;
+
     private static final Logger LOG = LoggerFactory.getLogger(StructVertexMapper.class);
 
-    public StructVertexMapper(ArrayVertexMapper arrayVertexMapper, MapVertexMapper mapVertexMapper) {
+    public StructVertexMapper(ArrayVertexMapper arrayVertexMapper, MapVertexMapper mapVertexMapper, DeleteHandlerV1 deleteHandler) {
         this.graph = AtlasGraphProvider.getGraphInstance();;
         this.mapVertexMapper = mapVertexMapper;
         this.arrVertexMapper = arrayVertexMapper;
+        this.deleteHandler = deleteHandler;
     }
 
     void init(final EntityGraphMapper entityVertexMapper) {
@@ -162,7 +165,12 @@ public class StructVertexMapper implements InstanceGraphMapper<AtlasEdge> {
             AtlasEdge currentEdge = graphHelper.getEdgeForLabel(ctx.getReferringVertex(), edgeLabel);
             Optional<AtlasEdge> edge = currentEdge != null ? Optional.of(currentEdge) : Optional.<AtlasEdge>absent();
             ctx.setExistingEdge(edge);
-            return toGraph(ctx);
+            AtlasEdge newEdge = toGraph(ctx);
+
+            if (currentEdge != null && !currentEdge.equals(newEdge)) {
+                deleteHandler.deleteEdgeReference(currentEdge, ctx.getAttrType().getTypeCategory(), false, true);
+            }
+            return newEdge;
         case ENTITY:
             edgeLabel = AtlasGraphUtilsV1.getEdgeLabel(ctx.getVertexPropertyKey());
             currentEdge = graphHelper.getEdgeForLabel(ctx.getReferringVertex(), edgeLabel);
@@ -170,7 +178,12 @@ public class StructVertexMapper implements InstanceGraphMapper<AtlasEdge> {
             edge = currentEdge != null ? Optional.of(currentEdge) : Optional.<AtlasEdge>absent();
             ctx.setElementType(instanceType);
             ctx.setExistingEdge(edge);
-            return entityVertexMapper.toGraph(ctx);
+            newEdge = entityVertexMapper.toGraph(ctx);
+
+            if (currentEdge != null && !currentEdge.equals(newEdge)) {
+                deleteHandler.deleteEdgeReference(currentEdge, ctx.getAttrType().getTypeCategory(), shouldManageChildReferences(ctx.getParentType(), ctx.getAttributeDef().getName()), true);
+            }
+            return newEdge;
         case MAP:
             return mapVertexMapper.toGraph(ctx);
         case ARRAY:

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStoreTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStoreTest.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStoreTest.java
index 25bae44..3b72032 100644
--- a/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStoreTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStoreTest.java
@@ -336,7 +336,6 @@ public class AtlasTypeDefGraphStoreTest {
     @Test(dependsOnMethods = "testGet")
     public void testCreateWithValidAttributes(){
         AtlasTypesDef hiveTypes = TestUtilsV2.defineHiveTypes();
-
         try {
             AtlasTypesDef createdTypes = typeDefStore.createTypesDef(hiveTypes);
             assertEquals(hiveTypes.getEnumDefs(), createdTypes.getEnumDefs(), "Data integrity issue while persisting");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/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 c902f81..31a619e 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
@@ -20,6 +20,7 @@ package org.apache.atlas.repository.store.graph.v1;
 import com.google.common.collect.ImmutableSet;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.TestUtilsV2;
 import org.apache.atlas.exception.AtlasBaseException;
@@ -51,6 +52,7 @@ import org.apache.atlas.typesystem.persistence.ReferenceableInstance;
 import org.apache.atlas.typesystem.persistence.StructInstance;
 import org.apache.atlas.typesystem.types.EnumValue;
 import org.apache.atlas.util.AtlasRepositoryConfiguration;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,6 +70,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.atlas.TestUtils.COLUMNS_ATTR_NAME;
+import static org.apache.atlas.TestUtils.TABLE_TYPE;
 import static org.apache.atlas.TestUtils.randomString;
 import static org.testng.Assert.assertEquals;
 
@@ -124,8 +128,10 @@ public class AtlasEntityStoreV1Test {
 
         EntityGraphDiscovery graphDiscovery = new AtlasEntityGraphDiscoveryV1(typeRegistry, entityResolvers);
 
-        entityStore = new AtlasEntityStoreV1(new EntityGraphMapper(arrVertexMapper, mapVertexMapper));
+        entityStore = new AtlasEntityStoreV1(new EntityGraphMapper(arrVertexMapper, mapVertexMapper, deleteHandler));
         entityStore.init(typeRegistry, graphDiscovery);
+
+        RequestContextV1.clear();
     }
 
     @Test
@@ -173,8 +179,7 @@ public class AtlasEntityStoreV1Test {
         
         validateMutationResponse(response, EntityMutations.EntityOperation.UPDATE, 5);
         AtlasEntityHeader deptEntity = response.getFirstEntityUpdated();
-        validateAttributes(deptEntity);
-
+        Assert.assertEquals(((List<AtlasEntity>)(((List<AtlasEntity>) deptEntity.getAttribute("employees")).get(1).getAttribute("subordinates"))).size(), 1);
 
         init();
         //add  entity back
@@ -186,28 +191,60 @@ public class AtlasEntityStoreV1Test {
 
         //test array of class with id
         final List<AtlasEntity> columns = new ArrayList<>();
-        Map<String, Object> values = new HashMap<>();
-        values.put(TestUtilsV2.NAME, "col1");
-        values.put("type", "type");
-        AtlasEntity col1 = new AtlasEntity(TestUtilsV2.COLUMN_TYPE, values);
+
+        AtlasEntity col1 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
+        col1.setAttribute(TestUtilsV2.NAME, "col1");
         columns.add(col1);
         AtlasEntity tableUpdated = new AtlasEntity(tableEntity);
         tableUpdated.setAttribute(TestUtilsV2.COLUMNS_ATTR_NAME, columns);
 
         init();
-        entityStore.createOrUpdate(col1);
+        response = entityStore.createOrUpdate(tableUpdated);
+        AtlasEntityHeader updatedTable = response.getFirstEntityUpdated();
+        validateAttributes(updatedTable);
+
+        //Complete update. Add  array elements - col3,col4
+        AtlasEntity col3 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
+        col1.setAttribute(TestUtilsV2.NAME, "col3");
+        columns.add(col3);
 
+        AtlasEntity col4 = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
+        col1.setAttribute(TestUtilsV2.NAME, "col4");
+        columns.add(col4);
+
+        tableUpdated.setAttribute(COLUMNS_ATTR_NAME, columns);
         init();
         response = entityStore.createOrUpdate(tableUpdated);
-        final AtlasEntityHeader updateTable = response.getFirstEntityUpdated();
-        validateAttributes(updateTable);
+        updatedTable = response.getFirstEntityUpdated();
+        validateAttributes(updatedTable);
+
+        //Swap elements
+        columns.clear();
+        columns.add(col4);
+        columns.add(col3);
+
+        tableUpdated.setAttribute(COLUMNS_ATTR_NAME, columns);
+        init();
+        response = entityStore.createOrUpdate(tableUpdated);
+        updatedTable = response.getFirstEntityUpdated();
+        Assert.assertEquals(((List<AtlasEntity>) 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);
+        assertEquals(response.getEntitiesByOperation(EntityMutations.EntityOperation.DELETE).size(), 2);
 
     }
     
     @Test(dependsOnMethods = "testCreate")
     public void testUpdateEntityWithMap() throws Exception {
 
-        AtlasEntity tableClone = new AtlasEntity(tableEntity);
+        final AtlasEntity tableClone = new AtlasEntity(tableEntity);
         final Map<String, AtlasStruct> partsMap = new HashMap<>();
         partsMap.put("part0", new AtlasStruct(TestUtils.PARTITION_STRUCT_TYPE,
             new HashMap<String, Object>() {{
@@ -219,7 +256,7 @@ public class AtlasEntityStoreV1Test {
 
         init();
         EntityMutationResponse response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition1 = response.getFirstEntityUpdated();
+        final AtlasEntityHeader tableDefinition1 = response.getFirstEntityUpdated();
         validateAttributes(tableDefinition1);
                 
         Assert.assertTrue(partsMap.get("part0").equals(((Map<String, AtlasStruct>) tableDefinition1.getAttribute("partitionsMap")).get("part0")));
@@ -261,7 +298,7 @@ public class AtlasEntityStoreV1Test {
         AtlasStruct partition2 = partsMap.get("part2");
         partition2.setAttribute(TestUtilsV2.NAME, "test2Updated");
         response = entityStore.createOrUpdate(tableClone);
-        AtlasEntityHeader tableDefinition4 = response.getFirstEntityUpdated();
+        final AtlasEntityHeader tableDefinition4 = response.getFirstEntityUpdated();
         validateAttributes(tableDefinition4);
 
         assertEquals(((Map<String, AtlasStruct>) tableDefinition4.getAttribute("partitionsMap")).size(), 2);
@@ -278,9 +315,8 @@ public class AtlasEntityStoreV1Test {
             new HashMap<String, Object>() {{
                 put(TestUtilsV2.NAME, "test1");
                 put("type", "string");
+                put("table", new AtlasObjectId(TABLE_TYPE, tableDefinition1.getGuid()));
             }});
-
-
         init();
         entityStore.createOrUpdate(col0Type);
 
@@ -288,6 +324,7 @@ public class AtlasEntityStoreV1Test {
             new HashMap<String, Object>() {{
                 put(TestUtilsV2.NAME, "test2");
                 put("type", "string");
+                put("table", new AtlasObjectId(TABLE_TYPE, tableDefinition1.getGuid()));
             }});
 
         init();
@@ -546,8 +583,8 @@ public class AtlasEntityStoreV1Test {
             List actualList = (List) actual;
             List expectedList = (List) expected;
 
-            if (!(expected == null && actualList.size() == 0)) {
-                Assert.assertEquals(actualList.size(), expectedList.size());
+            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);
                 }
@@ -679,5 +716,4 @@ public class AtlasEntityStoreV1Test {
         entityStore.createOrUpdate(tableEntity);
         Assert.fail("Expected exception while creating with required attribute null");
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/repository/src/test/java/org/apache/atlas/services/MetricsServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/services/MetricsServiceTest.java b/repository/src/test/java/org/apache/atlas/services/MetricsServiceTest.java
index bb8c223..fef6b2f 100644
--- a/repository/src/test/java/org/apache/atlas/services/MetricsServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/services/MetricsServiceTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas.services;
 
+import org.apache.atlas.AtlasException;
 import org.apache.atlas.model.metrics.AtlasMetrics;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.type.AtlasTypeRegistry;
@@ -49,7 +50,7 @@ public class MetricsServiceTest {
     private Number mockCount = 10;
 
     @BeforeClass
-    public void init() throws ScriptException {
+    public void init() throws ScriptException, AtlasException {
         Map<String, Object> aMockMap = new HashMap<>();
         Map<String, Object> bMockMap = new HashMap<>();
         Map<String, Object> cMockMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/server-api/src/main/java/org/apache/atlas/RequestContextV1.java
----------------------------------------------------------------------
diff --git a/server-api/src/main/java/org/apache/atlas/RequestContextV1.java b/server-api/src/main/java/org/apache/atlas/RequestContextV1.java
index 1fc11a2..23eb4ce 100644
--- a/server-api/src/main/java/org/apache/atlas/RequestContextV1.java
+++ b/server-api/src/main/java/org/apache/atlas/RequestContextV1.java
@@ -21,6 +21,7 @@ package org.apache.atlas;
 import org.apache.atlas.metrics.Metrics;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasObjectId;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.typesystem.types.ClassType;
@@ -39,9 +40,9 @@ public class RequestContextV1 {
 
     private static final ThreadLocal<RequestContextV1> CURRENT_CONTEXT = new ThreadLocal<>();
 
-    private Set<String> createdEntityIds = new LinkedHashSet<>();
-    private Set<String> updatedEntityIds = new LinkedHashSet<>();
-    private Set<String> deletedEntityIds = new LinkedHashSet<>();
+    private Set<AtlasObjectId> createdEntityIds = new LinkedHashSet<>();
+    private Set<AtlasObjectId> updatedEntityIds = new LinkedHashSet<>();
+    private Set<AtlasObjectId> deletedEntityIds = new LinkedHashSet<>();
 
     private String user;
     private final long requestTime;
@@ -77,34 +78,36 @@ public class RequestContextV1 {
         this.user = user;
     }
 
-    public void recordEntityCreate(Collection<String> createdEntityIds) {
+    public void recordEntityCreate(Collection<AtlasObjectId> createdEntityIds) {
         this.createdEntityIds.addAll(createdEntityIds);
     }
 
-    public void recordEntityCreate(String createdEntityId) {
+    public void recordEntityCreate(AtlasObjectId createdEntityId) {
         this.createdEntityIds.add(createdEntityId);
     }
 
-    public void recordEntityUpdate(Collection<String> updatedEntityIds) {
+    public void recordEntityUpdate(Collection<AtlasObjectId> updatedEntityIds) {
         this.updatedEntityIds.addAll(updatedEntityIds);
     }
 
-    public void recordEntityUpdate(String entityId) {
+    public void recordEntityUpdate(AtlasObjectId entityId) {
         this.updatedEntityIds.add(entityId);
     }
-    public void recordEntityDelete(String entityId) {
+
+
+    public void recordEntityDelete(AtlasObjectId entityId) {
         deletedEntityIds.add(entityId);
     }
 
-    public Collection<String> getCreatedEntityIds() {
+    public Collection<AtlasObjectId> getCreatedEntityIds() {
         return createdEntityIds;
     }
 
-    public Collection<String> getUpdatedEntityIds() {
+    public Collection<AtlasObjectId> getUpdatedEntityIds() {
         return updatedEntityIds;
     }
 
-    public Collection<String> getDeletedEntityIds() {
+    public Collection<AtlasObjectId> getDeletedEntityIds() {
         return deletedEntityIds;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/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 75a7183..bbfcc59 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,8 +53,8 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
     }
 
     @Override
-    public AtlasEntityWithAssociations fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
-        AtlasEntityWithAssociations ret = null;
+    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+        Object ret = null;
 
         if (v1Obj != null) {
             AtlasEntityType entityType = (AtlasEntityType) type;
@@ -62,10 +62,7 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
             if (v1Obj instanceof Id) {
                 Id id = (Id) v1Obj;
 
-                ret = new AtlasEntityWithAssociations(id.getTypeName());
-                ret.setGuid(id.getId()._getId());
-                EntityState state = id.getState();
-                ret.setStatus(convertState(state));
+                ret = new AtlasObjectId(id.getTypeName(), id._getId());
             } else if (v1Obj instanceof IReferenceableInstance) {
                 IReferenceableInstance entity    = (IReferenceableInstance) v1Obj;
                 Map<String, Object>    v1Attribs = null;
@@ -76,15 +73,15 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
                     LOG.error("IReferenceableInstance.getValuesMap() failed", excp);
                 }
 
-                ret =  new AtlasEntityWithAssociations(entity.getTypeName(), super.fromV1ToV2(entityType, v1Attribs));
-                ret.setGuid(entity.getId()._getId());
-                ret.setStatus(convertState(entity.getId().getState()));
+                AtlasEntityWithAssociations ret1 =  new AtlasEntityWithAssociations(entity.getTypeName(), super.fromV1ToV2(entityType, v1Attribs));
+                ret1.setGuid(entity.getId()._getId());
+                ret1.setStatus(convertState(entity.getId().getState()));
                 AtlasSystemAttributes systemAttributes = entity.getSystemAttributes();
-                ret.setCreatedBy(systemAttributes.createdBy);
-                ret.setCreateTime(systemAttributes.createdTime);
-                ret.setUpdatedBy(systemAttributes.modifiedBy);
-                ret.setUpdateTime(systemAttributes.modifiedTime);
-                ret.setVersion(new Long(entity.getId().version));
+                ret1.setCreatedBy(systemAttributes.createdBy);
+                ret1.setCreateTime(systemAttributes.createdTime);
+                ret1.setUpdatedBy(systemAttributes.modifiedBy);
+                ret1.setUpdateTime(systemAttributes.modifiedTime);
+                ret1.setVersion(new Long(entity.getId().version));
 
                 if (CollectionUtils.isNotEmpty(entity.getTraits())) {
                     List<AtlasClassification> classifications = new ArrayList<>();
@@ -98,8 +95,9 @@ public class AtlasEntityFormatConverter extends AtlasStructFormatConverter {
                         classifications.add(classification);
                     }
 
-                    ret.setClassifications(classifications);
+                    ret1.setClassifications(classifications);
                 }
+                ret = ret1;
             } else {
                 throw new AtlasBaseException(AtlasErrorCode.UNEXPECTED_TYPE, "Id or IReferenceableInstance",
                                              v1Obj.getClass().getCanonicalName());

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/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 8a5c9be..4b3dd4c 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 AtlasStruct fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
+    public Object fromV1ToV2(Object v1Obj, AtlasType type) throws AtlasBaseException {
         AtlasStruct ret = null;
 
         if (v1Obj != null) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/4367c491/webapp/src/test/java/org/apache/atlas/web/adapters/TestEntitiesREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/adapters/TestEntitiesREST.java b/webapp/src/test/java/org/apache/atlas/web/adapters/TestEntitiesREST.java
index 47e1fa3..cfe9909 100644
--- a/webapp/src/test/java/org/apache/atlas/web/adapters/TestEntitiesREST.java
+++ b/webapp/src/test/java/org/apache/atlas/web/adapters/TestEntitiesREST.java
@@ -81,7 +81,7 @@ public class TestEntitiesREST {
         dbEntity = TestUtilsV2.createDBEntity();
 
         tableEntity = TestUtilsV2.createTableEntity(dbEntity.getGuid());
-        final AtlasEntity colEntity = TestUtilsV2.createColumnEntity();
+        final AtlasEntity colEntity = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
         columns = new ArrayList<AtlasEntity>() {{ add(colEntity); }};
         tableEntity.setAttribute("columns", columns);
     }
@@ -132,7 +132,7 @@ public class TestEntitiesREST {
         AtlasEntity dbEntity = TestUtilsV2.createDBEntity();
 
         AtlasEntity tableEntity = TestUtilsV2.createTableEntity(dbEntity.getGuid());
-        final AtlasEntity colEntity = TestUtilsV2.createColumnEntity();
+        final AtlasEntity colEntity = TestUtilsV2.createColumnEntity(tableEntity.getGuid());
         List<AtlasEntity> columns = new ArrayList<AtlasEntity>() {{ add(colEntity); }};
         tableEntity.setAttribute("columns", columns);
 
@@ -195,7 +195,8 @@ public class TestEntitiesREST {
 
         if ( retrievedColumnEntity != null) {
             LOG.info("verifying entity of type {} ", columns.get(0).getTypeName());
-            verifyAttributes(retrievedColumnEntity.getAttributes(), columns.get(0).getAttributes());
+            Assert.assertEquals(columns.get(0).getAttribute(AtlasClient.NAME), retrievedColumnEntity.getAttribute(AtlasClient.NAME));
+            Assert.assertEquals(columns.get(0).getAttribute("type"), retrievedColumnEntity.getAttribute("type"));
         }
 
         if ( retrievedTableEntity != null) {