You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by dk...@apache.org on 2017/02/17 20:50:01 UTC

incubator-atlas git commit: ATLAS-1551 auto update of reverse references in V1 API

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 3a0865ad0 -> 852a71183


ATLAS-1551 auto update of reverse references in V1 API


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

Branch: refs/heads/master
Commit: 852a711831c2de2f2c9a469f01a872131649b38c
Parents: 3a0865a
Author: Dave Kantor <dk...@us.ibm.com>
Authored: Fri Feb 17 15:49:55 2017 -0500
Committer: Dave Kantor <dk...@us.ibm.com>
Committed: Fri Feb 17 15:49:55 2017 -0500

----------------------------------------------------------------------
 release-log.txt                                 |   1 +
 .../graph/TypedInstanceToGraphMapper.java       | 105 ++++++-
 ...hBackedMetadataRepositoryDeleteTestBase.java |  36 +--
 .../ReverseReferenceUpdateHardDeleteTest.java   |  52 ++++
 .../ReverseReferenceUpdateSoftDeleteTest.java   |  83 ++++++
 .../graph/ReverseReferenceUpdateTestBase.java   | 286 +++++++++++++++++++
 .../atlas/typesystem/types/utils/TypesUtil.java |  26 ++
 7 files changed, 560 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 0f9e0a1..9e17c54 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -9,6 +9,7 @@ ATLAS-1060 Add composite indexes for exact match performance improvements for al
 ATLAS-1127 Modify creation and modification timestamps to Date instead of Long(sumasai)
 
 ALL CHANGES:
+ATLAS-1551 auto update of reverse references in V1 API (dkantor)
 ATLAS-1565 Create EntityREST endpoints for delete operations (sarathkumarsubramanian via svimal2106)
 ATLAS-1547 Added tests for hard delete (mneethiraj)
 ATLAS-1546 (ATLAS-1546.3.patch)Hive hook should choose appropriate JAAS config if host uses kerberos ticket-cache (nixonrodrigues via kevalbhat)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
index e2bc028..40b7bf0 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
@@ -49,6 +49,7 @@ import org.apache.atlas.typesystem.persistence.ReferenceableInstance;
 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.DataTypes.TypeCategory;
 import org.apache.atlas.typesystem.types.EnumValue;
 import org.apache.atlas.typesystem.types.IDataType;
 import org.apache.atlas.typesystem.types.Multiplicity;
@@ -56,6 +57,7 @@ import org.apache.atlas.typesystem.types.ObjectGraphWalker;
 import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.TypeUtils;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.apache.atlas.util.AtlasRepositoryConfiguration;
 import org.apache.atlas.utils.MD5Utils;
 import org.slf4j.Logger;
@@ -249,6 +251,9 @@ public final class TypedInstanceToGraphMapper {
                     deleteHandler.deleteEdgeReference(currentEdge, attributeInfo.dataType().getTypeCategory(),
                             attributeInfo.isComposite, true);
                 }
+                if (attributeInfo.reverseAttributeName != null && newEdge != null) {
+                    addReverseReference(instanceVertex, attributeInfo.reverseAttributeName, newEdge);
+                }
                 break;
 
             case TRAIT:
@@ -421,24 +426,35 @@ public final class TypedInstanceToGraphMapper {
         List<Object> newElementsCreated = new ArrayList<>();
 
         if (!newAttributeEmpty) {
-            if (newElements != null && !newElements.isEmpty()) {
-                int index = 0;
-                for (; index < newElements.size(); index++) {
-                    Object currentElement = (currentElements != null && index < currentElements.size()) ?
-                            currentElements.get(index) : null;
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("Adding/updating element at position {}, current element {}, new element {}", index,
-                                currentElement, newElements.get(index));
-                    }
-
-                    Object newEntry = addOrUpdateCollectionEntry(instanceVertex, attributeInfo, elementType,
-                            newElements.get(index), currentElement, propertyName, operation);
-                    newElementsCreated.add(newEntry);
+            int index = 0;
+            for (; index < newElements.size(); index++) {
+                Object currentElement = (currentElements != null && index < currentElements.size()) ?
+                        currentElements.get(index) : null;
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Adding/updating element at position {}, current element {}, new element {}", index,
+                            currentElement, newElements.get(index));
                 }
+
+                Object newEntry = addOrUpdateCollectionEntry(instanceVertex, attributeInfo, elementType,
+                        newElements.get(index), currentElement, propertyName, operation);
+                newElementsCreated.add(newEntry);
             }
         }
 
         if(GraphHelper.isReference(elementType)) {
+            if (attributeInfo.reverseAttributeName != null && newElementsCreated.size() > 0) {
+                // Set/add the new reference value(s) on the reverse reference.
+                for (Object newElement : newElementsCreated) {
+                    if ((newElement instanceof AtlasEdge)) {
+                        AtlasEdge newEdge = (AtlasEdge) newElement;
+                        addReverseReference(instanceVertex, attributeInfo.reverseAttributeName, newEdge);
+                    }
+                    else {
+                        throw new AtlasException("Invalid array element type " + newElement.getClass().getName() + " - expected " + AtlasEdge.class.getName() +
+                            " for reference " + GraphHelper.getQualifiedFieldName(typedInstance, attributeInfo) + " on vertex " + GraphHelper.getVertexDetails(instanceVertex));
+                    }
+                }
+            }
 
             List<AtlasEdge> additionalEdges = removeUnusedEntries(instanceVertex, propertyName, (List)currentElements,
                     (List)newElementsCreated, elementType, attributeInfo);
@@ -866,4 +882,67 @@ public final class TypedInstanceToGraphMapper {
         }
         return new GuidMapping(mapping);
     }
+
+
+    private <V,E> void addReverseReference(AtlasVertex<V,E> vertex, String reverseAttributeName, AtlasEdge<V,E> edge)
+        throws AtlasException {
+
+        String typeName = GraphHelper.getTypeName(vertex);
+        Id id = GraphHelper.getIdFromVertex(typeName, vertex);
+
+        AtlasVertex<V, E> reverseVertex = edge.getInVertex();
+        String reverseTypeName = GraphHelper.getTypeName(reverseVertex);
+        Id reverseId = GraphHelper.getIdFromVertex(reverseTypeName, reverseVertex);
+        IDataType reverseType = typeSystem.getDataType(IDataType.class, reverseTypeName);
+        AttributeInfo reverseAttrInfo = TypesUtil.getFieldMapping(reverseType).fields.get(reverseAttributeName);
+        if (reverseAttrInfo.dataType().getTypeCategory() == TypeCategory.MAP) {
+            // If the reverse reference is a map, what would be used as the key?
+            // Not supporting automatic update of reverse map references.
+            LOG.debug("Automatic update of reverse map reference is not supported - reference = {}",
+                GraphHelper.getQualifiedFieldName(reverseType, reverseAttributeName));
+            return;
+        }
+
+        String propertyName = GraphHelper.getQualifiedFieldName(reverseType, reverseAttributeName);
+        String reverseEdgeLabel = GraphHelper.EDGE_LABEL_PREFIX + propertyName;
+        AtlasEdge<V, E> reverseEdge = graphHelper.getEdgeForLabel(reverseVertex, reverseEdgeLabel);
+
+        AtlasEdge<V, E> newEdge = null;
+        if (reverseEdge != null) {
+            newEdge = updateClassEdge(reverseVertex, reverseEdge, id, vertex, reverseAttrInfo, reverseEdgeLabel);
+        }
+        else {
+            newEdge = addClassEdge(reverseVertex, vertex, reverseEdgeLabel);
+        }
+
+        switch (reverseAttrInfo.dataType().getTypeCategory()) {
+        case CLASS:
+            if (reverseEdge != null && !reverseEdge.getId().toString().equals(newEdge.getId().toString())) {
+                // Disconnect old reference
+                deleteHandler.deleteEdgeReference(reverseEdge, reverseAttrInfo.dataType().getTypeCategory(),
+                    reverseAttrInfo.isComposite, true);
+            }
+            break;
+        case ARRAY:
+            // Add edge ID to property value
+            List<String> elements = reverseVertex.getProperty(propertyName, List.class);
+            if (elements == null) {
+                elements = new ArrayList<>();
+                elements.add(newEdge.getId().toString());
+                reverseVertex.setProperty(propertyName, elements);
+            }
+            else {
+               if (!elements.contains(newEdge.getId().toString())) {
+                    elements.add(newEdge.getId().toString());
+                    reverseVertex.setProperty(propertyName, elements);
+                }
+            }
+            break;
+        }
+
+        RequestContext requestContext = RequestContext.get();
+        GraphHelper.setProperty(reverseVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY,
+                requestContext.getRequestTime());
+        requestContext.recordEntityUpdate(reverseId._getId());
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteTestBase.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteTestBase.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteTestBase.java
index f18c308..f0424ab 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteTestBase.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteTestBase.java
@@ -24,11 +24,11 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasClient.EntityResult;
 import org.apache.atlas.AtlasException;
-import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.RequestContext;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.repository.Constants;
+import org.apache.atlas.CreateUpdateEntitiesResult;
 import org.apache.atlas.repository.MetadataRepository;
 import org.apache.atlas.repository.RepositoryException;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
@@ -63,10 +63,6 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -257,7 +253,8 @@ public abstract class GraphBackedMetadataRepositoryDeleteTestBase {
     private String createInstance(Referenceable entity) throws Exception {
         ClassType dataType = typeSystem.getDataType(ClassType.class, entity.getTypeName());
         ITypedReferenceableInstance instance = dataType.convert(entity, Multiplicity.REQUIRED);
-        List<String> results = repositoryService.createEntities(instance).getCreatedEntities();
+        CreateUpdateEntitiesResult result = repositoryService.createEntities(instance);
+        List<String> results = result.getCreatedEntities();
         return results.get(results.size() - 1);
     }
 
@@ -434,7 +431,7 @@ public abstract class GraphBackedMetadataRepositoryDeleteTestBase {
         Assert.assertNotNull(modificationTimestampPreUpdate);
 
         ITypedReferenceableInstance jane = repositoryService.getEntityDefinition(nameGuidMap.get("Jane"));
-        Id janeGuid = jane.getId();
+        Id janeId = jane.getId();
 
         // Update max's mentor reference to john.
         ClassType personType = typeSystem.getDataType(ClassType.class, "Person");
@@ -456,7 +453,7 @@ public abstract class GraphBackedMetadataRepositoryDeleteTestBase {
         Assert.assertTrue(creationTimestamp < modificationTimestampPostUpdate);
 
         // Update max's mentor reference to jane.
-        maxEntity.set("mentor", janeGuid);
+        maxEntity.set("mentor", janeId);
         entityResult = updatePartial(maxEntity);
         assertEquals(entityResult.getUpdateEntities().size(), 1);
         assertTrue(entityResult.getUpdateEntities().contains(maxGuid));
@@ -464,7 +461,7 @@ public abstract class GraphBackedMetadataRepositoryDeleteTestBase {
         // Verify the update was applied correctly - jane should now be max's mentor.
         max = repositoryService.getEntityDefinition(maxGuid);
         refTarget = (ITypedReferenceableInstance) max.get("mentor");
-        Assert.assertEquals(refTarget.getId()._getId(), janeGuid._getId());
+        Assert.assertEquals(refTarget.getId()._getId(), janeId._getId());
 
         // Verify modification timestamp was updated.
         vertex = GraphHelper.getInstance().getVertexForGUID(maxGuid);
@@ -473,21 +470,28 @@ public abstract class GraphBackedMetadataRepositoryDeleteTestBase {
         Assert.assertTrue(modificationTimestampPostUpdate < modificationTimestampPost2ndUpdate);
 
         ITypedReferenceableInstance julius = repositoryService.getEntityDefinition(nameGuidMap.get("Julius"));
-        Id juliusGuid = julius.getId();
+        Id juliusId = julius.getId();
         maxEntity = personType.createInstance(max.getId());
-        maxEntity.set("manager", juliusGuid);
+        maxEntity.set("manager", juliusId);
         entityResult = updatePartial(maxEntity);
-        //TODO ATLAS-499 should have updated julius' subordinates
-        assertEquals(entityResult.getUpdateEntities().size(), 2);
+        // Verify julius' subordinates were updated.
+        assertEquals(entityResult.getUpdateEntities().size(), 3);
         assertTrue(entityResult.getUpdateEntities().contains(maxGuid));
-        assertTrue(entityResult.getUpdateEntities().contains(janeGuid._getId()));
+        assertTrue(entityResult.getUpdateEntities().containsAll(Arrays.asList(maxGuid, janeId._getId(), juliusId._getId())));
 
         // Verify the update was applied correctly - julius should now be max's manager.
         max = repositoryService.getEntityDefinition(maxGuid);
         refTarget = (ITypedReferenceableInstance) max.get("manager");
-        Assert.assertEquals(refTarget.getId()._getId(), juliusGuid._getId());
+        Assert.assertEquals(refTarget.getId()._getId(), juliusId._getId());
+        Assert.assertEquals(refTarget.getId()._getId(), juliusId._getId());
+        julius = repositoryService.getEntityDefinition(nameGuidMap.get("Julius"));
+        Object object = julius.get("subordinates");
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 1);
+        Assert.assertTrue(refValues.contains(max.getId()));
 
-        assertTestUpdateEntity_MultiplicityOneNonCompositeReference(janeGuid._getId());
+        assertTestUpdateEntity_MultiplicityOneNonCompositeReference(janeId._getId());
     }
 
     protected abstract void assertTestUpdateEntity_MultiplicityOneNonCompositeReference(String janeGuid) throws Exception;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateHardDeleteTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateHardDeleteTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateHardDeleteTest.java
new file mode 100644
index 0000000..808305c
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateHardDeleteTest.java
@@ -0,0 +1,52 @@
+/**
+ * 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.graph;
+
+import java.util.List;
+
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
+import org.apache.atlas.typesystem.types.TypeSystem;
+import org.testng.Assert;
+
+
+/**
+ * Run tests in {@link ReverseReferenceUpdateTestBase} with hard delete enabled.
+ *
+ */
+public class ReverseReferenceUpdateHardDeleteTest extends ReverseReferenceUpdateTestBase {
+
+    @Override
+    DeleteHandler getDeleteHandler(TypeSystem typeSystem) {
+
+        return new HardDeleteHandler(typeSystem);
+    }
+
+    @Override
+    void assertTestOneToOneReference(Object refValue, ITypedReferenceableInstance expectedValue, ITypedReferenceableInstance referencingInstance) throws Exception {
+        // Verify reference was disconnected
+        Assert.assertNull(refValue);
+    }
+
+    @Override
+    void assertTestOneToManyReference(Object object, ITypedReferenceableInstance referencingInstance) {
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 1);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateSoftDeleteTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateSoftDeleteTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateSoftDeleteTest.java
new file mode 100644
index 0000000..6e20ddd
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateSoftDeleteTest.java
@@ -0,0 +1,83 @@
+/**
+ * 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.graph;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
+import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.typesystem.types.TypeSystem;
+import org.testng.Assert;
+
+
+/**
+ * Run tests in {@link ReverseReferenceUpdateTestBase} with soft delete enabled.
+ *
+ */
+public class ReverseReferenceUpdateSoftDeleteTest extends ReverseReferenceUpdateTestBase {
+
+    @Override
+    DeleteHandler getDeleteHandler(TypeSystem typeSystem) {
+
+        return new SoftDeleteHandler(typeSystem);
+    }
+
+    @Override
+    void assertTestOneToOneReference(Object actual, ITypedReferenceableInstance expectedValue, ITypedReferenceableInstance referencingInstance) throws Exception {
+        // Verify reference was not disconnected if soft deletes are enabled.
+        Assert.assertNotNull(actual);
+        Assert.assertTrue(actual instanceof ITypedReferenceableInstance);
+        ITypedReferenceableInstance referenceValue = (ITypedReferenceableInstance) actual;
+        Assert.assertEquals(referenceValue.getId()._getId(), expectedValue.getId()._getId());
+
+        //Verify reference edge was marked as DELETED.
+        AtlasVertex vertexForGUID = GraphHelper.getInstance().getVertexForGUID(referencingInstance.getId()._getId());
+        String edgeLabel = GraphHelper.getEdgeLabel(typeB, typeB.fieldMapping.fields.get("a"));
+        AtlasEdge edgeForLabel = GraphHelper.getInstance().getEdgeForLabel(vertexForGUID, edgeLabel);
+        String edgeState = edgeForLabel.getProperty(Constants.STATE_PROPERTY_KEY, String.class);
+        Assert.assertEquals(edgeState, Id.EntityState.DELETED.name());
+    }
+
+    @Override
+ void assertTestOneToManyReference(Object object, ITypedReferenceableInstance referencingInstance) throws Exception {
+        // Verify reference was not disconnected if soft deletes are enabled.
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 2);
+
+        // Verify that one of the reference edges is marked DELETED.
+        AtlasVertex vertexForGUID = GraphHelper.getInstance().getVertexForGUID(referencingInstance.getId()._getId());
+        String edgeLabel = GraphHelper.getEdgeLabel(typeB, typeB.fieldMapping.fields.get("manyA"));
+        Iterator<AtlasEdge> outGoingEdgesByLabel = GraphHelper.getInstance().getOutGoingEdgesByLabel(vertexForGUID, edgeLabel);
+        boolean found = false;
+        while (outGoingEdgesByLabel.hasNext()) {
+            AtlasEdge edge = outGoingEdgesByLabel.next();
+            String edgeState = edge.getProperty(Constants.STATE_PROPERTY_KEY, String.class);
+            if (edgeState.equals(Id.EntityState.DELETED.name())) {
+                found = true;
+                break;
+            }
+        }
+        Assert.assertTrue(found, "One edge for label " + edgeLabel + " should be marked " + Id.EntityState.DELETED.name());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateTestBase.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateTestBase.java b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateTestBase.java
new file mode 100644
index 0000000..b87f7fd
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/ReverseReferenceUpdateTestBase.java
@@ -0,0 +1,286 @@
+/**
+ * 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.graph;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.atlas.CreateUpdateEntitiesResult;
+import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.TestUtils;
+import org.apache.atlas.repository.MetadataRepository;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.AttributeDefinition;
+import org.apache.atlas.typesystem.types.ClassType;
+import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.EnumTypeDefinition;
+import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
+import org.apache.atlas.typesystem.types.Multiplicity;
+import org.apache.atlas.typesystem.types.StructTypeDefinition;
+import org.apache.atlas.typesystem.types.TraitType;
+import org.apache.atlas.typesystem.types.TypeSystem;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.inject.Inject;
+
+/**
+ * Verifies automatic update of reverse references
+ *
+ */
+@Guice(modules = RepositoryMetadataModule.class)
+public abstract class ReverseReferenceUpdateTestBase {
+
+    @Inject
+    MetadataRepository repositoryService;
+
+    private TypeSystem typeSystem;
+
+    protected ClassType typeA;
+    protected ClassType typeB;
+
+    abstract DeleteHandler getDeleteHandler(TypeSystem typeSystem);
+
+    abstract void assertTestOneToOneReference(Object actual, ITypedReferenceableInstance expectedValue, ITypedReferenceableInstance referencingInstance) throws Exception;
+    abstract void assertTestOneToManyReference(Object refValue, ITypedReferenceableInstance referencingInstance) throws Exception;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        typeSystem = TypeSystem.getInstance();
+        typeSystem.reset();
+
+        new GraphBackedSearchIndexer(new AtlasTypeRegistry());
+
+        HierarchicalTypeDefinition<ClassType> aDef = TypesUtil.createClassTypeDef("A", ImmutableSet.<String>of(),
+            TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
+            new AttributeDefinition("b", "B", Multiplicity.OPTIONAL, false, "a"), // 1-1
+            new AttributeDefinition("oneB", "B", Multiplicity.OPTIONAL, false, "manyA"), // 1-*
+            new AttributeDefinition("manyB", DataTypes.arrayTypeName("B"), Multiplicity.OPTIONAL,  false, "manyToManyA"), // *-*
+            new AttributeDefinition("map", DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
+                "B"), Multiplicity.OPTIONAL, false, "backToMap"));
+        HierarchicalTypeDefinition<ClassType> bDef = TypesUtil.createClassTypeDef("B", ImmutableSet.<String>of(),
+            TypesUtil.createRequiredAttrDef("name", DataTypes.STRING_TYPE),
+            new AttributeDefinition("a", "A", Multiplicity.OPTIONAL, false, "b"),
+            new AttributeDefinition("manyA", DataTypes.arrayTypeName("A"), Multiplicity.OPTIONAL, false, "oneB"),
+            new AttributeDefinition("manyToManyA", DataTypes.arrayTypeName("A"), Multiplicity.OPTIONAL, false, "manyB"),
+            new AttributeDefinition("backToMap", "A", Multiplicity.OPTIONAL, false, "map"));
+        TypesDef typesDef = TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(),
+            ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+            ImmutableList.of(aDef, bDef));
+        typeSystem.defineTypes(typesDef);
+        typeA = typeSystem.getDataType(ClassType.class, "A");
+        typeB = typeSystem.getDataType(ClassType.class, "B");
+
+        repositoryService = new GraphBackedMetadataRepository(getDeleteHandler(typeSystem));
+        repositoryService = TestUtils.addTransactionWrapper(repositoryService);
+    }
+
+    @BeforeMethod
+    public void setupContext() {
+        TestUtils.resetRequestContext();
+    }
+
+    @Test
+    public void testOneToOneReference() throws Exception {
+        ITypedReferenceableInstance a = typeA.createInstance();
+        a.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b1 = typeB.createInstance();
+        b1.setString("name", TestUtils.randomString());
+        a.set("b", b1);
+        // Create a.  This should also create b1 and set the reverse b1->a reference.
+        repositoryService.createEntities(a);
+        a = repositoryService.getEntityDefinition("A", "name", a.getString("name"));
+        b1 = repositoryService.getEntityDefinition("B", "name", b1.getString("name"));
+        Object object = a.get("b");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        ITypedReferenceableInstance refValue = (ITypedReferenceableInstance) object;
+        Assert.assertEquals(refValue.getId()._getId(), b1.getId()._getId());
+        object = b1.get("a");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        refValue = (ITypedReferenceableInstance) object;
+        Assert.assertEquals(refValue.getId()._getId(), a.getId()._getId());
+
+        ITypedReferenceableInstance b2 = typeB.createInstance();
+        b2.setString("name", TestUtils.randomString());
+        b2.set("a", a.getId());
+        // Create b2.  This should set the reverse a->b2 reference
+        // and disconnect b1->a.
+        repositoryService.createEntities(b2);
+        a = repositoryService.getEntityDefinition(a.getId()._getId());
+        b2 = repositoryService.getEntityDefinition("B", "name", b2.getString("name"));
+        object = a.get("b");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        refValue = (ITypedReferenceableInstance) object;
+        Assert.assertEquals(refValue.getId()._getId(), b2.getId()._getId());
+        object = b2.get("a");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        refValue = (ITypedReferenceableInstance) object;
+        Assert.assertEquals(refValue.getId()._getId(), a.getId()._getId());
+        // Verify b1->a was disconnected.
+        b1 = repositoryService.getEntityDefinition("B", "name", b1.getString("name"));
+        object = b1.get("a");
+        assertTestOneToOneReference(object, a, b1);
+    }
+
+    @Test
+    public void testOneToManyReference() throws Exception {
+        ITypedReferenceableInstance a1 = typeA.createInstance();
+        a1.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance a2 = typeA.createInstance();
+        a2.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b1 = typeB.createInstance();
+        b1.setString("name", TestUtils.randomString());
+        a1.set("oneB", b1);
+        ITypedReferenceableInstance b2 = typeB.createInstance();
+        b2.setString("name", TestUtils.randomString());
+        repositoryService.createEntities(a1, a2, b2);
+        a1 = repositoryService.getEntityDefinition("A", "name", a1.getString("name"));
+        a2 = repositoryService.getEntityDefinition("A", "name", a2.getString("name"));
+        b1 = repositoryService.getEntityDefinition("B", "name", b1.getString("name"));
+        b2 = repositoryService.getEntityDefinition("B", "name", b2.getString("name"));
+        Object object = b1.get("manyA");
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 1);
+        Assert.assertTrue(refValues.contains(a1.getId()));
+
+        a2.set("oneB", b1.getId());
+        repositoryService.updateEntities(a2);
+        b1 = repositoryService.getEntityDefinition(b1.getId()._getId());
+        object = b1.get("manyA");
+        Assert.assertTrue(object instanceof List);
+        refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 2);
+        Assert.assertTrue(refValues.containsAll(Arrays.asList(a1.getId(), a2.getId())));
+
+        b2.set("manyA", Collections.singletonList(a2));
+        repositoryService.updateEntities(b2);
+        a2 = repositoryService.getEntityDefinition("A", "name", a2.getString("name"));
+
+        // Verify reverse a2.oneB reference was set to b2.
+        object = a2.get("oneB");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        ITypedReferenceableInstance refValue = (ITypedReferenceableInstance) object;
+        Assert.assertEquals(refValue.getId()._getId(), b2.getId()._getId());
+
+        // Verify a2 was removed from b1.manyA reference list.
+        b1 = repositoryService.getEntityDefinition(b1.getId()._getId());
+        object = b1.get("manyA");
+        assertTestOneToManyReference(object, b1);
+    }
+
+    @Test
+    public void testManyToManyReference() throws Exception {
+        ITypedReferenceableInstance a1 = typeA.createInstance();
+        a1.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance a2 = typeA.createInstance();
+        a2.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b1 = typeB.createInstance();
+        b1.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b2 = typeB.createInstance();
+        b2.setString("name", TestUtils.randomString());
+        repositoryService.createEntities(a1, a2, b1, b2);
+        a1 = repositoryService.getEntityDefinition("A", "name", a1.getString("name"));
+        a2 = repositoryService.getEntityDefinition("A", "name", a2.getString("name"));
+        b1 = repositoryService.getEntityDefinition("B", "name", b1.getString("name"));
+        b2 = repositoryService.getEntityDefinition("B", "name", b2.getString("name"));
+
+        // Update a1 to add b1 to its manyB reference.
+        // This should update b1.manyToManyA.
+        a1.set("manyB", Arrays.asList(b1.getId()));
+        repositoryService.updateEntities(a1);
+
+        // Verify reverse b1.manyToManyA reference was updated.
+        b1 = repositoryService.getEntityDefinition(b1.getId()._getId());
+        Object object = b1.get("manyToManyA");
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>) object;
+        Assert.assertEquals(refValues.size(), 1);
+        Assert.assertTrue(refValues.contains(a1.getId()));
+    }
+
+    /**
+     * Auto-update of bi-directional references where one end is a map reference is
+     * not currently supported.  Verify that the auto-update is not applied in this case.
+     */
+    @Test
+    public void testMapReference() throws Exception {
+        ITypedReferenceableInstance a1 = typeA.createInstance();
+        a1.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance a2 = typeA.createInstance();
+        a2.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b1 = typeB.createInstance();
+        b1.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b2 = typeB.createInstance();
+        b2.setString("name", TestUtils.randomString());
+        repositoryService.createEntities(a1, a2, b1, b2);
+        a1 = repositoryService.getEntityDefinition("A", "name", a1.getString("name"));
+        a2 = repositoryService.getEntityDefinition("A", "name", a2.getString("name"));
+        b1 = repositoryService.getEntityDefinition("B", "name", b1.getString("name"));
+        b2 = repositoryService.getEntityDefinition("B", "name", b2.getString("name"));
+        a1.set("map", Collections.singletonMap("b1", b1));
+        repositoryService.updateEntities(a1);
+        // Verify reverse b1.manyToManyA reference was not updated.
+        b1 = repositoryService.getEntityDefinition(b1.getId()._getId());
+        Object object = b1.get("backToMap");
+        Assert.assertNull(object);
+    }
+
+    /**
+     * Verify that explicitly setting both ends of a reference
+     * does not cause duplicate entries due to auto-update of
+     * reverse reference.
+     */
+    @Test
+    public void testCallerHasSetBothEnds() throws Exception {
+        ITypedReferenceableInstance a = typeA.createInstance();
+        a.setString("name", TestUtils.randomString());
+        ITypedReferenceableInstance b1 = typeB.createInstance();
+        b1.setString("name", TestUtils.randomString());
+        // Set both sides of the reference.
+        a.set("oneB", b1);
+        b1.set("manyA", Collections.singletonList(a));
+
+        CreateUpdateEntitiesResult result = repositoryService.createEntities(a);
+        Map<String, String> guidAssignments = result.getGuidMapping().getGuidAssignments();
+        String aGuid = a.getId()._getId();
+        String b1Guid = guidAssignments.get(b1.getId()._getId());
+
+        a = repositoryService.getEntityDefinition(aGuid);
+        Object object = a.get("oneB");
+        Assert.assertTrue(object instanceof ITypedReferenceableInstance);
+        Assert.assertEquals(((ITypedReferenceableInstance)object).getId()._getId(), b1Guid);
+
+        b1 = repositoryService.getEntityDefinition(b1Guid);
+        object = b1.get("manyA");
+        Assert.assertTrue(object instanceof List);
+        List<ITypedReferenceableInstance> refValues = (List<ITypedReferenceableInstance>)object;
+        Assert.assertEquals(refValues.size(), 1);
+        Assert.assertEquals(refValues.get(0).getId()._getId(), aGuid);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/852a7118/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
index 5c977c1..f131458 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
@@ -28,9 +28,12 @@ import org.apache.atlas.typesystem.types.AttributeInfo;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.EnumTypeDefinition;
 import org.apache.atlas.typesystem.types.EnumValue;
+import org.apache.atlas.typesystem.types.FieldMapping;
+import org.apache.atlas.typesystem.types.HierarchicalType;
 import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
 import org.apache.atlas.typesystem.types.IDataType;
 import org.apache.atlas.typesystem.types.Multiplicity;
+import org.apache.atlas.typesystem.types.StructType;
 import org.apache.atlas.typesystem.types.StructTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.AtlasConstants;
@@ -129,4 +132,27 @@ public class TypesUtil {
             throw new RuntimeException(e);
         }
     }
+
+
+    /**
+     * Get the field mappings for the specified data type.
+     * Field mappings are only relevant for CLASS, TRAIT, and STRUCT types.
+     *
+     * @param type
+     * @return {@link FieldMapping} for the specified type
+     * @throws IllegalArgumentException if type is not a CLASS, TRAIT, or STRUCT type.
+     */
+    public static FieldMapping getFieldMapping(IDataType type) {
+        switch (type.getTypeCategory()) {
+        case CLASS:
+        case TRAIT:
+            return ((HierarchicalType)type).fieldMapping();
+
+        case STRUCT:
+            return ((StructType)type).fieldMapping();
+
+        default:
+            throw new IllegalArgumentException("Type " + type + " doesn't have any fields!");
+        }
+    }
 }