You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/07/24 15:57:07 UTC

[02/39] atlas git commit: ATLAS-1907: created RelationshipDefs for all base models and use it when creating edges for entities

http://git-wip-us.apache.org/repos/asf/atlas/blob/8fe110c3/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 a5b5730..e919d1c 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
@@ -29,6 +29,8 @@ 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.typedef.AtlasRelationshipDef;
+import org.apache.atlas.model.typedef.AtlasRelationshipEndDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.GraphHelper;
@@ -49,6 +51,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -65,6 +68,7 @@ import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_LONG;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_SHORT;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_STRING;
 import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX;
+import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.getIdFromVertex;
 
 
 public final class EntityGraphRetriever {
@@ -180,6 +184,8 @@ public final class EntityGraphRetriever {
 
             mapAttributes(entityVertex, entity, entityExtInfo);
 
+            mapRelationshipAttributes(entityVertex, entity, entityExtInfo);
+
             mapClassifications(entityVertex, entity, entityExtInfo);
         }
 
@@ -278,6 +284,23 @@ public final class EntityGraphRetriever {
         }
     }
 
+    private void mapRelationshipAttributes(AtlasVertex entityVertex, AtlasEntity entity, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {
+        AtlasType objType = typeRegistry.getType(entity.getTypeName());
+
+        if (!(objType instanceof AtlasEntityType)) {
+            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, entity.getTypeName());
+        }
+
+        AtlasEntityType entityType = (AtlasEntityType) objType;
+
+        for (AtlasAttribute attribute : entityType.getRelationshipAttributes().values()) {
+
+            Object attrValue = mapVertexToRelationshipAttribute(entityVertex, entityType, attribute, entityExtInfo);
+
+            entity.addRelationshipAttribute(attribute.getName(), attrValue);
+        }
+    }
+
     public List<AtlasClassification> getClassifications(String guid) throws AtlasBaseException {
 
         AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
@@ -394,6 +417,40 @@ public final class EntityGraphRetriever {
         return ret;
     }
 
+    private Object mapVertexToRelationshipAttribute(AtlasVertex entityVertex, AtlasEntityType entityType, AtlasAttribute attribute,
+                                                    AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {
+        Object                  ret                = null;
+        AtlasRelationshipDef    relationshipDef    = graphHelper.getRelationshipDef(entityVertex, entityType, attribute.getName());
+        AtlasRelationshipEndDef endDef1            = relationshipDef.getEndDef1();
+        AtlasRelationshipEndDef endDef2            = relationshipDef.getEndDef2();
+        AtlasEntityType         endDef1Type        = typeRegistry.getEntityTypeByName(endDef1.getType());
+        AtlasEntityType         endDef2Type        = typeRegistry.getEntityTypeByName(endDef2.getType());
+        AtlasRelationshipEndDef attributeEndDef    = null;
+
+        if (endDef1Type.isTypeOrSuperTypeOf(entityType.getTypeName()) && StringUtils.equals(endDef1.getName(), attribute.getName())) {
+            attributeEndDef = endDef1;
+
+        } else if (endDef2Type.isTypeOrSuperTypeOf(entityType.getTypeName()) && StringUtils.equals(endDef2.getName(), attribute.getName())) {
+            attributeEndDef = endDef2;
+        }
+
+        String relationshipLabel = attribute.getRelationshipEdgeLabel();
+
+        switch (attributeEndDef.getCardinality()) {
+            case SINGLE:
+                ret = mapVertexToObjectId(entityVertex, relationshipLabel, null, entityExtInfo, attributeEndDef.getIsContainer());
+                break;
+
+            case LIST:
+            case SET:
+                ret = mapVertexToRelationshipArrayAttribute(entityVertex, (AtlasArrayType) attribute.getAttributeType(), relationshipLabel,
+                                                            entityExtInfo, attributeEndDef.getIsContainer());
+                break;
+        }
+
+        return ret;
+    }
+
     private Map<String, Object> mapVertexToMap(AtlasVertex entityVertex, AtlasMapType atlasMapType, final String propertyName,
                                                AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException {
         List<String> mapKeys = GraphHelper.getListProperty(entityVertex, propertyName);
@@ -451,6 +508,40 @@ public final class EntityGraphRetriever {
         return arrValues;
     }
 
+    private List<Object> mapVertexToRelationshipArrayAttribute(AtlasVertex entityVertex, AtlasArrayType arrayType,
+                                                               String relationshipName, AtlasEntityExtInfo entityExtInfo,
+                                                               boolean isContainer) throws AtlasBaseException {
+
+        Iterator<AtlasEdge> relationshipEdges = graphHelper.getBothEdgesByLabel(entityVertex, relationshipName);
+        AtlasType           arrayElementType  = arrayType.getElementType();
+        List<AtlasEdge>     arrayElements     = new ArrayList<>();
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Mapping array attribute {} for vertex {}", arrayElementType.getTypeName(), entityVertex);
+        }
+
+        while (relationshipEdges.hasNext()) {
+            arrayElements.add(relationshipEdges.next());
+        }
+
+        if (CollectionUtils.isEmpty(arrayElements)) {
+            return null;
+        }
+
+        List arrValues = new ArrayList(arrayElements.size());
+
+        for (Object element : arrayElements) {
+            Object arrValue = mapVertexToCollectionEntry(entityVertex, arrayElementType, element, relationshipName,
+                                                         entityExtInfo, isContainer);
+
+            if (arrValue != null) {
+                arrValues.add(arrValue);
+            }
+        }
+
+        return arrValues;
+    }
+
     private Object mapVertexToCollectionEntry(AtlasVertex entityVertex, AtlasType arrayElement, Object value, String edgeLabel,
                                               AtlasEntityExtInfo entityExtInfo, boolean isOwnedAttribute) throws AtlasBaseException {
         Object ret = null;
@@ -538,7 +629,11 @@ public final class EntityGraphRetriever {
         }
 
         if (GraphHelper.elementExists(edge)) {
-            final AtlasVertex referenceVertex = edge.getInVertex();
+            AtlasVertex referenceVertex = edge.getInVertex();
+
+            if (StringUtils.equals(getIdFromVertex(referenceVertex), getIdFromVertex(entityVertex))) {
+                referenceVertex = edge.getOutVertex();
+            }
 
             if (referenceVertex != null) {
                 if (entityExtInfo != null && isOwnedAttribute) {

http://git-wip-us.apache.org/repos/asf/atlas/blob/8fe110c3/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipStoreV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipStoreV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipStoreV1Test.java
new file mode 100644
index 0000000..6770223
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasRelationshipStoreV1Test.java
@@ -0,0 +1,99 @@
+/**
+ * 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.RequestContextV1;
+import org.apache.atlas.TestModules;
+import org.apache.atlas.TestUtilsV2;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
+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.repository.store.graph.AtlasRelationshipStore;
+import org.apache.atlas.store.AtlasTypeDefStore;
+import org.apache.atlas.type.AtlasTypeRegistry;
+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 static org.mockito.Mockito.mock;
+
+@Guice(modules = TestModules.TestOnlyModule.class)
+public class AtlasRelationshipStoreV1Test {
+
+    @Inject
+    AtlasTypeRegistry typeRegistry;
+
+    @Inject
+    AtlasTypeDefStore typeDefStore;
+
+    @Inject
+    DeleteHandlerV1   deleteHandler;
+
+    @Inject
+    EntityGraphMapper graphMapper;
+
+    AtlasEntityStore          entityStore;
+    AtlasRelationshipStore    relationshipStore;
+    AtlasEntityWithExtInfo    dbEntity;
+    AtlasEntityWithExtInfo    tblEntity;
+    AtlasEntityChangeNotifier mockChangeNotifier = mock(AtlasEntityChangeNotifier.class);
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        new GraphBackedSearchIndexer(typeRegistry);
+
+        AtlasTypesDef[] testTypesDefs = new AtlasTypesDef[] { TestUtilsV2.defineDeptEmployeeTypes(),
+                                                              TestUtilsV2.defineHiveTypes() };
+
+        for (AtlasTypesDef typesDef : testTypesDefs) {
+            AtlasTypesDef typesToCreate = AtlasTypeDefStoreInitializer.getTypesToCreate(typesDef, typeRegistry);
+
+            if (!typesToCreate.isEmpty()) {
+                typeDefStore.createTypesDef(typesToCreate);
+            }
+        }
+
+        dbEntity   = TestUtilsV2.createDBEntityV2();
+        tblEntity  = TestUtilsV2.createTableEntityV2(dbEntity.getEntity());
+    }
+
+    @AfterClass
+    public void clear() {
+        AtlasGraphProvider.cleanup();
+    }
+
+    @BeforeTest
+    public void init() throws Exception {
+        entityStore       = new AtlasEntityStoreV1(deleteHandler, typeRegistry, mockChangeNotifier, graphMapper);
+        relationshipStore = new AtlasRelationshipStoreV1(typeRegistry);
+
+        RequestContextV1.clear();
+    }
+
+    @Test
+    public void testDbTableRelationship() throws Exception  {
+        // Add tests - in progress
+    }
+}