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 2018/02/17 00:20:17 UTC

atlas git commit: ATLAS-2443: include unique attributes in entity-delete notifications

Repository: atlas
Updated Branches:
  refs/heads/branch-0.8 11ed3ccd3 -> 18063ef20


ATLAS-2443: include unique attributes in entity-delete notifications

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


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

Branch: refs/heads/branch-0.8
Commit: 18063ef209a095be7e054b363dcb7b779500dc3f
Parents: 11ed3cc
Author: apoorvnaik <ap...@apache.org>
Authored: Tue Feb 13 20:24:26 2018 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Fri Feb 16 16:10:50 2018 -0800

----------------------------------------------------------------------
 .../atlas/repository/graph/DeleteHandler.java   | 22 +++++++++++-
 .../graph/GraphToTypedInstanceMapper.java       | 19 ++++++----
 .../graph/v1/AtlasEntityChangeNotifier.java     | 38 +++++++++++++++++++-
 .../store/graph/v1/DeleteHandlerV1.java         | 34 +++++++++++++++++-
 .../java/org/apache/atlas/RequestContext.java   |  6 ++--
 .../typesystem/persistence/StructInstance.java  | 12 ++++---
 6 files changed, 112 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
index f0fef1f..b8d95f0 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
@@ -36,9 +36,11 @@ import org.apache.atlas.repository.graph.GraphHelper.VertexInfo;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.exception.NullRequiredAttributeException;
 import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.typesystem.types.AttributeInfo;
+import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.FieldMapping;
 import org.apache.atlas.typesystem.types.HierarchicalType;
@@ -93,7 +95,25 @@ public abstract class DeleteHandler {
            // Record all deletion candidate GUIDs in RequestContext
            // and gather deletion candidate vertices.
            for (VertexInfo vertexInfo : compositeVertices) {
-               requestContext.recordEntityDelete(vertexInfo.getGuid(), vertexInfo.getTypeName());
+               ClassType                   entityType = typeSystem.getDataType(ClassType.class, vertexInfo.getTypeName());
+               ITypedReferenceableInstance entity     = entityType.createInstance(new Id(guid, 0, vertexInfo.getTypeName()));
+
+               // populate unique attributes only
+               for (AttributeInfo attributeInfo : entityType.fieldMapping().fields.values()) {
+                   if (!attributeInfo.isUnique) {
+                       continue;
+                   }
+
+                   DataTypes.TypeCategory attrTypeCategory = attributeInfo.dataType().getTypeCategory();
+
+                   if (attrTypeCategory == DataTypes.TypeCategory.PRIMITIVE) {
+                       GraphToTypedInstanceMapper.mapVertexToPrimitive(vertexInfo.getVertex(), entity, attributeInfo);
+                   } else if (attrTypeCategory == DataTypes.TypeCategory.ENUM) {
+                       GraphToTypedInstanceMapper.mapVertexToEnum(vertexInfo.getVertex(), entity, attributeInfo);
+                   }
+               }
+
+               requestContext.recordEntityDelete(entity);
                deletionCandidateVertices.add(vertexInfo.getVertex());
            }
        }

http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
index 00fc57f..55f7076 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphToTypedInstanceMapper.java
@@ -154,12 +154,7 @@ public final class GraphToTypedInstanceMapper {
             break;  // add only if vertex has this attribute
 
         case ENUM:
-            Object propertyValue = GraphHelper.getProperty(instanceVertex, vertexPropertyName);
-            if (propertyValue == null) {
-                return;
-            }
-
-            typedInstance.set(attributeInfo.name, dataType.convert(propertyValue, Multiplicity.REQUIRED));
+            mapVertexToEnum(instanceVertex, typedInstance, attributeInfo);
             break;
 
         case ARRAY:
@@ -381,7 +376,7 @@ public final class GraphToTypedInstanceMapper {
         }
     }
 
-    private void mapVertexToPrimitive(AtlasVertex<?,?> instanceVertex, ITypedInstance typedInstance,
+    public static void mapVertexToPrimitive(AtlasVertex<?,?> instanceVertex, ITypedInstance typedInstance,
             AttributeInfo attributeInfo) throws AtlasException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("Adding primitive {} from vertex {}", attributeInfo, instanceVertex);
@@ -419,6 +414,16 @@ public final class GraphToTypedInstanceMapper {
         }
     }
 
+    public static void mapVertexToEnum(AtlasVertex<?,?> instanceVertex, ITypedInstance typedInstance, AttributeInfo attributeInfo) throws AtlasException {
+        final String vertexPropertyName = GraphHelper.getQualifiedFieldName(typedInstance, attributeInfo);
+        final Object propertyValue      = GraphHelper.getProperty(instanceVertex, vertexPropertyName);
+        if (propertyValue == null) {
+            return;
+        }
+
+        typedInstance.set(attributeInfo.name, attributeInfo.dataType().convert(propertyValue, Multiplicity.REQUIRED));
+    }
+
 
     public ITypedInstance getReferredEntity(String edgeId, IDataType<?> referredType) throws AtlasException {
         final AtlasEdge edge = getGraph().getEdge(edgeId);

http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityChangeNotifier.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityChangeNotifier.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityChangeNotifier.java
index ffcca05..1530525 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityChangeNotifier.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityChangeNotifier.java
@@ -33,9 +33,13 @@ import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
+import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.typesystem.persistence.ReferenceableInstance;
+import org.apache.atlas.typesystem.types.ClassType;
+import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.util.AtlasRepositoryConfiguration;
 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;
@@ -45,6 +49,7 @@ import javax.inject.Inject;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 
@@ -185,7 +190,38 @@ public class AtlasEntityChangeNotifier {
         // fail, since the entity vertex would already be gone. Hence the special handling for delete operation
         if (operation == EntityOperation.DELETE) {
             for (AtlasEntityHeader entity : entityHeaders) {
-                ret.add(new ReferenceableInstance(entity.getGuid(), entity.getTypeName()));
+                ReferenceableInstance instance = null;
+
+                if (MapUtils.isNotEmpty(entity.getAttributes())) {
+                    try {
+                        TypeSystem typeSystem = TypeSystem.getInstance();
+                        ClassType  entityType = (ClassType) typeSystem.getDataType(entity.getTypeName());
+
+                        instance = (ReferenceableInstance) entityType.createInstance(new Id(entity.getGuid(), 0, entity.getTypeName()));
+
+                        // set all attributes to null
+                        for (String attribute : instance.fieldMapping().fields.keySet()) {
+                            instance.setNull(attribute);
+                        }
+
+                        // set attributes from entity header
+                        for (Map.Entry<String, Object> attr : entity.getAttributes().entrySet()) {
+                            try {
+                                instance.set(attr.getKey(), attr.getValue());
+                            } catch (AtlasException e) {
+                                LOG.warn("failed to set attribute {}={} for entity: type={}, guid={}", attr.getKey(), attr.getValue(), entity.getTypeName(), entity.getGuid(), e);
+                            }
+                        }
+                    } catch (AtlasException e) {
+                        LOG.warn("failed to get type {}", entity.getTypeName(), e);
+                    }
+                }
+
+                if (instance == null) {
+                    instance = new ReferenceableInstance(entity.getGuid(), entity.getTypeName());
+                }
+
+                ret.add(instance);
             }
         } else {
             for (AtlasEntityHeader entityHeader : entityHeaders) {

http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/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 d4fdc25..ad8ab8e 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
@@ -18,6 +18,7 @@
 package org.apache.atlas.repository.store.graph.v1;
 
 
+import org.apache.atlas.AtlasConstants;
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RequestContextV1;
@@ -39,16 +40,19 @@ import org.apache.atlas.type.AtlasStructType;
 import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
 import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.commons.collections.MapUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
@@ -107,7 +111,35 @@ public abstract class DeleteHandlerV1 {
             // Record all deletion candidate GUIDs in RequestContext
             // and gather deletion candidate vertices.
             for (GraphHelper.VertexInfo vertexInfo : compositeVertices) {
-                requestContext.recordEntityDelete(new AtlasObjectId(vertexInfo.getGuid(), vertexInfo.getTypeName()));
+                AtlasVertex         vertex     = vertexInfo.getVertex();
+                AtlasEntityType     entityType = typeRegistry.getEntityTypeByName(vertexInfo.getTypeName());
+                Map<String, Object> attributes = null;
+
+                if (entityType != null && MapUtils.isNotEmpty(entityType.getUniqAttributes())) {
+                    attributes = new HashMap<>();
+
+                    for (AtlasAttribute attribute : entityType.getUniqAttributes().values()) {
+                        Object attrVal = GraphHelper.getProperty(vertex, attribute.getQualifiedName());
+
+                        if (attrVal != null) {
+                            attributes.put(attribute.getName(), attrVal);
+                        }
+                    }
+
+                    // include clusterName attribute as well, if it is defined in the entity-type
+                    AtlasAttribute attrClusterName = entityType.getAttribute(AtlasConstants.CLUSTER_NAME_ATTRIBUTE);
+
+                    if (attrClusterName != null) {
+                        Object clusterName = GraphHelper.getProperty(vertex, attrClusterName.getQualifiedName());
+
+                        if (clusterName != null) {
+                            attributes.put(attrClusterName.getName(), clusterName);
+                        }
+                    }
+                }
+
+                requestContext.recordEntityDelete(new AtlasObjectId(vertexInfo.getGuid(), vertexInfo.getTypeName(), attributes));
+
                 deletionCandidateVertices.add(vertexInfo.getVertex());
             }
         }

http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/server-api/src/main/java/org/apache/atlas/RequestContext.java
----------------------------------------------------------------------
diff --git a/server-api/src/main/java/org/apache/atlas/RequestContext.java b/server-api/src/main/java/org/apache/atlas/RequestContext.java
index 3e729b5..3b2b575 100644
--- a/server-api/src/main/java/org/apache/atlas/RequestContext.java
+++ b/server-api/src/main/java/org/apache/atlas/RequestContext.java
@@ -159,10 +159,8 @@ public class RequestContext {
         this.updatedEntityIds.add(entityId);
     }
 
-    public void recordEntityDelete(String entityId, String typeName) throws AtlasException {
-        ClassType type = typeSystem.getDataType(ClassType.class, typeName);
-        ITypedReferenceableInstance entity = type.createInstance(new Id(entityId, 0, typeName));
-        if (deletedEntityIds.add(entityId)) {
+    public void recordEntityDelete(ITypedReferenceableInstance entity) throws AtlasException {
+        if (deletedEntityIds.add(entity.getId()._getId())) {
             deletedEntities.add(entity);
         }
     }

http://git-wip-us.apache.org/repos/asf/atlas/blob/18063ef2/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java b/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
index 766d2d0..d939a9b 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/persistence/StructInstance.java
@@ -260,6 +260,8 @@ public class StructInstance implements ITypedStruct {
             bools[pos] = false;
         } else if (i.dataType() == DataTypes.STRING_TYPE) {
             strings[pos] = null;
+        } else if (i.dataType().getTypeCategory() == DataTypes.TypeCategory.ENUM) {
+            ints[pos] = 0;
         } else if (i.dataType().getTypeCategory() == DataTypes.TypeCategory.ARRAY) {
             arrays[pos] = null;
         } else if (i.dataType().getTypeCategory() == DataTypes.TypeCategory.MAP) {
@@ -268,8 +270,8 @@ public class StructInstance implements ITypedStruct {
             || i.dataType().getTypeCategory() == DataTypes.TypeCategory.TRAIT) {
             structs[pos] = null;
         } else if (i.dataType().getTypeCategory() == DataTypes.TypeCategory.CLASS) {
-                ids[pos] = null;
-                referenceables[pos] = null;
+            ids[pos] = null;
+            referenceables[pos] = null;
         } else {
             throw new AtlasException(String.format("Unknown datatype %s", i.dataType()));
         }
@@ -283,10 +285,10 @@ public class StructInstance implements ITypedStruct {
     public Map<String, Object> getValuesMap() throws AtlasException {
         Map<String, Object> m = new HashMap<>();
         for (String attr : fieldMapping.fields.keySet()) {
-//            int pos = fieldMapping.fieldNullPos.get(attr);
-//            if (  explicitSets[pos] ) {
+            int pos = fieldMapping.fieldNullPos.get(attr);
+            if (!nullFlags[pos]) {
                 m.put(attr, get(attr));
-//            }
+            }
         }
         return m;
     }