You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sa...@apache.org on 2018/05/10 01:06:48 UTC

[1/3] atlas git commit: ATLAS-2662: Remove complex array and map attribute's edge information from entity vertex

Repository: atlas
Updated Branches:
  refs/heads/master a8fab3e77 -> 6e7aa6ed3


http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/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 0913e6f..8e0b2ed 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
@@ -19,15 +19,12 @@ package org.apache.atlas.repository.store.graph.v1;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.atlas.AtlasErrorCode;
-import org.apache.atlas.AtlasException;
 import org.apache.atlas.TestModules;
-import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.TestUtilsV2;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
-import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityExtInfo;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.atlas.model.instance.AtlasObjectId;
@@ -37,63 +34,35 @@ import org.apache.atlas.model.instance.EntityMutations;
 import org.apache.atlas.model.instance.EntityMutations.EntityOperation;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
-import org.apache.atlas.model.typedef.AtlasStructDef;
-import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.model.typedef.AtlasTypesDef;
-import org.apache.atlas.repository.graph.AtlasGraphProvider;
-import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
-import org.apache.atlas.repository.store.bootstrap.AtlasTypeDefStoreInitializer;
-import org.apache.atlas.repository.store.graph.AtlasEntityStore;
-import org.apache.atlas.runner.LocalSolrRunner;
-import org.apache.atlas.store.AtlasTypeDefStore;
-import org.apache.atlas.type.AtlasArrayType;
-import org.apache.atlas.type.AtlasMapType;
-import org.apache.atlas.type.AtlasStructType;
-import org.apache.atlas.type.AtlasType;
-import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.MapUtils;
-import org.apache.commons.collections.Transformer;
-import org.apache.commons.configuration.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
-import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
 import javax.inject.Inject;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import static org.apache.atlas.TestUtilsV2.COLUMNS_ATTR_NAME;
 import static org.apache.atlas.TestUtilsV2.COLUMN_TYPE;
 import static org.apache.atlas.TestUtilsV2.NAME;
-import static org.apache.atlas.TestUtilsV2.randomString;
 import static org.apache.atlas.TestUtilsV2.TABLE_TYPE;
-import static org.apache.atlas.graph.GraphSandboxUtil.useLocalSolr;
 import static org.mockito.Mockito.mock;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
 @Guice(modules = TestModules.TestOnlyModule.class)
-public class AtlasEntityStoreV1Test {
-    private static final Logger LOG = LoggerFactory.getLogger(AtlasEntityStoreV1Test.class);
-
-    @Inject
-    AtlasTypeRegistry typeRegistry;
-
-    @Inject
-    AtlasTypeDefStore typeDefStore;
-
-    AtlasEntityStore entityStore;
-
-    @Inject
-    DeleteHandlerV1 deleteHandler;
-
+public class AtlasEntityStoreV1Test extends AtlasEntityTestBase {
     private AtlasEntitiesWithExtInfo deptEntity;
     private AtlasEntityWithExtInfo   dbEntity;
     private AtlasEntityWithExtInfo   tblEntity;
@@ -101,86 +70,55 @@ public class AtlasEntityStoreV1Test {
     private AtlasEntityWithExtInfo   primitiveEntity;
 
     AtlasEntityChangeNotifier mockChangeNotifier = mock(AtlasEntityChangeNotifier.class);
+
     @Inject
     private EntityGraphMapper graphMapper;
 
     @Inject
-    private Configuration configuration;
     private String dbEntityGuid;
     private String tblEntityGuid;
 
     @BeforeClass
     public void setUp() throws Exception {
-        RequestContextV1.clear();
-        RequestContextV1.get().setUser(TestUtilsV2.TEST_USER, null);
-
-        new GraphBackedSearchIndexer(typeRegistry);
+        super.setUp();
 
         AtlasTypesDef[] testTypesDefs = new AtlasTypesDef[] { TestUtilsV2.defineDeptEmployeeTypes(),
                                                               TestUtilsV2.defineHiveTypes(),
                                                               TestUtilsV2.defineTypeWithNestedCollectionAttributes(),
                                                             };
+        createTypesDef(testTypesDefs);
 
-        for (AtlasTypesDef typesDef : testTypesDefs) {
-            AtlasTypesDef typesToCreate = AtlasTypeDefStoreInitializer.getTypesToCreate(typesDef, typeRegistry);
-
-            if (!typesToCreate.isEmpty()) {
-                typeDefStore.createTypesDef(typesToCreate);
-            }
-        }
-
-        deptEntity = TestUtilsV2.createDeptEg2();
-        dbEntity   = TestUtilsV2.createDBEntityV2();
-        tblEntity  = TestUtilsV2.createTableEntityV2(dbEntity.getEntity());
-
+        deptEntity                 = TestUtilsV2.createDeptEg2();
+        dbEntity                   = TestUtilsV2.createDBEntityV2();
+        tblEntity                  = TestUtilsV2.createTableEntityV2(dbEntity.getEntity());
         nestedCollectionAttrEntity = TestUtilsV2.createNestedCollectionAttrEntity();
+        primitiveEntity            = TestUtilsV2.createprimitiveEntityV2();
+
+        AtlasTypesDef typesDef11         = new AtlasTypesDef();
+        List          primitiveEntityDef = new ArrayList<AtlasEntityDef>();
 
-        AtlasTypesDef typesDef11 = new  AtlasTypesDef();
-        List primitiveEntityDef = new ArrayList<AtlasEntityDef>();
         primitiveEntityDef.add(TestUtilsV2.createPrimitiveEntityDef());
         typesDef11.setEntityDefs(primitiveEntityDef);
-        typeDefStore.createTypesDef( typesDef11 );
 
-        primitiveEntity = TestUtilsV2.createprimitiveEntityV2();
-    }
-
-    @AfterClass
-    public void clear() throws Exception {
-        AtlasGraphProvider.cleanup();
-
-        if (useLocalSolr()) {
-            LocalSolrRunner.stop();
-        }
-    }
-
-    @BeforeTest
-    public void init() throws Exception {
-        entityStore = new AtlasEntityStoreV1(deleteHandler, typeRegistry, mockChangeNotifier, graphMapper);
-        RequestContextV1.clear();
-        RequestContextV1.get().setUser(TestUtilsV2.TEST_USER, null);
+        typeDefStore.createTypesDef(typesDef11);
     }
 
     @Test
     public void testDefaultValueForPrimitiveTypes() throws Exception  {
-
         init();
 
-        EntityMutationResponse response = entityStore.createOrUpdate(new AtlasEntityStream(primitiveEntity), false);
-        List<AtlasEntityHeader> entitiesCreatedResponse = response.getEntitiesByOperation(EntityOperation.CREATE);
-        final Map<EntityOperation, List<AtlasEntityHeader>> entitiesMutated = response.getMutatedEntities();
-        List<AtlasEntityHeader> entitiesCreatedwithdefault = entitiesMutated.get(EntityOperation.CREATE);
-
-        AtlasEntity entityCreated   = getEntityFromStore(entitiesCreatedResponse.get(0));
-
-
-        Map attributesMap = entityCreated.getAttributes();
-        String description = (String) attributesMap.get("description");
-        String check = (String) attributesMap.get("check");
-        String   sourceCode =  (String) attributesMap.get("sourcecode");
-        float   diskUsage =  (float) attributesMap.get("diskUsage");
-        boolean   isstoreUse =  (boolean) attributesMap.get("isstoreUse");
-        int cost = (int)attributesMap.get("Cost");
+        EntityMutationResponse  response                   = entityStore.createOrUpdate(new AtlasEntityStream(primitiveEntity), false);
+        List<AtlasEntityHeader> entitiesCreatedResponse    = response.getEntitiesByOperation(EntityOperation.CREATE);
+        List<AtlasEntityHeader> entitiesCreatedwithdefault = response.getMutatedEntities().get(EntityOperation.CREATE);
+        AtlasEntity             entityCreated              = getEntityFromStore(entitiesCreatedResponse.get(0));
 
+        Map     attributesMap = entityCreated.getAttributes();
+        String  description   = (String) attributesMap.get("description");
+        String  check         = (String) attributesMap.get("check");
+        String  sourceCode    = (String) attributesMap.get("sourcecode");
+        float   diskUsage     = (float) attributesMap.get("diskUsage");
+        boolean isstoreUse    = (boolean) attributesMap.get("isstoreUse");
+        int     cost          = (int) attributesMap.get("Cost");
 
         assertEquals(description,"test");
         assertEquals(check,"check");
@@ -192,20 +130,20 @@ public class AtlasEntityStoreV1Test {
         assertEquals(cost,30);
     }
 
-
     @Test
     public void testCreate() throws Exception {
         init();
-        EntityMutationResponse response = entityStore.createOrUpdate(new AtlasEntityStream(deptEntity), false);
 
+        EntityMutationResponse response = entityStore.createOrUpdate(new AtlasEntityStream(deptEntity), false);
         validateMutationResponse(response, EntityOperation.CREATE, 5);
+
         AtlasEntityHeader dept1 = response.getFirstCreatedEntityByTypeName(TestUtilsV2.DEPARTMENT_TYPE);
         validateEntity(deptEntity, getEntityFromStore(dept1), deptEntity.getEntities().get(0));
 
         final Map<EntityOperation, List<AtlasEntityHeader>> entitiesMutated = response.getMutatedEntities();
         List<AtlasEntityHeader> entitiesCreated = entitiesMutated.get(EntityOperation.CREATE);
 
-        Assert.assertTrue(entitiesCreated.size() >= deptEntity.getEntities().size());
+        assertTrue(entitiesCreated.size() >= deptEntity.getEntities().size());
 
         for (int i = 0; i < deptEntity.getEntities().size(); i++) {
             AtlasEntity expected = deptEntity.getEntities().get(i);
@@ -252,7 +190,6 @@ public class AtlasEntityStoreV1Test {
         List<AtlasObjectId>      columns      = new ArrayList<>();
         AtlasEntitiesWithExtInfo entitiesInfo = new AtlasEntitiesWithExtInfo(tableEntity);
 
-
         AtlasEntity col1 = TestUtilsV2.createColumnEntity(tableEntity);
         col1.setAttribute(TestUtilsV2.NAME, "col1");
 
@@ -947,10 +884,10 @@ public class AtlasEntityStoreV1Test {
 
     @Test(dependsOnMethods = "testCreate")
     public void associateMultipleTagsToOneEntity() throws AtlasBaseException {
-        final String TAG_NAME = "tag_xy";
-        final String TAG_NAME_2 = TAG_NAME + "_2";
-        final String TAG_ATTRIBUTE_NAME = "testAttribute";
-        final String TAG_ATTRIBUTE_VALUE = "test-string";
+        final String TAG_NAME              = "tag_xy";
+        final String TAG_NAME_2            = TAG_NAME + "_2";
+        final String TAG_ATTRIBUTE_NAME    = "testAttribute";
+        final String TAG_ATTRIBUTE_VALUE   = "test-string";
         final String TAG_ATTRIBUTE_VALUE_2 = TAG_ATTRIBUTE_VALUE + "-2";
 
         createTag(TAG_NAME, "string");
@@ -973,8 +910,8 @@ public class AtlasEntityStoreV1Test {
 
     @Test(dependsOnMethods = "testCreate")
     public void associateSameTagToMultipleEntities() throws AtlasBaseException {
-        final String TAG_NAME = "tagx";
-        final String TAG_ATTRIBUTE_NAME = "testAttribute";
+        final String TAG_NAME            = "tagx";
+        final String TAG_ATTRIBUTE_NAME  = "testAttribute";
         final String TAG_ATTRIBUTE_VALUE = "test-string";
 
         createTag(TAG_NAME, "string");
@@ -984,16 +921,16 @@ public class AtlasEntityStoreV1Test {
         entityStore.addClassifications(dbEntityGuid, addedClassifications);
         entityStore.addClassifications(tblEntityGuid, addedClassifications);
 
-        AtlasEntity dbEntityFromDb = getEntityFromStore(dbEntityGuid);
+        AtlasEntity dbEntityFromDb  = getEntityFromStore(dbEntityGuid);
         AtlasEntity tblEntityFromDb = getEntityFromStore(tblEntityGuid);
 
-        Set<String> actualDBClassifications = new HashSet<>(CollectionUtils.collect(dbEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getTypeName()));
+        Set<String> actualDBClassifications  = new HashSet<>(CollectionUtils.collect(dbEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getTypeName()));
         Set<String> actualTblClassifications = new HashSet<>(CollectionUtils.collect(tblEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getTypeName()));
 
         assertTrue(actualDBClassifications.contains(TAG_NAME));
         assertTrue(actualTblClassifications.contains(TAG_NAME));
 
-        Set<String> actualDBAssociatedEntityGuid = new HashSet<>(CollectionUtils.collect(dbEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getEntityGuid()));
+        Set<String> actualDBAssociatedEntityGuid  = new HashSet<>(CollectionUtils.collect(dbEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getEntityGuid()));
         Set<String> actualTblAssociatedEntityGuid = new HashSet<>(CollectionUtils.collect(tblEntityFromDb.getClassifications(), o -> ((AtlasClassification) o).getEntityGuid()));
 
         assertTrue(actualDBAssociatedEntityGuid.contains(dbEntityGuid));
@@ -1002,128 +939,4 @@ public class AtlasEntityStoreV1Test {
         entityStore.deleteClassifications(dbEntityGuid, Collections.singletonList(TAG_NAME));
         entityStore.deleteClassifications(tblEntityGuid, Collections.singletonList(TAG_NAME));
     }
-
-
-    private AtlasClassificationDef getTagWithName(AtlasTypesDef typesDef, String tagName, String attributeType) {
-        AtlasClassificationDef aTag = new AtlasClassificationDef(tagName);
-        AtlasAttributeDef attributeDef = new AtlasAttributeDef("testAttribute", attributeType, true,
-                AtlasAttributeDef.Cardinality.SINGLE, 0, 1, false, true, false,
-                Collections.emptyList());
-
-        aTag.addAttribute(attributeDef);
-        typesDef.setClassificationDefs(Arrays.asList(aTag));
-        return aTag;
-    }
-
-    private void createTag(String tagName, String attributeType) {
-        try {
-            AtlasTypesDef typesDef = new AtlasTypesDef();
-            getTagWithName(typesDef, tagName, attributeType);
-            typeDefStore.createTypesDef(typesDef);
-        } catch (AtlasBaseException e) {
-            fail("Tag creation should've succeeded");
-        }
-    }
-
-    private String randomStrWithReservedChars() {
-        return randomString() + "\"${}%";
-    }
-
-    private void validateMutationResponse(EntityMutationResponse response, EntityMutations.EntityOperation op, int expectedNumCreated) {
-        List<AtlasEntityHeader> entitiesCreated = response.getEntitiesByOperation(op);
-        Assert.assertNotNull(entitiesCreated);
-        Assert.assertEquals(entitiesCreated.size(), expectedNumCreated);
-    }
-
-    private void validateEntity(AtlasEntityExtInfo entityExtInfo, AtlasEntity actual) throws AtlasBaseException, AtlasException {
-        validateEntity(entityExtInfo, actual, entityExtInfo.getEntity(actual.getGuid()));
-    }
-
-    private void validateEntity(AtlasEntityExtInfo entityExtInfo, AtlasStruct actual, AtlasStruct expected) throws AtlasBaseException, AtlasException {
-        if (expected == null) {
-            Assert.assertNull(actual, "expected null instance. Found " + actual);
-
-            return;
-        }
-
-        Assert.assertNotNull(actual, "found null instance");
-
-        AtlasStructType entityType = (AtlasStructType) typeRegistry.getType(actual.getTypeName());
-        for (String attrName : expected.getAttributes().keySet()) {
-            Object expectedVal = expected.getAttribute(attrName);
-            Object actualVal   = actual.getAttribute(attrName);
-
-            AtlasType attrType = entityType.getAttributeType(attrName);
-            validateAttribute(entityExtInfo, actualVal, expectedVal, attrType, attrName);
-        }
-    }
-
-    private void validateAttribute(AtlasEntityExtInfo entityExtInfo, Object actual, Object expected, AtlasType attributeType, String attrName) throws AtlasBaseException, AtlasException {
-        switch(attributeType.getTypeCategory()) {
-            case OBJECT_ID_TYPE:
-                Assert.assertTrue(actual instanceof AtlasObjectId);
-                String guid = ((AtlasObjectId) actual).getGuid();
-                Assert.assertTrue(AtlasTypeUtil.isAssignedGuid(guid), "expected assigned guid. found " + guid);
-                break;
-
-            case PRIMITIVE:
-            case ENUM:
-                Assert.assertEquals(actual, expected);
-                break;
-
-            case MAP:
-                AtlasMapType mapType     = (AtlasMapType) attributeType;
-                AtlasType    valueType   = mapType.getValueType();
-                Map          actualMap   = (Map) actual;
-                Map          expectedMap = (Map) expected;
-
-                if (MapUtils.isNotEmpty(expectedMap)) {
-                    Assert.assertTrue(MapUtils.isNotEmpty(actualMap));
-
-                    // deleted entries are included in the attribute; hence use >=
-                    Assert.assertTrue(actualMap.size() >= expectedMap.size());
-
-                    for (Object key : expectedMap.keySet()) {
-                        validateAttribute(entityExtInfo, actualMap.get(key), expectedMap.get(key), valueType, attrName);
-                    }
-                }
-                break;
-
-            case ARRAY:
-                AtlasArrayType arrType      = (AtlasArrayType) attributeType;
-                AtlasType      elemType     = arrType.getElementType();
-                List           actualList   = (List) actual;
-                List           expectedList = (List) expected;
-
-                if (CollectionUtils.isNotEmpty(expectedList)) {
-                    Assert.assertTrue(CollectionUtils.isNotEmpty(actualList));
-
-                    //actual list could have deleted entities. Hence size may not match.
-                    Assert.assertTrue(actualList.size() >= expectedList.size());
-
-                    for (int i = 0; i < expectedList.size(); i++) {
-                        validateAttribute(entityExtInfo, actualList.get(i), expectedList.get(i), elemType, attrName);
-                    }
-                }
-                break;
-            case STRUCT:
-                AtlasStruct expectedStruct = (AtlasStruct) expected;
-                AtlasStruct actualStruct   = (AtlasStruct) actual;
-
-                validateEntity(entityExtInfo, actualStruct, expectedStruct);
-                break;
-            default:
-                Assert.fail("Unknown type category");
-        }
-    }
-
-    private AtlasEntity getEntityFromStore(AtlasEntityHeader header) throws AtlasBaseException {
-        return header != null ? getEntityFromStore(header.getGuid()) : null;
-    }
-
-    private AtlasEntity getEntityFromStore(String guid) throws AtlasBaseException {
-        AtlasEntityWithExtInfo entity = guid != null ? entityStore.getById(guid) : null;
-
-        return entity != null ? entity.getEntity() : null;
-    }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityTestBase.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityTestBase.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityTestBase.java
new file mode 100644
index 0000000..56dc5bd
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityTestBase.java
@@ -0,0 +1,243 @@
+/**
+ * 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.repository.store.graph.v1;
+
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.RequestContextV1;
+import org.apache.atlas.TestModules;
+import org.apache.atlas.TestUtilsV2;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityExtInfo;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+import org.apache.atlas.model.instance.EntityMutations.EntityOperation;
+import org.apache.atlas.model.typedef.AtlasClassificationDef;
+import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
+import org.apache.atlas.repository.store.bootstrap.AtlasTypeDefStoreInitializer;
+import org.apache.atlas.repository.store.graph.AtlasEntityStore;
+import org.apache.atlas.runner.LocalSolrRunner;
+import org.apache.atlas.store.AtlasTypeDefStore;
+import org.apache.atlas.type.AtlasArrayType;
+import org.apache.atlas.type.AtlasMapType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.type.AtlasTypeUtil;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Guice;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.atlas.TestUtilsV2.randomString;
+import static org.apache.atlas.graph.GraphSandboxUtil.useLocalSolr;
+import static org.mockito.Mockito.mock;
+import static org.testng.Assert.fail;
+
+@Guice(modules = TestModules.TestOnlyModule.class)
+public class AtlasEntityTestBase {
+    @Inject
+    AtlasTypeRegistry typeRegistry;
+
+    @Inject
+    AtlasTypeDefStore typeDefStore;
+
+    @Inject
+    AtlasEntityStore entityStore;
+
+    @Inject
+    DeleteHandlerV1 deleteHandler;
+
+    @Inject
+    private EntityGraphMapper graphMapper;
+
+    AtlasEntityChangeNotifier mockChangeNotifier = mock(AtlasEntityChangeNotifier.class);
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        RequestContextV1.clear();
+        RequestContextV1.get().setUser(TestUtilsV2.TEST_USER, null);
+
+        new GraphBackedSearchIndexer(typeRegistry);
+    }
+
+    @AfterClass
+    public void clear() throws Exception {
+        AtlasGraphProvider.cleanup();
+
+        if (useLocalSolr()) {
+            LocalSolrRunner.stop();
+        }
+    }
+
+    @BeforeTest
+    public void init() throws Exception {
+        entityStore = new AtlasEntityStoreV1(deleteHandler, typeRegistry, mockChangeNotifier, graphMapper);
+
+        RequestContextV1.clear();
+        RequestContextV1.get().setUser(TestUtilsV2.TEST_USER, null);
+    }
+
+    protected AtlasClassificationDef getTagWithName(AtlasTypesDef typesDef, String tagName, String attributeType) {
+        AtlasClassificationDef aTag = new AtlasClassificationDef(tagName);
+        AtlasAttributeDef attributeDef = new AtlasAttributeDef("testAttribute", attributeType, true,
+                AtlasAttributeDef.Cardinality.SINGLE, 0, 1, false, true, false,
+                Collections.emptyList());
+
+        aTag.addAttribute(attributeDef);
+        typesDef.setClassificationDefs(Arrays.asList(aTag));
+        return aTag;
+    }
+
+    protected void createTag(String tagName, String attributeType) {
+        try {
+            AtlasTypesDef typesDef = new AtlasTypesDef();
+            getTagWithName(typesDef, tagName, attributeType);
+            typeDefStore.createTypesDef(typesDef);
+        } catch (AtlasBaseException e) {
+            fail("Tag creation should've succeeded");
+        }
+    }
+
+    protected String randomStrWithReservedChars() {
+        return randomString() + "\"${}%";
+    }
+
+    protected void validateMutationResponse(EntityMutationResponse response, EntityOperation op, int expectedNumCreated) {
+        List<AtlasEntityHeader> entitiesCreated = response.getEntitiesByOperation(op);
+        Assert.assertNotNull(entitiesCreated);
+        Assert.assertEquals(entitiesCreated.size(), expectedNumCreated);
+    }
+
+    protected void validateEntity(AtlasEntityExtInfo entityExtInfo, AtlasEntity actual) throws AtlasBaseException, AtlasException {
+        validateEntity(entityExtInfo, actual, entityExtInfo.getEntity(actual.getGuid()));
+    }
+
+    protected void validateEntity(AtlasEntityExtInfo entityExtInfo, AtlasStruct actual, AtlasStruct expected) throws AtlasBaseException, AtlasException {
+        if (expected == null) {
+            Assert.assertNull(actual, "expected null instance. Found " + actual);
+
+            return;
+        }
+
+        Assert.assertNotNull(actual, "found null instance");
+
+        AtlasStructType entityType = (AtlasStructType) typeRegistry.getType(actual.getTypeName());
+        for (String attrName : expected.getAttributes().keySet()) {
+            Object expectedVal = expected.getAttribute(attrName);
+            Object actualVal   = actual.getAttribute(attrName);
+
+            AtlasType attrType = entityType.getAttributeType(attrName);
+            validateAttribute(entityExtInfo, actualVal, expectedVal, attrType, attrName);
+        }
+    }
+
+    protected void validateAttribute(AtlasEntityExtInfo entityExtInfo, Object actual, Object expected, AtlasType attributeType, String attrName) throws AtlasBaseException, AtlasException {
+        switch(attributeType.getTypeCategory()) {
+            case OBJECT_ID_TYPE:
+                Assert.assertTrue(actual instanceof AtlasObjectId);
+                String guid = ((AtlasObjectId) actual).getGuid();
+                Assert.assertTrue(AtlasTypeUtil.isAssignedGuid(guid), "expected assigned guid. found " + guid);
+                break;
+
+            case PRIMITIVE:
+            case ENUM:
+                Assert.assertEquals(actual, expected);
+                break;
+
+            case MAP:
+                AtlasMapType mapType     = (AtlasMapType) attributeType;
+                AtlasType    valueType   = mapType.getValueType();
+                Map          actualMap   = (Map) actual;
+                Map          expectedMap = (Map) expected;
+
+                if (MapUtils.isNotEmpty(expectedMap)) {
+                    Assert.assertTrue(MapUtils.isNotEmpty(actualMap));
+
+                    // deleted entries are included in the attribute; hence use >=
+                    Assert.assertTrue(actualMap.size() >= expectedMap.size());
+
+                    for (Object key : expectedMap.keySet()) {
+                        validateAttribute(entityExtInfo, actualMap.get(key), expectedMap.get(key), valueType, attrName);
+                    }
+                }
+                break;
+
+            case ARRAY:
+                AtlasArrayType arrType      = (AtlasArrayType) attributeType;
+                AtlasType      elemType     = arrType.getElementType();
+                List           actualList   = (List) actual;
+                List           expectedList = (List) expected;
+
+                if (CollectionUtils.isNotEmpty(expectedList)) {
+                    Assert.assertTrue(CollectionUtils.isNotEmpty(actualList));
+
+                    //actual list could have deleted entities. Hence size may not match.
+                    Assert.assertTrue(actualList.size() >= expectedList.size());
+
+                    for (int i = 0; i < expectedList.size(); i++) {
+                        validateAttribute(entityExtInfo, actualList.get(i), expectedList.get(i), elemType, attrName);
+                    }
+                }
+                break;
+            case STRUCT:
+                AtlasStruct expectedStruct = (AtlasStruct) expected;
+                AtlasStruct actualStruct   = (AtlasStruct) actual;
+
+                validateEntity(entityExtInfo, actualStruct, expectedStruct);
+                break;
+            default:
+                Assert.fail("Unknown type category");
+        }
+    }
+
+    protected AtlasEntity getEntityFromStore(AtlasEntityHeader header) throws AtlasBaseException {
+        return header != null ? getEntityFromStore(header.getGuid()) : null;
+    }
+
+    protected AtlasEntity getEntityFromStore(String guid) throws AtlasBaseException {
+        AtlasEntityWithExtInfo entity = guid != null ? entityStore.getById(guid) : null;
+
+        return entity != null ? entity.getEntity() : null;
+    }
+
+    protected void createTypesDef(AtlasTypesDef[] testTypesDefs) throws AtlasBaseException {
+        for (AtlasTypesDef typesDef : testTypesDefs) {
+            AtlasTypesDef typesToCreate = AtlasTypeDefStoreInitializer.getTypesToCreate(typesDef, typeRegistry);
+
+            if (!typesToCreate.isEmpty()) {
+                typeDefStore.createTypesDef(typesToCreate);
+            }
+        }
+    }
+}


[2/3] atlas git commit: ATLAS-2662: Remove complex array and map attribute's edge information from entity vertex

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/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 f28d771..d285e69 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
@@ -51,6 +51,11 @@ import org.slf4j.LoggerFactory;
 
 import java.util.*;
 
+import static org.apache.atlas.model.TypeCategory.ARRAY;
+import static org.apache.atlas.model.TypeCategory.CLASSIFICATION;
+import static org.apache.atlas.model.TypeCategory.MAP;
+import static org.apache.atlas.model.TypeCategory.OBJECT_ID_TYPE;
+import static org.apache.atlas.model.TypeCategory.STRUCT;
 import static org.apache.atlas.model.instance.AtlasEntity.Status.DELETED;
 import static org.apache.atlas.model.typedef.AtlasRelationshipDef.PropagateTags.ONE_TO_TWO;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_NAME_PROPERTY_KEY;
@@ -58,7 +63,6 @@ import static org.apache.atlas.repository.Constants.CLASSIFICATION_LABEL;
 import static org.apache.atlas.repository.Constants.PROPAGATED_TRAIT_NAMES_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.RELATIONSHIP_GUID_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.TRAIT_NAMES_PROPERTY_KEY;
-import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX;
 import static org.apache.atlas.repository.graph.GraphHelper.addToPropagatedTraitNames;
 import static org.apache.atlas.repository.graph.GraphHelper.getAllClassificationEdges;
 import static org.apache.atlas.repository.graph.GraphHelper.getAssociatedEntityVertex;
@@ -66,7 +70,9 @@ import static org.apache.atlas.repository.graph.GraphHelper.getClassificationEdg
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationEntityGuid;
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationName;
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationVertices;
+import static org.apache.atlas.repository.graph.GraphHelper.getCollectionElementsUsingRelationship;
 import static org.apache.atlas.repository.graph.GraphHelper.getGuid;
+import static org.apache.atlas.repository.graph.GraphHelper.getMapValuesUsingRelationship;
 import static org.apache.atlas.repository.graph.GraphHelper.getPropagatedClassificationEdge;
 import static org.apache.atlas.repository.graph.GraphHelper.getPropagatedEdges;
 import static org.apache.atlas.repository.graph.GraphHelper.getPropagationEnabledClassificationVertices;
@@ -77,7 +83,9 @@ import static org.apache.atlas.repository.graph.GraphHelper.isPropagatedClassifi
 import static org.apache.atlas.repository.graph.GraphHelper.string;
 import static org.apache.atlas.repository.graph.GraphHelper.updateModificationMetadata;
 import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getIdFromEdge;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getQualifiedAttributePropertyKey;
 import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getState;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.isReference;
 
 public abstract class DeleteHandlerV1 {
     public static final Logger LOG = LoggerFactory.getLogger(DeleteHandlerV1.class);
@@ -198,63 +206,43 @@ public abstract class DeleteHandlerV1 {
                     continue;
                 }
 
-                String    edgeLabel = AtlasGraphUtilsV1.getAttributeEdgeLabel(entityType, attributeInfo.getName());
-                AtlasType attrType  = attributeInfo.getAttributeType();
+                String       edgeLabel    = attributeInfo.getRelationshipEdgeLabel();
+                AtlasType    attrType     = attributeInfo.getAttributeType();
+                TypeCategory typeCategory = attrType.getTypeCategory();
 
-                switch (attrType.getTypeCategory()) {
-                    case OBJECT_ID_TYPE: {
-                        AtlasEdge edge = graphHelper.getEdgeForLabel(vertex, edgeLabel);
+                if (typeCategory == OBJECT_ID_TYPE) {
+                    AtlasEdge edge = graphHelper.getEdgeForLabel(vertex, edgeLabel);
 
-                        if (edge != null && getState(edge) == AtlasEntity.Status.ACTIVE) {
-                            vertices.push(edge.getInVertex());
-                        }
+                    if (edge == null || getState(edge) == DELETED) {
+                        continue;
                     }
-                    break;
-
-                    case ARRAY: {
-                        AtlasArrayType arrType = (AtlasArrayType) attrType;
-
-                        if (arrType.getElementType().getTypeCategory() != TypeCategory.OBJECT_ID_TYPE) {
-                            continue;
-                        }
 
-                        Iterator<AtlasEdge> edges = graphHelper.getOutGoingEdgesByLabel(vertex, edgeLabel);
+                    vertices.push(edge.getInVertex());
 
-                        if (edges != null) {
-                            while (edges.hasNext()) {
-                                AtlasEdge edge = edges.next();
+                } else if (typeCategory == ARRAY || typeCategory == MAP) {
+                    TypeCategory elementType = null;
 
-                                if (edge != null && getState(edge) == AtlasEntity.Status.ACTIVE) {
-                                    vertices.push(edge.getInVertex());
-                                }
-                            }
-                        }
+                    if (typeCategory == ARRAY) {
+                        elementType = ((AtlasArrayType) attrType).getElementType().getTypeCategory();
+                    } else if (typeCategory == MAP) {
+                        elementType = ((AtlasMapType) attrType).getValueType().getTypeCategory();
                     }
-                    break;
-
-                    case MAP: {
-                        AtlasMapType mapType           = (AtlasMapType) attrType;
-                        TypeCategory valueTypeCategory = mapType.getValueType().getTypeCategory();
-
-                        if (valueTypeCategory != TypeCategory.OBJECT_ID_TYPE) {
-                            continue;
-                        }
 
-                        String       propertyName = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(entityType, attributeInfo.getName());
-                        List<String> keys         = vertex.getProperty(propertyName, List.class);
+                    if (elementType != OBJECT_ID_TYPE) {
+                        continue;
+                    }
 
-                        if (keys != null) {
-                            for (String key : keys) {
-                                String    mapEdgeLabel = GraphHelper.getQualifiedNameForMapKey(edgeLabel, key);
-                                AtlasEdge edge         = graphHelper.getEdgeForLabel(vertex, mapEdgeLabel);
+                    List<AtlasEdge> edges = getCollectionElementsUsingRelationship(vertex, attributeInfo);
 
-                                if (edge != null && getState(edge) == AtlasEntity.Status.ACTIVE) {
-                                    vertices.push(edge.getInVertex());
-                                }
+                    if (CollectionUtils.isNotEmpty(edges)) {
+                        for (AtlasEdge edge : edges) {
+                            if (edge == null || getState(edge) == DELETED) {
+                                continue;
                             }
+
+                            vertices.push(edge.getInVertex());
                         }
                     }
-                    break;
                 }
             }
         }
@@ -284,14 +272,14 @@ public abstract class DeleteHandlerV1 {
         }
 
         boolean isInternalType = isInternalType(entityVertex);
-        boolean forceDelete = (typeCategory == TypeCategory.STRUCT || typeCategory == TypeCategory.CLASSIFICATION)
+        boolean forceDelete = (typeCategory == STRUCT || typeCategory == CLASSIFICATION)
                                       && (forceDeleteStructTrait || isInternalType);
 
         if (LOG.isDebugEnabled()) {
             LOG.debug("isInternal = {}, forceDelete = {}", isInternalType, forceDelete);
         }
 
-        if (typeCategory == TypeCategory.STRUCT || typeCategory == TypeCategory.CLASSIFICATION || (typeCategory == TypeCategory.OBJECT_ID_TYPE && isOwned)) {
+        if (typeCategory == STRUCT || typeCategory == CLASSIFICATION || (typeCategory == OBJECT_ID_TYPE && isOwned)) {
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Processing delete for typeCategory={}, isOwned={}", typeCategory, isOwned);
             }
@@ -681,7 +669,7 @@ public abstract class DeleteHandlerV1 {
 
                 boolean   isOwned   = isEntityType && attributeInfo.isOwnedRef();
                 AtlasType attrType  = attributeInfo.getAttributeType();
-                String    edgeLabel = AtlasGraphUtilsV1.getAttributeEdgeLabel(structType, attributeInfo.getName());
+                String    edgeLabel = attributeInfo.getRelationshipEdgeLabel();
 
                 switch (attrType.getTypeCategory()) {
                     case OBJECT_ID_TYPE:
@@ -699,13 +687,11 @@ public abstract class DeleteHandlerV1 {
                         AtlasArrayType arrType  = (AtlasArrayType) attrType;
                         AtlasType      elemType = arrType.getElementType();
 
-                        if (AtlasGraphUtilsV1.isReference(elemType.getTypeCategory())) {
-                            Iterator<AtlasEdge> edges = graphHelper.getOutGoingEdgesByLabel(instanceVertex, edgeLabel);
-
-                            if (edges != null) {
-                                while (edges.hasNext()) {
-                                    AtlasEdge edge = edges.next();
+                        if (isReference(elemType.getTypeCategory())) {
+                            List<AtlasEdge> edges = getCollectionElementsUsingRelationship(instanceVertex, attributeInfo);
 
+                            if (CollectionUtils.isNotEmpty(edges)) {
+                                for (AtlasEdge edge : edges) {
                                     deleteEdgeReference(edge, elemType.getTypeCategory(), isOwned, false, instanceVertex);
                                 }
                             }
@@ -715,19 +701,13 @@ public abstract class DeleteHandlerV1 {
                     case MAP:
                         //For map attribute, if the value type is struct/class, delete all the references
                         AtlasMapType mapType           = (AtlasMapType) attrType;
-                        AtlasType    keyType           = mapType.getKeyType();
                         TypeCategory valueTypeCategory = mapType.getValueType().getTypeCategory();
-                        String       propertyName      = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(structType, attributeInfo.getName());
-
-                        if (AtlasGraphUtilsV1.isReference(valueTypeCategory)) {
-                            List<Object> keys = EntityGraphMapper.getArrayElementsProperty(keyType, instanceVertex, propertyName);
 
-                            if (keys != null) {
-                                for (Object key : keys) {
-                                    String mapEdgeLabel = GraphHelper.getQualifiedNameForMapKey(edgeLabel, (String) key);
+                        if (isReference(valueTypeCategory)) {
+                            List<AtlasEdge> edges = getMapValuesUsingRelationship(instanceVertex, attributeInfo);
 
-                                    deleteEdgeReference(instanceVertex, mapEdgeLabel, valueTypeCategory, isOwned);
-                                }
+                            for (AtlasEdge edge : edges) {
+                                deleteEdgeReference(edge, valueTypeCategory, isOwned, false, instanceVertex);
                             }
                         }
                      break;
@@ -772,8 +752,8 @@ public abstract class DeleteHandlerV1 {
         }
 
         AtlasStructType   parentType   = (AtlasStructType) typeRegistry.getType(typeName);
-        String            propertyName = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(parentType, attribute.getName());
-        String            edgeLabel    = EDGE_LABEL_PREFIX + propertyName;
+        String            propertyName = getQualifiedAttributePropertyKey(parentType, attribute.getName());
+        String            edgeLabel    = attribute.getRelationshipEdgeLabel();
         AtlasEdge         edge         = null;
         AtlasAttributeDef attrDef      = attribute.getAttributeDef();
         AtlasType         attrType     = attribute.getAttributeType();
@@ -796,14 +776,12 @@ public abstract class DeleteHandlerV1 {
 
             case ARRAY: {
                 //If its array attribute, find the right edge between the two vertices and update array property
-                List<String> elements = GraphHelper.getListProperty(outVertex, propertyName);
-
-                if (elements != null) {
-                    elements = new ArrayList<>(elements);
+                List<AtlasEdge> elementEdges = getCollectionElementsUsingRelationship(outVertex, attribute);
 
-                    for (String elementEdgeId : elements) {
-                        AtlasEdge elementEdge = graphHelper.getEdgeByEdgeId(outVertex, edgeLabel, elementEdgeId);
+                if (elementEdges != null) {
+                    elementEdges = new ArrayList<>(elementEdges);
 
+                    for (AtlasEdge elementEdge : elementEdges) {
                         if (elementEdge == null) {
                             continue;
                         }
@@ -814,26 +792,10 @@ public abstract class DeleteHandlerV1 {
                             edge = elementEdge;
 
                             //TODO element.size includes deleted items as well. should exclude
-                            if (!attrDef.getIsOptional() && elements.size() <= attrDef.getValuesMinCount()) {
+                            if (!attrDef.getIsOptional() && elementEdges.size() <= attrDef.getValuesMinCount()) {
                                 // Deleting this edge would violate the attribute's lower bound.
                                 throw new AtlasBaseException("Cannot remove array element from required attribute " + propertyName + " on " + GraphHelper.getVertexDetails(outVertex) + " " + GraphHelper.getEdgeDetails(elementEdge));
                             }
-
-                            if (shouldUpdateInverseReferences) {
-                                //if composite attribute, remove the reference as well. else, just remove the edge
-                                //for example, when table is deleted, process still references the table
-                                //but when column is deleted, table will not reference the deleted column
-                                if (LOG.isDebugEnabled()) {
-                                    LOG.debug("Removing edge {} from the array attribute {}", string(elementEdge), attribute.getName());
-                                }
-
-                                // Remove all occurrences of the edge ID from the list.
-                                // This prevents dangling edge IDs (i.e. edge IDs for deleted edges)
-                                // from the remaining in the list if there are duplicates.
-                                elements.removeAll(Collections.singletonList(elementEdge.getId().toString()));
-                                GraphHelper.setProperty(outVertex, propertyName, elements);
-                                break;
-                            }
                         }
                     }
                 }
@@ -842,37 +804,22 @@ public abstract class DeleteHandlerV1 {
 
             case MAP: {
                 //If its map attribute, find the right edge between two vertices and update map property
-                List<String> keys = GraphHelper.getListProperty(outVertex, propertyName);
+                List<AtlasEdge> mapEdges = getMapValuesUsingRelationship(outVertex, attribute);
 
-                if (keys != null) {
-                    keys = new ArrayList<>(keys);
-
-                    for (String key : keys) {
-                        String    keyPropertyName = GraphHelper.getQualifiedNameForMapKey(propertyName, key);
-                        String    mapEdgeId       = GraphHelper.getSingleValuedProperty(outVertex, keyPropertyName, String.class);
-                        AtlasEdge mapEdge         = graphHelper.getEdgeByEdgeId(outVertex, keyPropertyName, mapEdgeId);
+                if (mapEdges != null) {
+                    mapEdges = new ArrayList<>(mapEdges);
 
+                    for (AtlasEdge mapEdge : mapEdges) {
                         if (mapEdge != null) {
                             AtlasVertex mapVertex = mapEdge.getInVertex();
 
                             if (mapVertex.getId().toString().equals(inVertex.getId().toString())) {
                                 //TODO keys.size includes deleted items as well. should exclude
-                                if (attrDef.getIsOptional() || keys.size() > attrDef.getValuesMinCount()) {
+                                if (attrDef.getIsOptional() || mapEdges.size() > attrDef.getValuesMinCount()) {
                                     edge = mapEdge;
                                 } else {
                                     // Deleting this entry would violate the attribute's lower bound.
-                                    throw new AtlasBaseException("Cannot remove map entry " + keyPropertyName + " from required attribute " + propertyName + " on " + GraphHelper.getVertexDetails(outVertex) + " " + GraphHelper.getEdgeDetails(mapEdge));
-                                }
-
-                                if (shouldUpdateInverseReferences) {
-                                    //remove this key
-                                    if (LOG.isDebugEnabled()) {
-                                        LOG.debug("Removing edge {}, key {} from the map attribute {}", string(mapEdge), key, attribute.getName());
-                                    }
-
-                                    keys.remove(key);
-                                    GraphHelper.setProperty(outVertex, propertyName, keys);
-                                    GraphHelper.setProperty(outVertex, keyPropertyName, null);
+                                    throw new AtlasBaseException("Cannot remove map entry " + propertyName + " from required attribute " + propertyName + " on " + GraphHelper.getVertexDetails(outVertex) + " " + GraphHelper.getEdgeDetails(mapEdge));
                                 }
                                 break;
                             }
@@ -968,7 +915,7 @@ public abstract class DeleteHandlerV1 {
 
                 removeTagPropagation(classificationVertex);
 
-                deleteEdgeReference(edge, TypeCategory.CLASSIFICATION, false, false, instanceVertex);
+                deleteEdgeReference(edge, CLASSIFICATION, false, false, instanceVertex);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/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 1728549..cd00639 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
@@ -56,7 +56,6 @@ import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.atlas.utils.AtlasJson;
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -68,26 +67,36 @@ import java.util.*;
 import java.util.stream.Collectors;
 
 import static org.apache.atlas.model.TypeCategory.CLASSIFICATION;
+import static org.apache.atlas.model.instance.AtlasEntity.Status.DELETED;
 import static org.apache.atlas.model.instance.AtlasRelatedObjectId.KEY_RELATIONSHIP_ATTRIBUTES;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.CREATE;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.DELETE;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.PARTIAL_UPDATE;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.UPDATE;
 import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality.SET;
+import static org.apache.atlas.repository.Constants.ATTRIBUTE_KEY_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_LABEL;
 import static org.apache.atlas.repository.Constants.STATE_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.TRAIT_NAMES_PROPERTY_KEY;
+import static org.apache.atlas.repository.Constants.ATTRIBUTE_INDEX_PROPERTY_KEY;
+import static org.apache.atlas.repository.graph.GraphHelper.getCollectionElementsUsingRelationship;
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationEdge;
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationVertex;
+import static org.apache.atlas.repository.graph.GraphHelper.getListProperty;
+import static org.apache.atlas.repository.graph.GraphHelper.getMapElementsProperty;
+import static org.apache.atlas.repository.graph.GraphHelper.getQualifiedNameForMapKey;
+import static org.apache.atlas.repository.graph.GraphHelper.getStatus;
 import static org.apache.atlas.repository.graph.GraphHelper.getTraitLabel;
 import static org.apache.atlas.repository.graph.GraphHelper.getTraitNames;
 import static org.apache.atlas.repository.graph.GraphHelper.getTypeName;
 import static org.apache.atlas.repository.graph.GraphHelper.getTypeNames;
 import static org.apache.atlas.repository.graph.GraphHelper.isPropagationEnabled;
 import static org.apache.atlas.repository.graph.GraphHelper.isRelationshipEdge;
+import static org.apache.atlas.repository.graph.GraphHelper.setListProperty;
 import static org.apache.atlas.repository.graph.GraphHelper.string;
 import static org.apache.atlas.repository.graph.GraphHelper.updateModificationMetadata;
 import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getIdFromVertex;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.isReference;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.IN;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.OUT;
 
@@ -794,64 +803,59 @@ public class EntityGraphMapper {
             LOG.debug("==> mapMapValue({})", ctx);
         }
 
-        @SuppressWarnings("unchecked")
-        Map<Object, Object> newVal  = (Map<Object, Object>) ctx.getValue();
-        Map<String, Object> newMap  = new HashMap<>();
-        AtlasMapType        mapType = (AtlasMapType) ctx.getAttrType();
+        Map<Object, Object> newVal      = (Map<Object, Object>) ctx.getValue();
+        Map<String, Object> newMap      = new HashMap<>();
+        AtlasMapType        mapType     = (AtlasMapType) ctx.getAttrType();
+        AtlasAttribute      attribute   = ctx.getAttribute();
+        Map<String, Object> currentMap  = getMapElementsProperty(mapType, ctx.getReferringVertex(), ctx.getVertexProperty(), attribute);
+        boolean             isReference = isReference(mapType.getValueType());
 
-        try {
-            AtlasAttribute      attribute   = ctx.getAttribute();
-            List<String> currentKeys = GraphHelper.getListProperty(ctx.getReferringVertex(), ctx.getVertexProperty());
-            Map<String, Object> currentMap  = new HashMap<>();
-
-            if (CollectionUtils.isNotEmpty(currentKeys)) {
-                for (String key : currentKeys) {
-                    String propertyNameForKey  = GraphHelper.getQualifiedNameForMapKey(ctx.getVertexProperty(), GraphHelper.encodePropertyKey(key));
-                    Object propertyValueForKey = getMapValueProperty(mapType.getValueType(), ctx.getReferringVertex(), propertyNameForKey);
+        if (MapUtils.isNotEmpty(newVal)) {
+            AtlasAttribute inverseRefAttribute = attribute.getInverseRefAttribute();
 
-                    currentMap.put(key, propertyValueForKey);
-                }
-            }
+            for (Map.Entry<Object, Object> entry : newVal.entrySet()) {
+                String    key          = entry.getKey().toString();
+                String    propertyName = (isReference) ? ctx.getVertexProperty() : getQualifiedNameForMapKey(ctx.getVertexProperty(), GraphHelper.encodePropertyKey(key));
+                AtlasEdge existingEdge = getEdgeIfExists(mapType, currentMap, key);
 
-            if (MapUtils.isNotEmpty(newVal)) {
-                boolean isReference = AtlasGraphUtilsV1.isReference(mapType.getValueType());
-                AtlasAttribute inverseRefAttribute = attribute.getInverseRefAttribute();
-                for (Map.Entry<Object, Object> entry : newVal.entrySet()) {
-                    String    key          = entry.getKey().toString();
-                    String    propertyName = GraphHelper.getQualifiedNameForMapKey(ctx.getVertexProperty(), GraphHelper.encodePropertyKey(key));
-                    AtlasEdge existingEdge = getEdgeIfExists(mapType, currentMap, key);
+                AttributeMutationContext mapCtx =  new AttributeMutationContext(ctx.getOp(), ctx.getReferringVertex(), attribute, entry.getValue(),
+                                                                                 propertyName, mapType.getValueType(), existingEdge);
 
-                    AttributeMutationContext mapCtx =  new AttributeMutationContext(ctx.getOp(), ctx.getReferringVertex(), attribute, entry.getValue(), propertyName, mapType.getValueType(), existingEdge);
+                //Add/Update/Remove property value
+                Object newEntry = mapCollectionElementsToVertex(mapCtx, context);
 
-                    //Add/Update/Remove property value
-                    Object newEntry = mapCollectionElementsToVertex(mapCtx, context);
-                    setMapValueProperty(mapType.getValueType(), ctx.getReferringVertex(), propertyName, newEntry);
+                if (isReference) {
+                    AtlasEdge edge = (AtlasEdge) newEntry;
+                    edge.setProperty(ATTRIBUTE_KEY_PROPERTY_KEY, key);
+                } else {
+                    ctx.getReferringVertex().setProperty(propertyName, newEntry);
+                }
 
-                    newMap.put(key, newEntry);
+                newMap.put(key, newEntry);
 
-                    // If value type indicates this attribute is a reference, and the attribute has an inverse reference attribute,
-                    // update the inverse reference value.
-                    if (isReference && newEntry instanceof AtlasEdge && inverseRefAttribute != null) {
-                        AtlasEdge newEdge = (AtlasEdge) newEntry;
+                // If value type indicates this attribute is a reference, and the attribute has an inverse reference attribute,
+                // update the inverse reference value.
+                if (isReference && newEntry instanceof AtlasEdge && inverseRefAttribute != null) {
+                    AtlasEdge newEdge = (AtlasEdge) newEntry;
 
-                        addInverseReference(context, inverseRefAttribute, newEdge, getRelationshipAttributes(ctx.getValue()));
-                    }
+                    addInverseReference(context, inverseRefAttribute, newEdge, getRelationshipAttributes(ctx.getValue()));
                 }
             }
+        }
 
-            Map<String, Object> finalMap = removeUnusedMapEntries(attribute, ctx.getReferringVertex(), ctx.getVertexProperty(), currentMap, newMap);
+        Map<String, Object> finalMap = removeUnusedMapEntries(attribute, ctx.getReferringVertex(), ctx.getVertexProperty(), currentMap, newMap);
 
-            for (Object newEntry : newMap.values()) {
-                updateInConsistentOwnedMapVertices(ctx, mapType, newEntry);
-            }
+        for (Object newEntry : newMap.values()) {
+            updateInConsistentOwnedMapVertices(ctx, mapType, newEntry);
+        }
 
+        // for dereference on way out for primitive map types
+        if (!isReference) {
             Set<String> newKeys = new LinkedHashSet<>(newMap.keySet());
+
             newKeys.addAll(finalMap.keySet());
 
-            // for dereference on way out
-            GraphHelper.setListProperty(ctx.getReferringVertex(), ctx.getVertexProperty(), new ArrayList<>(newKeys));
-        } catch (AtlasException e) {
-            throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, e);
+            setListProperty(ctx.getReferringVertex(), ctx.getVertexProperty(), new ArrayList<>(newKeys));
         }
 
         if (LOG.isDebugEnabled()) {
@@ -870,16 +874,16 @@ public class EntityGraphMapper {
         List           newElements         = (List) ctx.getValue();
         AtlasArrayType arrType             = (AtlasArrayType) attribute.getAttributeType();
         AtlasType      elementType         = arrType.getElementType();
-        boolean        isReference         = AtlasGraphUtilsV1.isReference(elementType);
+        boolean        isReference         = isReference(elementType);
         AtlasAttribute inverseRefAttribute = attribute.getInverseRefAttribute();
         Cardinality    cardinality         = attribute.getAttributeDef().getCardinality();
         List<Object>   newElementsCreated  = new ArrayList<>();
         List<Object>   currentElements;
 
-        if (isRelationshipAttribute(attribute)) {
-            currentElements = getArrayElementsUsingRelationship(ctx.getReferringVertex(), attribute, elementType);
+        if (isReference) {
+            currentElements = (List) getCollectionElementsUsingRelationship(ctx.getReferringVertex(), attribute);
         } else {
-            currentElements = getArrayElementsProperty(elementType, ctx.getReferringVertex(), ctx.getVertexProperty());
+            currentElements = (List) getArrayElementsProperty(elementType, ctx.getReferringVertex(), ctx.getVertexProperty());
         }
 
         if (CollectionUtils.isNotEmpty(newElements)) {
@@ -912,6 +916,15 @@ public class EntityGraphMapper {
             newElementsCreated.addAll(additionalEdges);
         }
 
+        // add index to attributes of array type
+       for (int index = 0; index < newElementsCreated.size(); index++) {
+           Object element = newElementsCreated.get(index);
+
+           if (element instanceof AtlasEdge) {
+               AtlasGraphUtilsV1.setProperty((AtlasEdge) element, ATTRIBUTE_INDEX_PROPERTY_KEY, index);
+            }
+        }
+
         // for dereference on way out
         setArrayElementsProperty(elementType, ctx.getReferringVertex(), ctx.getVertexProperty(), newElementsCreated);
 
@@ -922,22 +935,6 @@ public class EntityGraphMapper {
         return newElementsCreated;
     }
 
-    private boolean isRelationshipAttribute(AtlasAttribute attribute) {
-        boolean ret = false;
-
-        if (attribute != null) {
-            AtlasStructType structType    = attribute.getDefinedInType();
-            String          attributeName = attribute.getName();
-
-            if (structType instanceof AtlasEntityType) {
-                ret = ((AtlasEntityType) structType).hasRelationshipAttribute(attributeName);
-            }
-        }
-
-        return ret;
-    }
-
-
     private AtlasEdge createVertex(AtlasStruct struct, AtlasVertex referringVertex, String edgeLabel, EntityMutationContext context) throws AtlasBaseException {
         AtlasVertex vertex = createStructVertex(struct);
 
@@ -1089,7 +1086,7 @@ public class EntityGraphMapper {
 
 
     public static Object getMapValueProperty(AtlasType elementType, AtlasVertex vertex, String vertexPropertyName) {
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
+        if (isReference(elementType)) {
             return vertex.getProperty(vertexPropertyName, AtlasEdge.class);
         } else if (elementType instanceof AtlasArrayType) {
             return vertex.getProperty(vertexPropertyName, List.class);
@@ -1102,7 +1099,7 @@ public class EntityGraphMapper {
     }
 
     private static void setMapValueProperty(AtlasType elementType, AtlasVertex vertex, String vertexPropertyName, Object value) {
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
+        if (isReference(elementType)) {
             vertex.setPropertyFromElementId(vertexPropertyName, (AtlasEdge)value);
         }
         else {
@@ -1113,16 +1110,18 @@ public class EntityGraphMapper {
     //Remove unused entries from map
     private Map<String, Object> removeUnusedMapEntries(AtlasAttribute attribute, AtlasVertex vertex, String propertyName,
                                                        Map<String, Object> currentMap, Map<String, Object> newMap)
-                                                                             throws AtlasException, AtlasBaseException {
+                                                       throws AtlasBaseException {
+
         AtlasMapType        mapType       = (AtlasMapType) attribute.getAttributeType();
+        boolean             isReference   = isReference(mapType.getValueType());
         Map<String, Object> additionalMap = new HashMap<>();
 
         for (String currentKey : currentMap.keySet()) {
             boolean shouldDeleteKey = !newMap.containsKey(currentKey);
 
-            if (AtlasGraphUtilsV1.isReference(mapType.getValueType())) {
+            if (isReference) {
                 //Delete the edge reference if its not part of new edges created/updated
-                AtlasEdge currentEdge = (AtlasEdge)currentMap.get(currentKey);
+                AtlasEdge currentEdge = (AtlasEdge) currentMap.get(currentKey);
 
                 if (!newMap.values().contains(currentEdge)) {
                     boolean deleted = deleteHandler.deleteEdgeReference(currentEdge, mapType.getValueType().getTypeCategory(), attribute.isOwnedRef(), true, vertex);
@@ -1134,8 +1133,9 @@ public class EntityGraphMapper {
                 }
             }
 
-            if (shouldDeleteKey) {
-                String propertyNameForKey = GraphHelper.getQualifiedNameForMapKey(propertyName, GraphHelper.encodePropertyKey(currentKey));
+            if (!isReference && shouldDeleteKey) {
+                String propertyNameForKey = getQualifiedNameForMapKey(propertyName, GraphHelper.encodePropertyKey(currentKey));
+
                 GraphHelper.setProperty(vertex, propertyNameForKey, null);
             }
         }
@@ -1146,7 +1146,7 @@ public class EntityGraphMapper {
     private static AtlasEdge getEdgeIfExists(AtlasMapType mapType, Map<String, Object> currentMap, String keyStr) {
         AtlasEdge ret = null;
 
-        if (AtlasGraphUtilsV1.isReference(mapType.getValueType())) {
+        if (isReference(mapType.getValueType())) {
             Object val = currentMap.get(keyStr);
 
             if (val != null) {
@@ -1232,7 +1232,7 @@ public class EntityGraphMapper {
     }
 
     public static List<Object> getArrayElementsProperty(AtlasType elementType, AtlasVertex vertex, String vertexPropertyName) {
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
+        if (isReference(elementType)) {
             return (List)vertex.getListProperty(vertexPropertyName, AtlasEdge.class);
         }
         else {
@@ -1240,25 +1240,10 @@ public class EntityGraphMapper {
         }
     }
 
-    public static List<Object> getArrayElementsUsingRelationship(AtlasVertex vertex, AtlasAttribute attribute, AtlasType elementType) {
-        List<Object> ret = null;
-
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
-
-            AtlasRelationshipEdgeDirection edgeDirection = attribute.getRelationshipEdgeDirection();
-            String                         edgeLabel = attribute.getRelationshipEdgeLabel();
-
-            Iterator<AtlasEdge> edgesForLabel = GraphHelper.getEdgesForLabel(vertex, edgeLabel, edgeDirection);
-
-            ret = IteratorUtils.toList(edgesForLabel);
-        }
-        return ret;
-    }
-
     private AtlasEdge getEdgeAt(List<Object> currentElements, int index, AtlasType elemType) {
         AtlasEdge ret = null;
 
-        if (AtlasGraphUtilsV1.isReference(elemType)) {
+        if (isReference(elemType)) {
             if (currentElements != null && index < currentElements.size()) {
                 ret = (AtlasEdge) currentElements.get(index);
             }
@@ -1273,7 +1258,7 @@ public class EntityGraphMapper {
         if (CollectionUtils.isNotEmpty(currentEntries)) {
             AtlasType entryType = ((AtlasArrayType) attribute.getAttributeType()).getElementType();
 
-            if (AtlasGraphUtilsV1.isReference(entryType)) {
+            if (isReference(entryType)) {
                 Collection<AtlasEdge> edgesToRemove = CollectionUtils.subtract(currentEntries, newEntries);
 
                 if (CollectionUtils.isNotEmpty(edgesToRemove)) {
@@ -1296,15 +1281,11 @@ public class EntityGraphMapper {
         return Collections.emptyList();
     }
     private void setArrayElementsProperty(AtlasType elementType, AtlasVertex vertex, String vertexPropertyName, List<Object> values) {
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
-            GraphHelper.setListPropertyFromElementIds(vertex, vertexPropertyName, (List) values);
-        }
-        else {
+        if (!isReference(elementType)) {
             GraphHelper.setProperty(vertex, vertexPropertyName, values);
         }
     }
 
-
     private AtlasEntityHeader constructHeader(AtlasEntity entity, final AtlasEntityType type, AtlasVertex vertex) {
         AtlasEntityHeader header = new AtlasEntityHeader(entity.getTypeName());
 
@@ -1324,9 +1305,9 @@ public class EntityGraphMapper {
     private void updateInConsistentOwnedMapVertices(AttributeMutationContext ctx, AtlasMapType mapType, Object val) {
         if (mapType.getValueType().getTypeCategory() == TypeCategory.OBJECT_ID_TYPE) {
             AtlasEdge edge = (AtlasEdge) val;
-            if (ctx.getAttribute().isOwnedRef() &&
-                GraphHelper.getStatus(edge) == AtlasEntity.Status.DELETED &&
-                GraphHelper.getStatus(edge.getInVertex()) == AtlasEntity.Status.DELETED) {
+
+            if (ctx.getAttribute().isOwnedRef() && getStatus(edge) == DELETED && getStatus(edge.getInVertex()) == DELETED) {
+
                 //Resurrect the vertex and edge to ACTIVE state
                 GraphHelper.setProperty(edge, STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name());
                 GraphHelper.setProperty(edge.getInVertex(), STATE_PROPERTY_KEY, AtlasEntity.Status.ACTIVE.name());
@@ -1813,6 +1794,7 @@ public class EntityGraphMapper {
             currentEntityId = getIdFromOutVertex(currentEdge);
         }
 
+
         return currentEntityId;
     }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
index 0d65471..183a2f6 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
@@ -54,6 +54,7 @@ import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.type.AtlasTypeUtil;
 import org.apache.atlas.utils.AtlasJson;
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -97,15 +98,19 @@ import static org.apache.atlas.repository.graph.GraphHelper.getAdjacentEdgesByLa
 import static org.apache.atlas.repository.graph.GraphHelper.getAllClassificationEdges;
 import static org.apache.atlas.repository.graph.GraphHelper.getAllTraitNames;
 import static org.apache.atlas.repository.graph.GraphHelper.getBlockedClassificationIds;
+import static org.apache.atlas.repository.graph.GraphHelper.getArrayElementsProperty;
 import static org.apache.atlas.repository.graph.GraphHelper.getClassificationVertices;
 import static org.apache.atlas.repository.graph.GraphHelper.getGuid;
 import static org.apache.atlas.repository.graph.GraphHelper.getIncomingEdgesByLabel;
+import static org.apache.atlas.repository.graph.GraphHelper.getPrimitiveMap;
+import static org.apache.atlas.repository.graph.GraphHelper.getReferenceMap;
 import static org.apache.atlas.repository.graph.GraphHelper.getOutGoingEdgesByLabel;
 import static org.apache.atlas.repository.graph.GraphHelper.getPropagateTags;
 import static org.apache.atlas.repository.graph.GraphHelper.getRelationshipGuid;
 import static org.apache.atlas.repository.graph.GraphHelper.getTypeName;
 import static org.apache.atlas.repository.graph.GraphHelper.isPropagationEnabled;
 import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getIdFromVertex;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.isReference;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.BOTH;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.IN;
@@ -614,10 +619,10 @@ public final class EntityGraphRetriever {
                 ret = mapVertexToObjectId(entityVertex, edgeLabel, null, entityExtInfo, isOwnedAttribute, edgeDirection);
                 break;
             case ARRAY:
-                ret = mapVertexToArray(entityVertex, (AtlasArrayType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute, edgeDirection);
+                ret = mapVertexToArray(entityVertex, vertexPropertyName, entityExtInfo, isOwnedAttribute, attribute);
                 break;
             case MAP:
-                ret = mapVertexToMap(entityVertex, (AtlasMapType) attrType, vertexPropertyName, entityExtInfo, isOwnedAttribute, edgeDirection);
+                ret = mapVertexToMap(entityVertex, vertexPropertyName, entityExtInfo, isOwnedAttribute, attribute);
                 break;
             case CLASSIFICATION:
                 // do nothing
@@ -627,45 +632,50 @@ public final class EntityGraphRetriever {
         return ret;
     }
 
-    private Map<String, Object> mapVertexToMap(AtlasVertex entityVertex, AtlasMapType atlasMapType, final String propertyName,
-                                               AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute,
-                                               AtlasRelationshipEdgeDirection edgeDirection) throws AtlasBaseException {
+    private Map<String, Object> mapVertexToMap(AtlasVertex entityVertex, final String propertyName, AtlasEntityExtInfo entityExtInfo,
+                                               boolean isOwnedAttribute, AtlasAttribute attribute) throws AtlasBaseException {
 
-        List<String> mapKeys = GraphHelper.getListProperty(entityVertex, propertyName);
-
-        if (CollectionUtils.isEmpty(mapKeys)) {
-            return null;
-        }
+        Map<String, Object> ret          = null;
+        AtlasMapType        mapType      = (AtlasMapType) attribute.getAttributeType();
+        AtlasType           mapValueType = mapType.getValueType();
 
         if (LOG.isDebugEnabled()) {
-            LOG.debug("Mapping map attribute {} for vertex {}", atlasMapType.getTypeName(), entityVertex);
+            LOG.debug("Mapping map attribute {} for vertex {}", mapType.getTypeName(), entityVertex);
         }
 
-        Map<String, Object> ret          = new HashMap<>(mapKeys.size());
-        AtlasType           mapValueType = atlasMapType.getValueType();
+        if (isReference(mapValueType)) {
+            Map<String, Object> currentMap = getReferenceMap(entityVertex, attribute);
 
-        for (String mapKey : mapKeys) {
-            final String keyPropertyName = propertyName + "." + mapKey;
-            final String edgeLabel       = EDGE_LABEL_PREFIX + keyPropertyName;
-            final Object keyValue        = GraphHelper.getMapValueProperty(mapValueType, entityVertex, keyPropertyName);
+            if (MapUtils.isNotEmpty(currentMap)) {
+                ret = new HashMap<>();
 
-            Object mapValue = mapVertexToCollectionEntry(entityVertex, mapValueType, keyValue, edgeLabel,
-                                                         entityExtInfo, isOwnedAttribute, edgeDirection);
-
-            if (mapValue != null) {
-                ret.put(mapKey, mapValue);
+                for (Map.Entry<String, Object> entry : currentMap.entrySet()) {
+                    String mapKey    = entry.getKey();
+                    Object keyValue  = entry.getValue();
+                    Object mapValue  = mapVertexToCollectionEntry(entityVertex, mapValueType, keyValue, attribute.getRelationshipEdgeLabel(),
+                                                                  entityExtInfo, isOwnedAttribute, attribute.getRelationshipEdgeDirection());
+                    if (mapValue != null) {
+                        ret.put(mapKey, mapValue);
+                    }
+                }
             }
+        } else {
+            ret = getPrimitiveMap(entityVertex, propertyName, mapValueType);
+        }
+
+        if (MapUtils.isEmpty(ret)) {
+            ret = null;
         }
 
         return ret;
     }
 
-    private List<Object> mapVertexToArray(AtlasVertex entityVertex, AtlasArrayType arrayType, String propertyName,
-                                          AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute,
-                                          AtlasRelationshipEdgeDirection edgeDirection)  throws AtlasBaseException {
+    private List<Object> mapVertexToArray(AtlasVertex entityVertex, String propertyName, AtlasEntityExtInfo entityExtInfo,
+                                          boolean isOwnedAttribute, AtlasAttribute attribute)  throws AtlasBaseException {
 
-        AtlasType    arrayElementType = arrayType.getElementType();
-        List<Object> arrayElements    = GraphHelper.getArrayElementsProperty(arrayElementType, entityVertex, propertyName);
+        AtlasArrayType arrayType        = (AtlasArrayType) attribute.getAttributeType();
+        AtlasType      arrayElementType = arrayType.getElementType();
+        List<Object>   arrayElements    = getArrayElementsProperty(arrayElementType, entityVertex, propertyName, attribute);
 
         if (CollectionUtils.isEmpty(arrayElements)) {
             return null;
@@ -675,8 +685,9 @@ public final class EntityGraphRetriever {
             LOG.debug("Mapping array attribute {} for vertex {}", arrayElementType.getTypeName(), entityVertex);
         }
 
-        List   arrValues = new ArrayList(arrayElements.size());
-        String edgeLabel = EDGE_LABEL_PREFIX + propertyName;
+        List                           arrValues     = new ArrayList(arrayElements.size());
+        String                         edgeLabel     = attribute.getRelationshipEdgeLabel();
+        AtlasRelationshipEdgeDirection edgeDirection = attribute.getRelationshipEdgeDirection();
 
         for (Object element : arrayElements) {
             // When internal types are deleted, sometimes the collection type attribute will contain a null value

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasComplexAttributesTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasComplexAttributesTest.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasComplexAttributesTest.java
new file mode 100644
index 0000000..e494728
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasComplexAttributesTest.java
@@ -0,0 +1,396 @@
+/**
+ * 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.repository.store.graph.v1;
+
+import org.apache.atlas.TestModules;
+import org.apache.atlas.TestUtilsV2;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.atlas.TestUtilsV2.ENTITY_TYPE;
+import static org.apache.atlas.TestUtilsV2.ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR;
+import static org.apache.atlas.TestUtilsV2.ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR_DELETE;
+import static org.apache.atlas.TestUtilsV2.NAME;
+import static org.apache.atlas.repository.graph.GraphHelper.getStatus;
+import static org.apache.atlas.type.AtlasTypeUtil.getAtlasObjectId;
+import static org.testng.Assert.assertNull;
+import static org.testng.AssertJUnit.assertEquals;
+
+@Guice(modules = TestModules.TestOnlyModule.class)
+public class AtlasComplexAttributesTest extends AtlasEntityTestBase {
+    private AtlasEntityWithExtInfo complexCollectionAttrEntity;
+    private AtlasEntityWithExtInfo complexCollectionAttrEntityForDelete;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        super.setUp();
+
+        // create typeDefs
+        AtlasTypesDef[] testTypesDefs = new AtlasTypesDef[] { TestUtilsV2.defineTypeWithComplexCollectionAttributes() };
+        createTypesDef(testTypesDefs);
+
+        // create entity
+        complexCollectionAttrEntity          = TestUtilsV2.createComplexCollectionAttrEntity();
+        complexCollectionAttrEntityForDelete = TestUtilsV2.createComplexCollectionAttrEntity();
+    }
+
+    @Test
+    public void testCreateComplexAttributeEntity() throws Exception {
+        init();
+
+        EntityMutationResponse response      = entityStore.createOrUpdate(new AtlasEntityStream(complexCollectionAttrEntity), false);
+        AtlasEntityHeader      entityCreated = response.getFirstCreatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+
+        validateEntity(complexCollectionAttrEntity, getEntityFromStore(entityCreated));
+    }
+
+    @Test(dependsOnMethods = "testCreateComplexAttributeEntity")
+    public void testStructArray() throws Exception {
+        init();
+        AtlasEntity              complexEntity       = getEntityFromStore(complexCollectionAttrEntity.getEntity().getGuid());
+        AtlasEntitiesWithExtInfo complexEntitiesInfo = new AtlasEntitiesWithExtInfo(complexEntity);
+
+        // Modify array of structs
+        List<AtlasStruct> structList = new ArrayList<>(Arrays.asList(new AtlasStruct("struct_type", "name", "structArray00"),
+                                                                     new AtlasStruct("struct_type", "name", "structArray11"),
+                                                                     new AtlasStruct("struct_type", "name", "structArray22")));
+        complexEntity.setAttribute("listOfStructs", structList);
+
+        EntityMutationResponse response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        AtlasEntityHeader      updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a new element to array of struct
+        init();
+        structList.add(new AtlasStruct("struct_type", "name", "structArray33"));
+        complexEntity.setAttribute("listOfStructs", structList);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // remove one of the struct values - structArray00
+        init();
+        structList.remove(0);
+        complexEntity.setAttribute("listOfStructs", structList);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Update struct value within array of struct
+        init();
+        structList.get(0).setAttribute(NAME, "structArray11-edit");
+        complexEntity.setAttribute("listOfStructs", structList);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a repeated element to array of struct
+        init();
+        structList.add(new AtlasStruct("struct_type", "name", "structArray33"));
+        complexEntity.setAttribute("listOfStructs", structList);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Remove all elements. Should set array attribute to null
+        init();
+        structList.clear();
+        complexEntity.setAttribute("listOfStructs", structList);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+    }
+
+    @Test(dependsOnMethods = "testStructArray")
+    public void testEntityArray() throws Exception {
+        init();
+        AtlasEntity              complexEntity       = getEntityFromStore(complexCollectionAttrEntity.getEntity().getGuid());
+        AtlasEntitiesWithExtInfo complexEntitiesInfo = new AtlasEntitiesWithExtInfo(complexEntity);
+
+        // Modify array of entities
+        AtlasEntity e0Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray00"); put("isReplicated", true); }});
+        AtlasEntity e1Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray11"); put("isReplicated", false); }});
+        AtlasEntity e2Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray22"); put("isReplicated", true); }});
+
+        List<AtlasObjectId> entityList = new ArrayList<>(Arrays.asList(getAtlasObjectId(e0Array), getAtlasObjectId(e1Array), getAtlasObjectId(e2Array)));
+
+        complexEntity.setAttribute("listOfEntities", entityList);
+        complexEntitiesInfo.addReferredEntity(e0Array);
+        complexEntitiesInfo.addReferredEntity(e1Array);
+        complexEntitiesInfo.addReferredEntity(e2Array);
+
+        init();
+        EntityMutationResponse response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        AtlasEntityHeader      updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a new element to array of entities
+        init();
+        AtlasEntity e3Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray33"); put("isReplicated", true); }});
+        entityList.add(getAtlasObjectId(e3Array));
+        complexEntity.setAttribute("listOfEntities", entityList);
+        complexEntitiesInfo.addReferredEntity(e3Array);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // remove one of the entity values - entityArray00
+        init();
+        entityList.remove(0);
+        complexEntity.setAttribute("listOfEntities", entityList);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Update entity value within array of struct
+        init();
+        e1Array.setAttribute(NAME, "entityArray11-edit");
+        complexEntity.setAttribute("listOfEntities", entityList);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a repeated element to array of struct
+        init();
+        AtlasEntity e3Array_duplicate = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray33"); put("isReplicated", true); }});
+        entityList.add(getAtlasObjectId(e3Array_duplicate));
+        complexEntity.setAttribute("listOfEntities", entityList);
+        complexEntitiesInfo.addReferredEntity(e3Array_duplicate);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Remove all elements. Should set array attribute to null
+        init();
+        entityList.clear();
+        complexEntity.setAttribute("listOfEntities", entityList);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+    }
+
+    @Test(dependsOnMethods = "testEntityArray")
+    public void testStructMap() throws Exception {
+        init();
+        AtlasEntity              complexEntity       = getEntityFromStore(complexCollectionAttrEntity.getEntity().getGuid());
+        AtlasEntitiesWithExtInfo complexEntitiesInfo = new AtlasEntitiesWithExtInfo(complexEntity);
+
+        // Modify map of structs
+        HashMap<String, AtlasStruct> structMap = new HashMap<String, AtlasStruct>() {{
+                                                        put("key00", new AtlasStruct("struct_type", "name", "structMap00"));
+                                                        put("key11", new AtlasStruct("struct_type", "name", "structMap11"));
+                                                        put("key22", new AtlasStruct("struct_type", "name", "structMap22")); }};
+
+        complexEntity.setAttribute("mapOfStructs", structMap);
+
+        EntityMutationResponse response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        AtlasEntityHeader      updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a new element to map of struct - structMap6
+        init();
+        structMap.put("key33", new AtlasStruct("struct_type", "name", "structMap33"));
+        complexEntity.setAttribute("mapOfStructs", structMap);
+
+        response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // remove one of the entity values - structMap
+        init();
+        structMap.remove("key00");
+        complexEntity.setAttribute("mapOfStructs", structMap);
+
+        response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Update struct value within map of struct
+        init();
+        structMap.get("key11").setAttribute("name", "structMap11-edit");
+        complexEntity.setAttribute("mapOfStructs", structMap);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a repeated element to array of struct
+        init();
+        structMap.put("key33", new AtlasStruct("struct_type", "name", "structMap33"));
+        complexEntity.setAttribute("mapOfStructs", structMap);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        // no update since duplicate entry
+        assertNull(updatedComplexEntity);
+
+        // Remove all elements. Should set array attribute to null
+        init();
+        structMap.clear();
+        complexEntity.setAttribute("mapOfStructs", structMap);
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+    }
+
+    @Test(dependsOnMethods = "testStructMap")
+    public void testEntityMap() throws Exception {
+        init();
+        AtlasEntity              complexEntity       = getEntityFromStore(complexCollectionAttrEntity.getEntity().getGuid());
+        AtlasEntitiesWithExtInfo complexEntitiesInfo = new AtlasEntitiesWithExtInfo(complexEntity);
+
+        // Modify map of entities
+        AtlasEntity e0MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue00"); put("isReplicated", false); }});
+        AtlasEntity e1MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue11"); put("isReplicated", true); }});
+        AtlasEntity e2MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue22"); put("isReplicated", false); }});
+
+        HashMap<String, Object> entityMap = new HashMap<String, Object>() {{ put("key00", getAtlasObjectId(e0MapValue));
+                                                                             put("key11", getAtlasObjectId(e1MapValue));
+                                                                             put("key22", getAtlasObjectId(e2MapValue)); }};
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e0MapValue);
+        complexEntitiesInfo.addReferredEntity(e1MapValue);
+        complexEntitiesInfo.addReferredEntity(e2MapValue);
+
+        init();
+        EntityMutationResponse response             = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        AtlasEntityHeader      updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a new element to map of entities
+        init();
+        AtlasEntity e3MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue33"); put("isReplicated", false); }});
+        entityMap.put("key33", getAtlasObjectId(e3MapValue));
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e3MapValue);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // remove one of the entity values - [key00 : entityMapValue00]
+        init();
+        entityMap.remove("key00");
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e3MapValue);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Update entity value within map of entities
+        init();
+
+        AtlasEntity e1MapValueEdit = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue11-edit"); put("isReplicated", false); }});
+        entityMap.clear();
+        entityMap.put("key11", getAtlasObjectId(e1MapValueEdit));
+        entityMap.put("key22", getAtlasObjectId(e2MapValue));
+        entityMap.put("key33", getAtlasObjectId(e3MapValue));
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e1MapValueEdit);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // add a repeated element to map of entities
+        init();
+        e3MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue33"); put("isReplicated", false); }});
+        entityMap.put("key33", getAtlasObjectId(e3MapValue));
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e3MapValue);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+
+        // Remove all elements. Should set map attribute to null
+        init();
+        entityMap.clear();
+        complexEntity.setAttribute("mapOfEntities", entityMap);
+        complexEntitiesInfo.addReferredEntity(e3MapValue);
+
+        response = entityStore.createOrUpdate(new AtlasEntityStream(complexEntitiesInfo), false);
+        updatedComplexEntity = response.getFirstUpdatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+        validateEntity(complexEntitiesInfo, getEntityFromStore(updatedComplexEntity));
+    }
+
+    @Test(dependsOnMethods = "testEntityMap")
+    public void testDeleteEntityRemoveReferences() throws Exception {
+        init();
+
+        complexCollectionAttrEntityForDelete.getEntity().setAttribute(NAME, ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR_DELETE);
+
+        EntityMutationResponse response      = entityStore.createOrUpdate(new AtlasEntityStream(complexCollectionAttrEntityForDelete), false);
+        AtlasEntityHeader      entityCreated = response.getFirstCreatedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+
+        validateEntity(complexCollectionAttrEntityForDelete, getEntityFromStore(entityCreated));
+
+        // delete entity and check if referenced complex attribute edges are deleted
+        response = entityStore.deleteById(entityCreated.getGuid());
+
+        AtlasEntityHeader entityDeleted = response.getFirstDeletedEntityByTypeName(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+
+        AtlasEntityWithExtInfo deletedEntityWithExtInfo = entityStore.getById(entityDeleted.getGuid());
+        AtlasVertex            deletedEntityVertex      = AtlasGraphUtilsV1.findByGuid(entityDeleted.getGuid());
+        Iterator<AtlasEdge>    edges                    = deletedEntityVertex.getEdges(AtlasEdgeDirection.OUT).iterator();
+
+        // validate all attribute edges are deleted
+        while (edges != null && edges.hasNext()) {
+            assertEquals(getStatus(edges.next()), AtlasEntity.Status.DELETED);
+        }
+
+        AtlasEntity                deletedEntity  = deletedEntityWithExtInfo.getEntity();
+        List<AtlasObjectId>        listOfEntities = (List<AtlasObjectId>) deletedEntity.getAttribute("listOfEntities");
+        Map<String, AtlasObjectId> mapOfEntities  = (Map<String, AtlasObjectId>) deletedEntity.getAttribute("mapOfEntities");
+
+        // validate entity attributes are deleted
+        for (AtlasObjectId o  : listOfEntities) {
+            AtlasEntity entity = deletedEntityWithExtInfo.getEntity(o.getGuid());
+            assertEquals(entity.getStatus(), AtlasEntity.Status.DELETED);
+        }
+
+        for (AtlasObjectId o  : mapOfEntities.values()) {
+            AtlasEntity entity = deletedEntityWithExtInfo.getEntity(o.getGuid());
+            assertEquals(entity.getStatus(), AtlasEntity.Status.DELETED);
+        }
+    }
+}
\ No newline at end of file


[3/3] atlas git commit: ATLAS-2662: Remove complex array and map attribute's edge information from entity vertex

Posted by sa...@apache.org.
ATLAS-2662: Remove complex array and map attribute's edge information from entity vertex


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

Branch: refs/heads/master
Commit: 6e7aa6ed312f381c478c9f5e535a7c86ae80d51e
Parents: a8fab3e
Author: Sarath Subramanian <ss...@hortonworks.com>
Authored: Wed May 9 16:13:05 2018 -0700
Committer: Sarath Subramanian <ss...@hortonworks.com>
Committed: Wed May 9 17:44:47 2018 -0700

----------------------------------------------------------------------
 .../org/apache/atlas/repository/Constants.java  |   2 +
 .../java/org/apache/atlas/AtlasErrorCode.java   |   1 +
 .../atlas/model/instance/AtlasObjectId.java     |  13 +-
 .../org/apache/atlas/type/AtlasStructType.java  |  11 +
 .../test/java/org/apache/atlas/TestUtilsV2.java | 519 +++++++++++--------
 .../atlas/repository/graph/GraphHelper.java     | 122 ++++-
 .../store/graph/v1/AtlasGraphUtilsV1.java       |   5 +-
 .../store/graph/v1/DeleteHandlerV1.java         | 169 +++---
 .../store/graph/v1/EntityGraphMapper.java       | 176 +++----
 .../store/graph/v1/EntityGraphRetriever.java    |  69 +--
 .../graph/v1/AtlasComplexAttributesTest.java    | 396 ++++++++++++++
 .../store/graph/v1/AtlasEntityStoreV1Test.java  | 275 ++--------
 .../store/graph/v1/AtlasEntityTestBase.java     | 243 +++++++++
 13 files changed, 1281 insertions(+), 720 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/common/src/main/java/org/apache/atlas/repository/Constants.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/repository/Constants.java b/common/src/main/java/org/apache/atlas/repository/Constants.java
index 16a3aaa..3732556 100644
--- a/common/src/main/java/org/apache/atlas/repository/Constants.java
+++ b/common/src/main/java/org/apache/atlas/repository/Constants.java
@@ -126,6 +126,8 @@ public final class Constants {
     public static final String CLASSIFICATION_EDGE_STATE_PROPERTY_KEY         = STATE_PROPERTY_KEY;
     public static final String CLASSIFICATION_LABEL                           = "classifiedAs";
     public static final String TERM_ASSIGNMENT_LABEL                          = "r:AtlasGlossarySemanticAssignment";
+    public static final String ATTRIBUTE_INDEX_PROPERTY_KEY                   = INTERNAL_PROPERTY_KEY_PREFIX + "index";
+    public static final String ATTRIBUTE_KEY_PROPERTY_KEY                     = INTERNAL_PROPERTY_KEY_PREFIX + "key";
 
     public static final String VERTEX_ID_IN_IMPORT_KEY = "__vIdInImport";
     public static final String EDGE_ID_IN_IMPORT_KEY   = "__eIdInImport";

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
index 5f89ffb..5e3d818 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -142,6 +142,7 @@ public enum AtlasErrorCode {
     RELATIONSHIP_END_IS_NULL(400, "ATLAS-400-00-07D", "Relationship end is invalid. Expected {0} but is NULL"),
     INVALID_TERM_RELATION_TO_SELF(400, "ATLAS-400-00-07E", "Invalid Term relationship: Term can't have a relationship with self"),
     INVALID_CHILD_CATEGORY_DIFFERENT_GLOSSARY(400, "ATLAS-400-00-07F", "Invalid child category relationship: Child category (guid = {0}) belongs to different glossary"),
+    ATTRIBUTE_TYPE_INVALID(400, "ATLAS-400-00-080", "{0}.{1}: invalid attribute type. Attribute cannot be of type classification"),
 
     UNAUTHORIZED_ACCESS(403, "ATLAS-403-00-001", "{0} is not authorized to perform {1}"),
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java b/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
index ab5f145..a8958a8 100644
--- a/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
+++ b/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
@@ -34,6 +34,8 @@ import javax.xml.bind.annotation.XmlSeeAlso;
 import org.apache.atlas.model.PList;
 import org.apache.atlas.model.SearchFilter.SortType;
 import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.commons.lang.StringUtils;
+
 import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.NONE;
 import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.PUBLIC_ONLY;
 
@@ -163,13 +165,12 @@ public class AtlasObjectId  implements Serializable {
 
         AtlasObjectId that = (AtlasObjectId) o;
 
-        // if guid is null, equality should be based on typeName/uniqueAttributes
-        if (guid != null && Objects.equals(guid, that.guid)) {
-            return true;
+        // if guid is empty/null, equality should be based on typeName/uniqueAttributes
+        if (StringUtils.isEmpty(guid) && StringUtils.isEmpty(that.guid)) {
+            return Objects.equals(typeName, that.typeName) && Objects.equals(uniqueAttributes, that.uniqueAttributes);
+        } else {
+            return Objects.equals(guid, that.guid);
         }
-
-        return Objects.equals(typeName, that.typeName) &&
-               Objects.equals(uniqueAttributes, that.uniqueAttributes);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java b/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
index 2f870dc..5e2e010 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
@@ -98,6 +98,17 @@ public class AtlasStructType extends AtlasType {
                 arrayType.setMaxCount(attributeDef.getValuesMaxCount());
             }
 
+            //check if attribute type is not classification
+            if (attrType instanceof AtlasArrayType) {
+                attrType = ((AtlasArrayType) attrType).getElementType();
+            } else if (attrType instanceof AtlasMapType) {
+                attrType = ((AtlasMapType) attrType).getValueType();
+            }
+
+            if (attrType instanceof AtlasClassificationType) {
+                throw new AtlasBaseException(AtlasErrorCode.ATTRIBUTE_TYPE_INVALID, getTypeName(), attributeDef.getName());
+            }
+
             a.put(attributeDef.getName(), attribute);
         }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/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 836710c..f48ccef 100755
--- a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
+++ b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
@@ -30,7 +30,6 @@ import org.apache.atlas.model.typedef.AtlasEnumDef;
 import org.apache.atlas.model.typedef.AtlasEnumDef.AtlasEnumElementDef;
 import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
-import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
 import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.atlas.type.AtlasTypeUtil;
@@ -48,7 +47,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality.LIST;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality.SINGLE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF;
+import static org.apache.atlas.type.AtlasTypeUtil.createClassTypeDef;
+import static org.apache.atlas.type.AtlasTypeUtil.createOptionalAttrDef;
+import static org.apache.atlas.type.AtlasTypeUtil.createRequiredAttrDef;
 import static org.apache.atlas.type.AtlasTypeUtil.createStructTypeDef;
+import static org.apache.atlas.type.AtlasTypeUtil.createUniqueRequiredAttrDef;
+import static org.apache.atlas.type.AtlasTypeUtil.getAtlasObjectId;
 
 
 /**
@@ -59,6 +68,8 @@ public final class TestUtilsV2 {
     public static final long TEST_DATE_IN_LONG = 1418265358440L;
 
     public static final String TEST_USER = "testUser";
+    public static final String STRUCT_TYPE = "struct_type";
+    public static final String ENTITY_TYPE = "entity_type";
 
     private static AtomicInteger seq = new AtomicInteger();
 
@@ -85,68 +96,68 @@ public final class TestUtilsV2 {
 
         AtlasStructDef addressDetails =
                 createStructTypeDef("Address", "Address"+_description,
-                        AtlasTypeUtil.createRequiredAttrDef("street", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("city", "string"));
+                        createRequiredAttrDef("street", "string"),
+                        createRequiredAttrDef("city", "string"));
 
         AtlasEntityDef deptTypeDef =
-                AtlasTypeUtil.createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description, Collections.<String>emptySet(),
-                        AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
+                createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description, Collections.<String>emptySet(),
+                        createUniqueRequiredAttrDef("name", "string"),
                         new AtlasAttributeDef("employees", String.format("array<%s>", "Employee"), true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1, false, false, false,
+                                SINGLE, 0, 1, false, false, false,
                             new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
-                                add(new AtlasStructDef.AtlasConstraintDef(AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                add(new AtlasStructDef.AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF));
                             }}));
 
-        AtlasEntityDef personTypeDef = AtlasTypeUtil.createClassTypeDef("Person", "Person"+_description, Collections.<String>emptySet(),
-                AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                AtlasTypeUtil.createOptionalAttrDef("address", "Address"),
-                AtlasTypeUtil.createOptionalAttrDef("birthday", "date"),
-                AtlasTypeUtil.createOptionalAttrDef("hasPets", "boolean"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfCars", "byte"),
-                AtlasTypeUtil.createOptionalAttrDef("houseNumber", "short"),
-                AtlasTypeUtil.createOptionalAttrDef("carMileage", "int"),
-                AtlasTypeUtil.createOptionalAttrDef("age", "float"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
-                AtlasTypeUtil.createOptionalAttrDef("approximationOfPi", "bigdecimal")
+        AtlasEntityDef personTypeDef = createClassTypeDef("Person", "Person"+_description, Collections.<String>emptySet(),
+                createUniqueRequiredAttrDef("name", "string"),
+                createOptionalAttrDef("address", "Address"),
+                createOptionalAttrDef("birthday", "date"),
+                createOptionalAttrDef("hasPets", "boolean"),
+                createOptionalAttrDef("numberOfCars", "byte"),
+                createOptionalAttrDef("houseNumber", "short"),
+                createOptionalAttrDef("carMileage", "int"),
+                createOptionalAttrDef("age", "float"),
+                createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
+                createOptionalAttrDef("approximationOfPi", "bigdecimal")
         );
 
-        AtlasEntityDef employeeTypeDef = AtlasTypeUtil.createClassTypeDef("Employee", "Employee"+_description, Collections.singleton("Person"),
-                AtlasTypeUtil.createOptionalAttrDef("orgLevel", "OrgLevel"),
+        AtlasEntityDef employeeTypeDef = createClassTypeDef("Employee", "Employee"+_description, Collections.singleton("Person"),
+                createOptionalAttrDef("orgLevel", "OrgLevel"),
                 new AtlasAttributeDef("department", "Department", false,
-                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        SINGLE, 1, 1,
                         false, false, false,
-                        new ArrayList<AtlasConstraintDef>()),
+                        new ArrayList<>()),
                 new AtlasAttributeDef("manager", "Manager", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                 new ArrayList<AtlasConstraintDef>() {{
                         add(new AtlasConstraintDef(
-                            AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
-                            put(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
+                            CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
+                            put(CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
                         }}));
                     }}),
                 new AtlasAttributeDef("mentor", EMPLOYEE_TYPE, true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
-                AtlasTypeUtil.createOptionalAttrDef("shares", "long"),
-                AtlasTypeUtil.createOptionalAttrDef("salary", "double")
+                createOptionalAttrDef("shares", "long"),
+                createOptionalAttrDef("salary", "double")
                 );
 
         employeeTypeDef.getAttribute("department").addConstraint(
             new AtlasConstraintDef(
-                AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
-                put(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "employees");
+                CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
+                put(CONSTRAINT_PARAM_ATTRIBUTE, "employees");
             }}));
 
-        AtlasEntityDef managerTypeDef = AtlasTypeUtil.createClassTypeDef("Manager", "Manager"+_description, Collections.singleton("Employee"),
+        AtlasEntityDef managerTypeDef = createClassTypeDef("Manager", "Manager"+_description, Collections.singleton("Employee"),
                 new AtlasAttributeDef("subordinates", String.format("array<%s>", "Employee"), false, AtlasAttributeDef.Cardinality.SET,
                         1, 10, false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()));
 
         AtlasClassificationDef securityClearanceTypeDef =
                 AtlasTypeUtil.createTraitTypeDef("SecurityClearance", "SecurityClearance"+_description, Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
         AtlasTypesDef ret = new AtlasTypesDef(Collections.singletonList(orgLevelEnum), Collections.singletonList(addressDetails),
                 Collections.singletonList(securityClearanceTypeDef),
@@ -158,27 +169,27 @@ public final class TestUtilsV2 {
     }
 
     public static AtlasTypesDef defineInverseReferenceTestTypes() {
-        AtlasEntityDef aDef = AtlasTypeUtil.createClassTypeDef("A", Collections.<String>emptySet(),
-            AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-            new AtlasAttributeDef("b", "B", true, Cardinality.SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-1
-            new AtlasAttributeDef("oneB", "B", true, Cardinality.SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-*
-            new AtlasAttributeDef("manyB", AtlasBaseTypeDef.getArrayTypeName("B"), true, Cardinality.SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()),
-            new AtlasAttributeDef("mapToB", AtlasBaseTypeDef.getMapTypeName("string", "B"), true, Cardinality.SINGLE, 0, 1, false, false, false,
+        AtlasEntityDef aDef = createClassTypeDef("A", Collections.<String>emptySet(),
+            createUniqueRequiredAttrDef("name", "string"),
+            new AtlasAttributeDef("b", "B", true, SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-1
+            new AtlasAttributeDef("oneB", "B", true, SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-*
+            new AtlasAttributeDef("manyB", AtlasBaseTypeDef.getArrayTypeName("B"), true, SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()),
+            new AtlasAttributeDef("mapToB", AtlasBaseTypeDef.getMapTypeName("string", "B"), true, SINGLE, 0, 1, false, false, false,
                 Collections.<AtlasConstraintDef>singletonList(new AtlasConstraintDef(
-                AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "mappedFromA"))))); // *-*
+                CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(CONSTRAINT_PARAM_ATTRIBUTE, "mappedFromA"))))); // *-*
 
-        AtlasEntityDef bDef = AtlasTypeUtil.createClassTypeDef("B", Collections.<String>emptySet(),
-            AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-            new AtlasAttributeDef("a", "A", true, Cardinality.SINGLE, 0, 1, false, false, false,
+        AtlasEntityDef bDef = createClassTypeDef("B", Collections.<String>emptySet(),
+            createUniqueRequiredAttrDef("name", "string"),
+            new AtlasAttributeDef("a", "A", true, SINGLE, 0, 1, false, false, false,
                 Collections.<AtlasConstraintDef>singletonList(new AtlasConstraintDef(
-                    AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "b")))),
-            new AtlasAttributeDef("manyA", AtlasBaseTypeDef.getArrayTypeName("A"), true, Cardinality.SINGLE, 0, 1, false, false, false,
+                    CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(CONSTRAINT_PARAM_ATTRIBUTE, "b")))),
+            new AtlasAttributeDef("manyA", AtlasBaseTypeDef.getArrayTypeName("A"), true, SINGLE, 0, 1, false, false, false,
                 Collections.<AtlasConstraintDef>singletonList(new AtlasConstraintDef(
-                    AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "oneB")))),
-            new AtlasAttributeDef("manyToManyA", AtlasBaseTypeDef.getArrayTypeName("A"), true, Cardinality.SINGLE, 0, 1, false, false, false,
+                    CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(CONSTRAINT_PARAM_ATTRIBUTE, "oneB")))),
+            new AtlasAttributeDef("manyToManyA", AtlasBaseTypeDef.getArrayTypeName("A"), true, SINGLE, 0, 1, false, false, false,
                 Collections.<AtlasConstraintDef>singletonList(new AtlasConstraintDef(
-                    AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "manyB")))),
-            new AtlasAttributeDef("mappedFromA", "A", true, Cardinality.SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()));
+                    CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(CONSTRAINT_PARAM_ATTRIBUTE, "manyB")))),
+            new AtlasAttributeDef("mappedFromA", "A", true, SINGLE, 0, 1, false, false, false, Collections.<AtlasConstraintDef>emptyList()));
 
         AtlasTypesDef ret = new AtlasTypesDef(Collections.<AtlasEnumDef>emptyList(), Collections.<AtlasStructDef>emptyList(), Collections.<AtlasClassificationDef>emptyList(), Arrays.asList(aDef, bDef));
 
@@ -198,63 +209,63 @@ public final class TestUtilsV2 {
 
         AtlasStructDef addressDetails =
                 createStructTypeDef("Address", "Address"+_description,
-                        AtlasTypeUtil.createRequiredAttrDef("street", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("city", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("zip", "int"));
+                        createRequiredAttrDef("street", "string"),
+                        createRequiredAttrDef("city", "string"),
+                        createOptionalAttrDef("zip", "int"));
 
         AtlasEntityDef deptTypeDef =
-                AtlasTypeUtil.createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description,
+                createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description,
                         Collections.<String>emptySet(),
-                        AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("dep-code", "string"),
+                        createUniqueRequiredAttrDef("name", "string"),
+                        createOptionalAttrDef("dep-code", "string"),
                         new AtlasAttributeDef("employees", String.format("array<%s>", "Employee"), true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1, false, false, false,
+                                SINGLE, 0, 1, false, false, false,
                             new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
-                                add(new AtlasStructDef.AtlasConstraintDef(AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                add(new AtlasStructDef.AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF));
                             }}));
 
-        AtlasEntityDef personTypeDef = AtlasTypeUtil.createClassTypeDef("Person", "Person"+_description,
+        AtlasEntityDef personTypeDef = createClassTypeDef("Person", "Person"+_description,
                 Collections.<String>emptySet(),
-                AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                AtlasTypeUtil.createOptionalAttrDef("email", "string"),
-                AtlasTypeUtil.createOptionalAttrDef("address", "Address"),
-                AtlasTypeUtil.createOptionalAttrDef("birthday", "date"),
-                AtlasTypeUtil.createOptionalAttrDef("hasPets", "boolean"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfCars", "byte"),
-                AtlasTypeUtil.createOptionalAttrDef("houseNumber", "short"),
-                AtlasTypeUtil.createOptionalAttrDef("carMileage", "int"),
-                AtlasTypeUtil.createOptionalAttrDef("age", "float"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
-                AtlasTypeUtil.createOptionalAttrDef("approximationOfPi", "bigdecimal")
+                createUniqueRequiredAttrDef("name", "string"),
+                createOptionalAttrDef("email", "string"),
+                createOptionalAttrDef("address", "Address"),
+                createOptionalAttrDef("birthday", "date"),
+                createOptionalAttrDef("hasPets", "boolean"),
+                createOptionalAttrDef("numberOfCars", "byte"),
+                createOptionalAttrDef("houseNumber", "short"),
+                createOptionalAttrDef("carMileage", "int"),
+                createOptionalAttrDef("age", "float"),
+                createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
+                createOptionalAttrDef("approximationOfPi", "bigdecimal")
         );
 
-        AtlasEntityDef employeeTypeDef = AtlasTypeUtil.createClassTypeDef("Employee", "Employee"+_description,
+        AtlasEntityDef employeeTypeDef = createClassTypeDef("Employee", "Employee"+_description,
                 Collections.singleton("Person"),
-                AtlasTypeUtil.createOptionalAttrDef("orgLevel", "OrgLevel"),
-                AtlasTypeUtil.createOptionalAttrDef("empCode", "string"),
+                createOptionalAttrDef("orgLevel", "OrgLevel"),
+                createOptionalAttrDef("empCode", "string"),
                 new AtlasAttributeDef("department", "Department", false,
-                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        SINGLE, 1, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("manager", "Manager", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                     new ArrayList<AtlasConstraintDef>() {{
                         add(new AtlasConstraintDef(
-                            AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
-                            put(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
+                            CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
+                            put(CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
                         }}));
                     }}),
                 new AtlasAttributeDef("mentor", EMPLOYEE_TYPE, true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
-                AtlasTypeUtil.createOptionalAttrDef("shares", "long"),
-                AtlasTypeUtil.createOptionalAttrDef("salary", "double")
+                createOptionalAttrDef("shares", "long"),
+                createOptionalAttrDef("salary", "double")
 
         );
 
-        AtlasEntityDef managerTypeDef = AtlasTypeUtil.createClassTypeDef("Manager", "Manager"+_description,
+        AtlasEntityDef managerTypeDef = createClassTypeDef("Manager", "Manager"+_description,
                 Collections.singleton("Employee"),
                 new AtlasAttributeDef("subordinates", String.format("array<%s>", "Employee"), false, AtlasAttributeDef.Cardinality.SET,
                         1, 10, false, false, false,
@@ -262,7 +273,7 @@ public final class TestUtilsV2 {
 
         AtlasClassificationDef securityClearanceTypeDef =
                 AtlasTypeUtil.createTraitTypeDef("SecurityClearance", "SecurityClearance"+_description, Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
         AtlasTypesDef ret = new AtlasTypesDef(Collections.singletonList(orgLevelEnum),
                 Collections.singletonList(addressDetails),
@@ -287,52 +298,52 @@ public final class TestUtilsV2 {
 
         AtlasStructDef addressDetails =
                 createStructTypeDef("Address", "Address"+_description,
-                        AtlasTypeUtil.createRequiredAttrDef("street", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("city", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("zip", "int"));
+                        createRequiredAttrDef("street", "string"),
+                        createRequiredAttrDef("city", "string"),
+                        createRequiredAttrDef("zip", "int"));
 
         AtlasEntityDef deptTypeDef =
-                AtlasTypeUtil.createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description, Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("dep-code", "string"),
+                createClassTypeDef(DEPARTMENT_TYPE, "Department"+_description, Collections.<String>emptySet(),
+                        createRequiredAttrDef("name", "string"),
+                        createRequiredAttrDef("dep-code", "string"),
                         new AtlasAttributeDef("employees", String.format("array<%s>", "Person"), true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1, false, false, false,
+                                SINGLE, 0, 1, false, false, false,
                             new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
-                                add(new AtlasStructDef.AtlasConstraintDef(AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                add(new AtlasStructDef.AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF));
                             }}));
 
-        AtlasEntityDef personTypeDef = AtlasTypeUtil.createClassTypeDef("Person", "Person"+_description, Collections.<String>emptySet(),
-                AtlasTypeUtil.createRequiredAttrDef("name", "string"),
-                AtlasTypeUtil.createRequiredAttrDef("emp-code", "string"),
-                AtlasTypeUtil.createOptionalAttrDef("orgLevel", "OrgLevel"),
-                AtlasTypeUtil.createOptionalAttrDef("address", "Address"),
+        AtlasEntityDef personTypeDef = createClassTypeDef("Person", "Person"+_description, Collections.<String>emptySet(),
+                createRequiredAttrDef("name", "string"),
+                createRequiredAttrDef("emp-code", "string"),
+                createOptionalAttrDef("orgLevel", "OrgLevel"),
+                createOptionalAttrDef("address", "Address"),
                 new AtlasAttributeDef("department", "Department", false,
-                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        SINGLE, 1, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("manager", "Manager", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                     new ArrayList<AtlasConstraintDef>() {{
                         add(new AtlasConstraintDef(
-                            AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
-                            put(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
+                            CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
+                            put(CONSTRAINT_PARAM_ATTRIBUTE, "subordinates");
                         }}));
                     }}),
                 new AtlasAttributeDef("mentor", "Person", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
-                AtlasTypeUtil.createOptionalAttrDef("birthday", "date"),
-                AtlasTypeUtil.createOptionalAttrDef("hasPets", "boolean"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfCars", "byte"),
-                AtlasTypeUtil.createOptionalAttrDef("houseNumber", "short"),
-                AtlasTypeUtil.createOptionalAttrDef("carMileage", "int"),
-                AtlasTypeUtil.createOptionalAttrDef("shares", "long"),
-                AtlasTypeUtil.createOptionalAttrDef("salary", "double"),
-                AtlasTypeUtil.createRequiredAttrDef("age", "float"),
-                AtlasTypeUtil.createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
-                AtlasTypeUtil.createOptionalAttrDef("approximationOfPi", "bigdecimal")
+                createOptionalAttrDef("birthday", "date"),
+                createOptionalAttrDef("hasPets", "boolean"),
+                createOptionalAttrDef("numberOfCars", "byte"),
+                createOptionalAttrDef("houseNumber", "short"),
+                createOptionalAttrDef("carMileage", "int"),
+                createOptionalAttrDef("shares", "long"),
+                createOptionalAttrDef("salary", "double"),
+                createRequiredAttrDef("age", "float"),
+                createOptionalAttrDef("numberOfStarsEstimate", "biginteger"),
+                createOptionalAttrDef("approximationOfPi", "bigdecimal")
         );
 
         AtlasTypesDef ret = new AtlasTypesDef(Collections.singletonList(orgLevelEnum),
@@ -355,7 +366,7 @@ public final class TestUtilsV2 {
 
         /******* Department - HR *******/
         AtlasEntity   hrDept   = new AtlasEntity(DEPARTMENT_TYPE, "name", "hr");
-        AtlasObjectId hrDeptId = AtlasTypeUtil.getAtlasObjectId(hrDept);
+        AtlasObjectId hrDeptId = getAtlasObjectId(hrDept);
 
         /******* Address Entities *******/
         AtlasStruct janeAddr = new AtlasStruct(ADDRESS_TYPE);
@@ -376,14 +387,14 @@ public final class TestUtilsV2 {
 
         /******* Manager - Jane (John and Max subordinates) *******/
         AtlasEntity   jane   = new AtlasEntity(MANAGER_TYPE);
-        AtlasObjectId janeId = AtlasTypeUtil.getAtlasObjectId(jane);
+        AtlasObjectId janeId = getAtlasObjectId(jane);
             jane.setAttribute("name", "Jane");
             jane.setAttribute("department", hrDeptId);
             jane.setAttribute("address", janeAddr);
 
         /******* Manager - Julius (no subordinates) *******/
         AtlasEntity   julius   = new AtlasEntity(MANAGER_TYPE);
-        AtlasObjectId juliusId = AtlasTypeUtil.getAtlasObjectId(julius);
+        AtlasObjectId juliusId = getAtlasObjectId(julius);
             julius.setAttribute("name", "Julius");
             julius.setAttribute("department", hrDeptId);
             julius.setAttribute("address", juliusAddr);
@@ -391,7 +402,7 @@ public final class TestUtilsV2 {
 
         /******* Employee - Max (Manager: Jane, Mentor: Julius) *******/
         AtlasEntity   max   = new AtlasEntity(EMPLOYEE_TYPE);
-        AtlasObjectId maxId = AtlasTypeUtil.getAtlasObjectId(max);
+        AtlasObjectId maxId = getAtlasObjectId(max);
             max.setAttribute("name", "Max");
             max.setAttribute("department", hrDeptId);
             max.setAttribute("address", maxAddr);
@@ -410,7 +421,7 @@ public final class TestUtilsV2 {
 
         /******* Employee - John (Manager: Jane, Mentor: Max) *******/
         AtlasEntity   john   = new AtlasEntity(EMPLOYEE_TYPE);
-        AtlasObjectId johnId = AtlasTypeUtil.getAtlasObjectId(john);
+        AtlasObjectId johnId = getAtlasObjectId(john);
             john.setAttribute("name", "John");
             john.setAttribute("department", hrDeptId);
             john.setAttribute("address", johnAddr);
@@ -486,8 +497,8 @@ public final class TestUtilsV2 {
         julius.setAttribute("address", juliusAddr);
         julius.setAttribute("subordinates", Collections.emptyList());
 
-        AtlasObjectId janeId = AtlasTypeUtil.getAtlasObjectId(jane);
-        AtlasObjectId johnId = AtlasTypeUtil.getAtlasObjectId(john);
+        AtlasObjectId janeId = getAtlasObjectId(jane);
+        AtlasObjectId johnId = getAtlasObjectId(john);
 
         //TODO - Change to MANAGER_TYPE for JULIUS
         AtlasObjectId maxId = new AtlasObjectId(max.getGuid(), EMPLOYEE_TYPE);
@@ -542,16 +553,18 @@ public final class TestUtilsV2 {
     public static final String COLUMNS_MAP = "columnsMap";
     public static final String COLUMNS_ATTR_NAME = "columns";
     public static final String ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR = "entity_with_nested_collection_attr";
+    public static final String ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR = "entity_with_complex_collection_attr";
+    public static final String ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR_DELETE = "entity_with_complex_collection_attr_delete";
 
     public static final String NAME = "name";
 
     public static AtlasTypesDef simpleType(){
         AtlasEntityDef superTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef("h_type", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createOptionalAttrDef("attr", "string"));
+                createClassTypeDef("h_type", Collections.<String>emptySet(),
+                        createOptionalAttrDef("attr", "string"));
 
         AtlasStructDef structTypeDefinition = new AtlasStructDef("s_type", "structType", "1.0",
-                Arrays.asList(AtlasTypeUtil.createRequiredAttrDef("name", "string")));
+                Arrays.asList(createRequiredAttrDef("name", "string")));
 
         AtlasClassificationDef traitTypeDefinition =
                 AtlasTypeUtil.createTraitTypeDef("t_type", "traitType", Collections.<String>emptySet());
@@ -569,15 +582,15 @@ public final class TestUtilsV2 {
 
     public static AtlasTypesDef simpleTypeUpdated(){
         AtlasEntityDef superTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef("h_type", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createOptionalAttrDef("attr", "string"));
+                createClassTypeDef("h_type", Collections.<String>emptySet(),
+                        createOptionalAttrDef("attr", "string"));
 
         AtlasEntityDef newSuperTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef("new_h_type", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createOptionalAttrDef("attr", "string"));
+                createClassTypeDef("new_h_type", Collections.<String>emptySet(),
+                        createOptionalAttrDef("attr", "string"));
 
         AtlasStructDef structTypeDefinition = new AtlasStructDef("s_type", "structType", "1.0",
-                Arrays.asList(AtlasTypeUtil.createRequiredAttrDef("name", "string")));
+                Arrays.asList(createRequiredAttrDef("name", "string")));
 
         AtlasClassificationDef traitTypeDefinition =
                 AtlasTypeUtil.createTraitTypeDef("t_type", "traitType", Collections.<String>emptySet());
@@ -594,8 +607,8 @@ public final class TestUtilsV2 {
 
     public static AtlasTypesDef simpleTypeUpdatedDiff() {
         AtlasEntityDef newSuperTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef("new_h_type", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createOptionalAttrDef("attr", "string"));
+                createClassTypeDef("new_h_type", Collections.<String>emptySet(),
+                        createOptionalAttrDef("attr", "string"));
 
         AtlasTypesDef ret = AtlasTypeUtil.getTypesDef(Collections.<AtlasEnumDef>emptyList(),
                 Collections.<AtlasStructDef>emptyList(),
@@ -611,24 +624,24 @@ public final class TestUtilsV2 {
     public static AtlasTypesDef defineHiveTypes() {
         String _description = "_description";
         AtlasEntityDef superTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef(SUPER_TYPE_NAME, "SuperType_description", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createOptionalAttrDef("namespace", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("cluster", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("colo", "string"));
+                createClassTypeDef(SUPER_TYPE_NAME, "SuperType_description", Collections.<String>emptySet(),
+                        createOptionalAttrDef("namespace", "string"),
+                        createOptionalAttrDef("cluster", "string"),
+                        createOptionalAttrDef("colo", "string"));
         AtlasEntityDef databaseTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef(DATABASE_TYPE, DATABASE_TYPE + _description,Collections.singleton(SUPER_TYPE_NAME),
-                        AtlasTypeUtil.createUniqueRequiredAttrDef(NAME, "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("isReplicated", "boolean"),
-                        AtlasTypeUtil.createOptionalAttrDef("created", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("parameters", "map<string,string>"),
-                        AtlasTypeUtil.createRequiredAttrDef("description", "string"));
+                createClassTypeDef(DATABASE_TYPE, DATABASE_TYPE + _description,Collections.singleton(SUPER_TYPE_NAME),
+                        createUniqueRequiredAttrDef(NAME, "string"),
+                        createOptionalAttrDef("isReplicated", "boolean"),
+                        createOptionalAttrDef("created", "string"),
+                        createOptionalAttrDef("parameters", "map<string,string>"),
+                        createRequiredAttrDef("description", "string"));
 
 
         AtlasStructDef structTypeDefinition = new AtlasStructDef("serdeType", "serdeType" + _description, "1.0",
                 Arrays.asList(
-                        AtlasTypeUtil.createRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("serde", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("description", "string")));
+                        createRequiredAttrDef("name", "string"),
+                        createRequiredAttrDef("serde", "string"),
+                        createOptionalAttrDef("description", "string")));
 
         AtlasEnumElementDef values[] = {
                 new AtlasEnumElementDef("MANAGED", "Element Description", 1),
@@ -637,45 +650,45 @@ public final class TestUtilsV2 {
         AtlasEnumDef enumTypeDefinition = new AtlasEnumDef("tableType", "tableType" + _description, "1.0", Arrays.asList(values));
 
         AtlasEntityDef columnsDefinition =
-                AtlasTypeUtil.createClassTypeDef(COLUMN_TYPE, COLUMN_TYPE + "_description",
+                createClassTypeDef(COLUMN_TYPE, COLUMN_TYPE + "_description",
                         Collections.<String>emptySet(),
-                        AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("type", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("description", "string"),
+                        createUniqueRequiredAttrDef("name", "string"),
+                        createRequiredAttrDef("type", "string"),
+                        createOptionalAttrDef("description", "string"),
                         new AtlasAttributeDef("table", TABLE_TYPE,
                         true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
                             add(new AtlasStructDef.AtlasConstraintDef(
-                                AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
-                                put(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "columns");
+                                CONSTRAINT_TYPE_INVERSE_REF, new HashMap<String, Object>() {{
+                                put(CONSTRAINT_PARAM_ATTRIBUTE, "columns");
                             }}));
                         }})
                         );
 
         AtlasStructDef partitionDefinition = new AtlasStructDef("partition_struct_type", "partition_struct_type" + _description, "1.0",
-                Arrays.asList(AtlasTypeUtil.createRequiredAttrDef("name", "string")));
+                Arrays.asList(createRequiredAttrDef("name", "string")));
 
         AtlasAttributeDef[] attributeDefinitions = new AtlasAttributeDef[]{
                 new AtlasAttributeDef("location", "string", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("inputFormat", "string", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("outputFormat", "string", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("compressed", "boolean", false,
-                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        SINGLE, 1, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("numBuckets", "int", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
         };
@@ -687,32 +700,32 @@ public final class TestUtilsV2 {
         AtlasAttributeDef[] partClsAttributes = new AtlasAttributeDef[]{
                 new AtlasAttributeDef("values", "array<string>",
                         true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("table", TABLE_TYPE, true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("createTime", "long", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("lastAccessTime", "long", true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("sd", STORAGE_DESC_TYPE, false,
-                        AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                        SINGLE, 1, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("columns", String.format("array<%s>", COLUMN_TYPE),
                         true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()),
                 new AtlasAttributeDef("parameters", String.format("map<%s,%s>", "string", "string"), true,
-                        AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                        SINGLE, 0, 1,
                         false, false, false,
                         Collections.<AtlasConstraintDef>emptyList())};
 
@@ -723,87 +736,93 @@ public final class TestUtilsV2 {
         AtlasEntityDef processClsType =
                 new AtlasEntityDef(PROCESS_TYPE, PROCESS_TYPE + _description, "1.0",
                         Arrays.asList(new AtlasAttributeDef("outputs", "array<" + TABLE_TYPE + ">", true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList())),
                         Collections.<String>emptySet());
 
         AtlasEntityDef tableTypeDefinition =
-                AtlasTypeUtil.createClassTypeDef(TABLE_TYPE, TABLE_TYPE + _description, Collections.singleton(SUPER_TYPE_NAME),
-                        AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("description", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("type", "string"),
-                        AtlasTypeUtil.createOptionalAttrDef("created", "date"),
+                createClassTypeDef(TABLE_TYPE, TABLE_TYPE + _description, Collections.singleton(SUPER_TYPE_NAME),
+                        createUniqueRequiredAttrDef("name", "string"),
+                        createOptionalAttrDef("description", "string"),
+                        createRequiredAttrDef("type", "string"),
+                        createOptionalAttrDef("created", "date"),
                         // enum
                         new AtlasAttributeDef("tableType", "tableType", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         // array of strings
                         new AtlasAttributeDef("columnNames",
                                 String.format("array<%s>", "string"), true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         // array of classes
                         new AtlasAttributeDef("columns", String.format("array<%s>", COLUMN_TYPE),
                                 true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
-                                    add(new AtlasStructDef.AtlasConstraintDef(AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                    add(new AtlasStructDef.AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF));
                                 }}),
                         // array of structs
                         new AtlasAttributeDef("partitions", String.format("array<%s>", "partition_struct_type"),
                                 true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
+                                false, false, false,
+                                Collections.<AtlasConstraintDef>emptyList()),
+                        // map of structs
+                        new AtlasAttributeDef("partitionsMap", String.format("map<%s,%s>", "string", "partition_struct_type"),
+                                true,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         // map of primitives
                         new AtlasAttributeDef("parametersMap", String.format("map<%s,%s>", "string", "string"),
                                 true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         //map of classes -
                         new AtlasAttributeDef(COLUMNS_MAP,
                                 String.format("map<%s,%s>", "string", COLUMN_TYPE),
                                 true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                  new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
                                      add(new AtlasStructDef.AtlasConstraintDef(
-                                             AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                             CONSTRAINT_TYPE_OWNED_REF));
                                      }}
                                 ),
                         //map of structs
                         new AtlasAttributeDef("partitionsMap",
                                 String.format("map<%s,%s>", "string", "partition_struct_type"),
                                 true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         // struct reference
                         new AtlasAttributeDef("serde1", "serdeType", true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("serde2", "serdeType", true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         // class reference
                         new AtlasAttributeDef("database", DATABASE_TYPE, false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         //class reference as composite
                         new AtlasAttributeDef("databaseComposite", DATABASE_TYPE, true,
-                                AtlasAttributeDef.Cardinality.SINGLE, 0, 1,
+                                SINGLE, 0, 1,
                                 false, false, false,
                                 new ArrayList<AtlasStructDef.AtlasConstraintDef>() {{
                                     add(new AtlasStructDef.AtlasConstraintDef(
-                                            AtlasConstraintDef.CONSTRAINT_TYPE_OWNED_REF));
+                                            CONSTRAINT_TYPE_OWNED_REF));
                                 }}
                         ));
 
@@ -812,16 +831,16 @@ public final class TestUtilsV2 {
 
         AtlasClassificationDef classificationTypeDefinition =
                 AtlasTypeUtil.createTraitTypeDef(CLASSIFICATION, CLASSIFICATION + _description, Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("tag", "string"));
+                        createRequiredAttrDef("tag", "string"));
 
         AtlasClassificationDef fetlClassificationTypeDefinition =
                 AtlasTypeUtil.createTraitTypeDef("fetl" + CLASSIFICATION, "fetl" + CLASSIFICATION + _description, Collections.singleton(CLASSIFICATION),
-                        AtlasTypeUtil.createRequiredAttrDef("tag", "string"));
+                        createRequiredAttrDef("tag", "string"));
 
         AtlasClassificationDef phiTypeDefinition = AtlasTypeUtil.createTraitTypeDef(PHI, PHI + _description, Collections.<String>emptySet(),
-                                                                                    AtlasTypeUtil.createRequiredAttrDef("stringAttr", "string"),
-                                                                                    AtlasTypeUtil.createRequiredAttrDef("booleanAttr", "boolean"),
-                                                                                    AtlasTypeUtil.createRequiredAttrDef("integerAttr", "int"));
+                                                                                    createRequiredAttrDef("stringAttr", "string"),
+                                                                                    createRequiredAttrDef("booleanAttr", "boolean"),
+                                                                                    createRequiredAttrDef("integerAttr", "int"));
 
         AtlasTypesDef ret = AtlasTypeUtil.getTypesDef(Collections.singletonList(enumTypeDefinition),
                                                       Arrays.asList(structTypeDefinition, partitionDefinition),
@@ -835,90 +854,90 @@ public final class TestUtilsV2 {
 
     public static AtlasTypesDef defineTypeWithNestedCollectionAttributes() {
         AtlasEntityDef nestedCollectionAttributesEntityType =
-                AtlasTypeUtil.createClassTypeDef(ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR, ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR + "_description", null,
-                        AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
+                createClassTypeDef(ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR, ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR + "_description", null,
+                        createUniqueRequiredAttrDef("name", "string"),
 
                         new AtlasAttributeDef("mapOfArrayOfStrings", "map<string,array<string>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfArrayOfBooleans", "map<string,array<boolean>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfArrayOfInts", "map<string,array<int>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfArrayOfFloats", "map<string,array<float>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfArrayOfDates", "map<string,array<date>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
 
                         new AtlasAttributeDef("mapOfMapOfStrings", "map<string,map<string,string>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfMapOfBooleans", "map<string,map<string,boolean>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfMapOfInts", "map<string,map<string,int>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfMapOfFloats", "map<string,map<string,float>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("mapOfMapOfDates", "map<string,map<string,date>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
 
                         new AtlasAttributeDef("arrayOfArrayOfStrings", "array<array<string>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfArrayOfBooleans", "array<array<boolean>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfArrayOfInts", "array<array<int>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfArrayOfFloats", "array<array<float>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfArrayOfDates", "array<array<date>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
 
                         new AtlasAttributeDef("arrayOfMapOfStrings", "array<map<string,string>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfMapOfBooleans", "array<map<string,boolean>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfMapOfInts", "array<map<string,int>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfMapOfFloats", "array<map<string,float>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList()),
                         new AtlasAttributeDef("arrayOfMapOfDates", "array<map<string,date>>", false,
-                                AtlasAttributeDef.Cardinality.SINGLE, 1, 1,
+                                SINGLE, 1, 1,
                                 false, false, false,
                                 Collections.<AtlasConstraintDef>emptyList())
                 );
@@ -933,6 +952,31 @@ public final class TestUtilsV2 {
         return ret;
     }
 
+    public static AtlasTypesDef defineTypeWithComplexCollectionAttributes() {
+        AtlasStructDef structType =
+                new AtlasStructDef(STRUCT_TYPE, "struct_type_description", "1.0", Arrays.asList(createRequiredAttrDef(NAME, "string")));
+
+        AtlasEntityDef entityType =
+                createClassTypeDef(ENTITY_TYPE, "entity_type_description", Collections.emptySet(), createUniqueRequiredAttrDef(NAME, "string"), createOptionalAttrDef("isReplicated", "boolean"));
+
+        AtlasEntityDef complexCollectionEntityType =
+                createClassTypeDef(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR, ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR + "_description", null,
+                        createUniqueRequiredAttrDef("name", "string"),
+                        new AtlasAttributeDef("listOfStructs", String.format("array<%s>", STRUCT_TYPE), true, LIST, 1, 1, false, false, false, Collections.emptyList()),
+                        new AtlasAttributeDef("listOfEntities", String.format("array<%s>", ENTITY_TYPE), true, LIST, 1, 1, false, false, false, new ArrayList<AtlasConstraintDef>() {{ add(new AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF)); }}),
+                        new AtlasAttributeDef("mapOfStructs", String.format("map<%s,%s>", "string", STRUCT_TYPE), true, SINGLE, 1, 1, false, false, false, Collections.emptyList()),
+                        new AtlasAttributeDef("mapOfEntities", String.format("map<%s,%s>", "string", ENTITY_TYPE), true, SINGLE, 1, 1, false, false, false, new ArrayList<AtlasConstraintDef>() {{ add(new AtlasConstraintDef(CONSTRAINT_TYPE_OWNED_REF)); }}) );
+
+        AtlasTypesDef ret = AtlasTypeUtil.getTypesDef(Collections.emptyList(),
+                                                      Arrays.asList(structType),
+                                                      Collections.emptyList(),
+                                                      Arrays.asList(entityType, complexCollectionEntityType));
+
+        populateSystemAttributes(ret);
+
+        return ret;
+    }
+
     public static AtlasEntityWithExtInfo createNestedCollectionAttrEntity() {
         AtlasEntity entity = new AtlasEntity(ENTITY_TYPE_WITH_NESTED_COLLECTION_ATTR);
 
@@ -977,6 +1021,44 @@ public final class TestUtilsV2 {
         return new AtlasEntityWithExtInfo(entity);
     }
 
+    public static AtlasEntityWithExtInfo createComplexCollectionAttrEntity() {
+        AtlasEntity entity = new AtlasEntity(ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+
+        entity.setAttribute(NAME, ENTITY_TYPE_WITH_COMPLEX_COLLECTION_ATTR);
+
+        entity.setAttribute("listOfStructs", Arrays.asList(new AtlasStruct("struct_type", "name", "structArray0"),
+                                                           new AtlasStruct("struct_type", "name", "structArray1"),
+                                                           new AtlasStruct("struct_type", "name", "structArray2")));
+
+        entity.setAttribute("mapOfStructs", new HashMap<String, AtlasStruct>() {{ put("key0", new AtlasStruct("struct_type", "name", "structMap0"));
+                                                                                  put("key1", new AtlasStruct("struct_type", "name", "structMap1"));
+                                                                                  put("key2", new AtlasStruct("struct_type", "name", "structMap2")); }});
+
+        AtlasEntity e1Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray0"); put("isReplicated", true); }});
+        AtlasEntity e2Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray1"); put("isReplicated", false); }});
+        AtlasEntity e3Array = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityArray2"); put("isReplicated", true); }});
+
+        entity.setAttribute("listOfEntities", Arrays.asList(getAtlasObjectId(e1Array), getAtlasObjectId(e2Array), getAtlasObjectId(e3Array)));
+
+        AtlasEntity e1MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue0"); put("isReplicated", false); }});
+        AtlasEntity e2MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue1"); put("isReplicated", true); }});
+        AtlasEntity e3MapValue = new AtlasEntity(ENTITY_TYPE, new HashMap<String, Object>() {{ put(NAME, "entityMapValue2"); put("isReplicated", false); }});
+
+        entity.setAttribute("mapOfEntities", new HashMap<String, Object>() {{ put("key0", getAtlasObjectId(e1MapValue));
+                                                                              put("key1", getAtlasObjectId(e2MapValue));
+                                                                              put("key2", getAtlasObjectId(e3MapValue)); }});
+        AtlasEntityWithExtInfo ret = new AtlasEntityWithExtInfo(entity);
+
+        ret.addReferredEntity(e1Array);
+        ret.addReferredEntity(e2Array);
+        ret.addReferredEntity(e3Array);
+        ret.addReferredEntity(e1MapValue);
+        ret.addReferredEntity(e2MapValue);
+        ret.addReferredEntity(e3MapValue);
+
+        return ret;
+    }
+
     public static final String randomString() {
         return RandomStringUtils.randomAlphanumeric(10);
     }
@@ -1018,7 +1100,7 @@ public final class TestUtilsV2 {
         entity.setAttribute("description", "random table");
         entity.setAttribute("type", "type");
         entity.setAttribute("tableType", "MANAGED");
-        entity.setAttribute("database", AtlasTypeUtil.getAtlasObjectId(dbEntity));
+        entity.setAttribute("database", getAtlasObjectId(dbEntity));
         entity.setAttribute("created", new Date());
 
         Map<String, Object> partAttributes = new HashMap<String, Object>() {{
@@ -1041,12 +1123,19 @@ public final class TestUtilsV2 {
         tblEntity.setAttribute("description", "random table");
         tblEntity.setAttribute("type", "type");
         tblEntity.setAttribute("tableType", "MANAGED");
-        tblEntity.setAttribute("database", AtlasTypeUtil.getAtlasObjectId(dbEntity));
+        tblEntity.setAttribute("database", getAtlasObjectId(dbEntity));
         tblEntity.setAttribute("created", new Date());
 
         final AtlasStruct partitionStruct = new AtlasStruct("partition_struct_type", "name", "part0");
 
         tblEntity.setAttribute("partitions", new ArrayList<AtlasStruct>() {{ add(partitionStruct); }});
+
+        tblEntity.setAttribute("partitionsMap", new HashMap<String, AtlasStruct>() {{
+            put("part0", new AtlasStruct("partition_struct_type", "name", "part0"));
+            put("part1", new AtlasStruct("partition_struct_type", "name", "part1"));
+            put("part2", new AtlasStruct("partition_struct_type", "name", "part2"));
+        }});
+
         tblEntity.setAttribute("parametersMap",
                 new java.util.HashMap<String, String>() {{ put("key1", "value1"); }});
 
@@ -1120,7 +1209,7 @@ public final class TestUtilsV2 {
         AtlasEntity entity = new AtlasEntity(COLUMN_TYPE);
         entity.setAttribute(NAME, colName);
         entity.setAttribute("type", "VARCHAR(32)");
-        entity.setAttribute("table", AtlasTypeUtil.getAtlasObjectId(tableEntity));
+        entity.setAttribute("table", getAtlasObjectId(tableEntity));
         return entity;
     }
 
@@ -1136,11 +1225,11 @@ public final class TestUtilsV2 {
     public static List<AtlasClassificationDef> getClassificationWithValidSuperType() {
         AtlasClassificationDef securityClearanceTypeDef =
                 AtlasTypeUtil.createTraitTypeDef("SecurityClearance1", "SecurityClearance_description", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
         AtlasClassificationDef janitorSecurityClearanceTypeDef =
                 AtlasTypeUtil.createTraitTypeDef("JanitorClearance", "JanitorClearance_description", Collections.singleton("SecurityClearance1"),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
         List<AtlasClassificationDef> ret = Arrays.asList(securityClearanceTypeDef, janitorSecurityClearanceTypeDef);
 
@@ -1152,7 +1241,7 @@ public final class TestUtilsV2 {
     public static List<AtlasClassificationDef> getClassificationWithName(String name) {
         AtlasClassificationDef classificationTypeDef =
                 AtlasTypeUtil.createTraitTypeDef(name, "s_description", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
 
         List<AtlasClassificationDef> ret = Arrays.asList(classificationTypeDef);
@@ -1165,7 +1254,7 @@ public final class TestUtilsV2 {
     public static AtlasClassificationDef getSingleClassificationWithName(String name) {
         AtlasClassificationDef classificaitonTypeDef =
                 AtlasTypeUtil.createTraitTypeDef(name, "s_description", Collections.<String>emptySet(),
-                        AtlasTypeUtil.createRequiredAttrDef("level", "int"));
+                        createRequiredAttrDef("level", "int"));
 
         populateSystemAttributes(classificaitonTypeDef);
 
@@ -1177,7 +1266,7 @@ public final class TestUtilsV2 {
     }
 
     public static List<AtlasEntityDef> getEntityWithValidSuperType() {
-        AtlasEntityDef developerTypeDef = AtlasTypeUtil.createClassTypeDef("Developer", "Developer_description", Collections.singleton("Employee"),
+        AtlasEntityDef developerTypeDef = createClassTypeDef("Developer", "Developer_description", Collections.singleton("Employee"),
                 new AtlasAttributeDef("language", String.format("array<%s>", "string"), false, AtlasAttributeDef.Cardinality.SET,
                         1, 10, false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()));
@@ -1190,7 +1279,7 @@ public final class TestUtilsV2 {
     }
 
     public static List<AtlasEntityDef> getEntityWithName(String name) {
-        AtlasEntityDef developerTypeDef = AtlasTypeUtil.createClassTypeDef(name, "Developer_description", Collections.<String>emptySet(),
+        AtlasEntityDef developerTypeDef = createClassTypeDef(name, "Developer_description", Collections.<String>emptySet(),
                 new AtlasAttributeDef("language", String.format("array<%s>", "string"), false, AtlasAttributeDef.Cardinality.SET,
                         1, 10, false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()));
@@ -1203,7 +1292,7 @@ public final class TestUtilsV2 {
     }
 
     public static AtlasEntityDef getSingleEntityWithName(String name) {
-        AtlasEntityDef developerTypeDef = AtlasTypeUtil.createClassTypeDef(name, "Developer_description", Collections.<String>emptySet(),
+        AtlasEntityDef developerTypeDef = createClassTypeDef(name, "Developer_description", Collections.<String>emptySet(),
                 new AtlasAttributeDef("language", String.format("array<%s>", "string"), false, AtlasAttributeDef.Cardinality.SET,
                         1, 10, false, false, false,
                         Collections.<AtlasConstraintDef>emptyList()));

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
index 7540b4c..7490a15 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphHelper.java
@@ -26,12 +26,13 @@ import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity.Status;
 import org.apache.atlas.model.instance.AtlasObjectId;
 import org.apache.atlas.model.instance.AtlasRelationship;
 import org.apache.atlas.model.typedef.AtlasRelationshipDef;
 import org.apache.atlas.repository.graphdb.AtlasVertexQuery;
+import org.apache.atlas.type.AtlasArrayType;
+import org.apache.atlas.type.AtlasMapType;
 import org.apache.atlas.util.AtlasGremlinQueryProvider;
 import org.apache.atlas.v1.model.instance.Id;
 import org.apache.atlas.v1.model.instance.Referenceable;
@@ -54,6 +55,7 @@ import org.apache.atlas.exception.EntityNotFoundException;
 import org.apache.atlas.util.AttributeValueMap;
 import org.apache.atlas.util.IndexedInstance;
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,6 +77,8 @@ import java.util.Set;
 import java.util.UUID;
 
 import static org.apache.atlas.model.instance.AtlasEntity.Status.DELETED;
+import static org.apache.atlas.repository.Constants.ATTRIBUTE_INDEX_PROPERTY_KEY;
+import static org.apache.atlas.repository.Constants.ATTRIBUTE_KEY_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_IS_PROPAGATED_PROPERTY_KEY;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_ENTITY_GUID;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_LABEL;
@@ -82,6 +86,7 @@ import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_NAME_PRO
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_VERTEX_NAME_KEY;
 import static org.apache.atlas.repository.Constants.CLASSIFICATION_VERTEX_PROPAGATE_KEY;
 import static org.apache.atlas.repository.Constants.PROPAGATED_TRAIT_NAMES_PROPERTY_KEY;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.isReference;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.BOTH;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.IN;
 import static org.apache.atlas.type.AtlasStructType.AtlasAttribute.AtlasRelationshipEdgeDirection.OUT;
@@ -1170,6 +1175,11 @@ public final class GraphHelper {
         return AtlasGraphUtilsV1.getProperty(classificationVertex, CLASSIFICATION_ENTITY_GUID, String.class);
     }
 
+    public static Integer getIndexValue(AtlasEdge edge) {
+        Integer index = edge.getProperty(ATTRIBUTE_INDEX_PROPERTY_KEY, Integer.class);
+        return (index == null) ? 0: index;
+    }
+
     public static boolean isPropagatedClassificationEdge(AtlasEdge edge) {
         boolean ret = false;
 
@@ -1557,44 +1567,100 @@ public final class GraphHelper {
         return typeName != null && typeName.startsWith(Constants.INTERNAL_PROPERTY_KEY_PREFIX);
     }
 
-    public static void setArrayElementsProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName, List<Object> values) {
-        String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
-        if(AtlasGraphUtilsV1.isReference(elementType)) {
-            setListPropertyFromElementIds(instanceVertex, actualPropertyName, (List)values);
-        }
-        else {
-            setProperty(instanceVertex, actualPropertyName, values);
+    public static Object getMapValueProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName) {
+        String vertexPropertyName = GraphHelper.encodePropertyKey(propertyName);
+
+        if (!AtlasGraphUtilsV1.isReference(elementType)) {
+            return instanceVertex.getProperty(vertexPropertyName, Object.class);
         }
+
+        return null;
     }
 
-    public static void setMapValueProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName, Object value) {
-        String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
-        if(AtlasGraphUtilsV1.isReference(elementType)) {
-            instanceVertex.setPropertyFromElementId(actualPropertyName, (AtlasEdge)value);
-        }
-        else {
-            instanceVertex.setProperty(actualPropertyName, value);
+    public static List<Object> getArrayElementsProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName, AtlasAttribute attribute) {
+        String encodedPropertyName = GraphHelper.encodePropertyKey(propertyName);
+
+        if (isReference(elementType)) {
+            return (List) getCollectionElementsUsingRelationship(instanceVertex, attribute);
+        } else {
+            return (List) instanceVertex.getListProperty(encodedPropertyName);
         }
     }
 
-    public static Object getMapValueProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName) {
-        String vertexPropertyName = GraphHelper.encodePropertyKey(propertyName);
+    public static Map<String, Object> getMapElementsProperty(AtlasMapType mapType, AtlasVertex instanceVertex, String propertyName, AtlasAttribute attribute) {
+        AtlasType mapValueType = mapType.getValueType();
 
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
-            return instanceVertex.getProperty(vertexPropertyName, AtlasEdge.class);
+        if (isReference(mapValueType)) {
+            return getReferenceMap(instanceVertex, attribute);
         } else {
-            return instanceVertex.getProperty(vertexPropertyName, Object.class);
+            return getPrimitiveMap(instanceVertex, propertyName, mapValueType);
         }
     }
 
-    // newly added
-    public static List<Object>  getArrayElementsProperty(AtlasType elementType, AtlasVertex instanceVertex, String propertyName) {
-        String encodedPropertyName = GraphHelper.encodePropertyKey(propertyName);
-        if(AtlasGraphUtilsV1.isReference(elementType)) {
-            return (List)instanceVertex.getListProperty(encodedPropertyName, AtlasEdge.class);
+    // map elements for reference types - AtlasObjectId, AtlasStruct
+    public static Map<String, Object> getReferenceMap(AtlasVertex instanceVertex, AtlasAttribute attribute) {
+        Map<String, Object> ret            = new HashMap<>();
+        List<AtlasEdge>     referenceEdges = getCollectionElementsUsingRelationship(instanceVertex, attribute);
+
+        for (AtlasEdge edge : referenceEdges) {
+            String key = edge.getProperty(ATTRIBUTE_KEY_PROPERTY_KEY, String.class);
+
+            if (StringUtils.isNotEmpty(key)) {
+                ret.put(key, edge);
+            }
         }
-        else {
-            return (List)instanceVertex.getListProperty(encodedPropertyName);
+
+        return ret;
+    }
+
+    public static List<AtlasEdge> getMapValuesUsingRelationship(AtlasVertex vertex, AtlasAttribute attribute) {
+        String                         edgeLabel     = attribute.getRelationshipEdgeLabel();
+        AtlasRelationshipEdgeDirection edgeDirection = attribute.getRelationshipEdgeDirection();
+        Iterator<AtlasEdge>            edgesForLabel = getEdgesForLabel(vertex, edgeLabel, edgeDirection);
+
+        return (List<AtlasEdge>) IteratorUtils.toList(edgesForLabel);
+    }
+
+    // map elements for primitive types
+    public static Map<String,Object> getPrimitiveMap(AtlasVertex instanceVertex, String propertyName, AtlasType mapValueType) {
+        String              encodedPropertyName = encodePropertyKey(propertyName);
+        List<String>        currentKeys         = getListProperty(instanceVertex, encodedPropertyName);
+        Map<String, Object> ret                 = new HashMap<>();
+
+        if (CollectionUtils.isNotEmpty(currentKeys)) {
+            for (String key : currentKeys) {
+                String propertyNameForKey  = getQualifiedNameForMapKey(encodedPropertyName, encodePropertyKey(key));
+                Object propertyValueForKey = getMapValueProperty(mapValueType, instanceVertex, propertyNameForKey);
+
+                ret.put(key, propertyValueForKey);
+            }
+        }
+
+        return ret;
+    }
+
+    public static List<AtlasEdge> getCollectionElementsUsingRelationship(AtlasVertex vertex, AtlasAttribute attribute) {
+        List<AtlasEdge>                ret;
+        String                         edgeLabel     = attribute.getRelationshipEdgeLabel();
+        AtlasRelationshipEdgeDirection edgeDirection = attribute.getRelationshipEdgeDirection();
+        Iterator<AtlasEdge>            edgesForLabel = getEdgesForLabel(vertex, edgeLabel, edgeDirection);
+
+        ret = IteratorUtils.toList(edgesForLabel);
+
+        sortCollectionElements(attribute, ret);
+
+        return ret;
+    }
+
+    private static void sortCollectionElements(AtlasAttribute attribute, List<AtlasEdge> edges) {
+        // sort array elements based on edge index
+        if (attribute.getAttributeType() instanceof AtlasArrayType && CollectionUtils.isNotEmpty(edges)) {
+            Collections.sort(edges, (e1, e2) -> {
+                Integer e1Index = getIndexValue(e1);
+                Integer e2Index = getIndexValue(e2);
+
+                return e1Index.compareTo(e2Index);
+            });
         }
     }
 
@@ -1730,7 +1796,7 @@ public final class GraphHelper {
 
     }
 
-    public static void setListProperty(AtlasVertex instanceVertex, String propertyName, ArrayList<String> value) throws AtlasException {
+    public static void setListProperty(AtlasVertex instanceVertex, String propertyName, ArrayList<String> value) {
         String actualPropertyName = GraphHelper.encodePropertyKey(propertyName);
         instanceVertex.setListProperty(actualPropertyName, value);
     }

http://git-wip-us.apache.org/repos/asf/atlas/blob/6e7aa6ed/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
index bd0dc0b..3e6f819 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
@@ -146,9 +146,8 @@ public class AtlasGraphUtilsV1 {
 
     public static boolean isReference(TypeCategory typeCategory) {
         return typeCategory == TypeCategory.STRUCT ||
-            typeCategory == TypeCategory.ENTITY ||
-            typeCategory == TypeCategory.CLASSIFICATION ||
-            typeCategory == TypeCategory.OBJECT_ID_TYPE;
+               typeCategory == TypeCategory.ENTITY ||
+               typeCategory == TypeCategory.OBJECT_ID_TYPE;
     }
 
     public static String encodePropertyKey(String key) {