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/03/01 17:01:02 UTC

incubator-atlas git commit: ATLAS-1552: automatic update of inverse references in V2 code path (dkantor)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 214c1572a -> 3860c95d3


ATLAS-1552: automatic update of inverse references in V2 code path (dkantor)


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

Branch: refs/heads/master
Commit: 3860c95d35d0052eebc48e1ef6afce8b8d437576
Parents: 214c157
Author: Dave Kantor <dk...@us.ibm.com>
Authored: Wed Mar 1 10:10:37 2017 -0500
Committer: Dave Kantor <dk...@us.ibm.com>
Committed: Wed Mar 1 10:10:37 2017 -0500

----------------------------------------------------------------------
 .../org/apache/atlas/type/AtlasStructType.java  |  22 +-
 .../test/java/org/apache/atlas/TestUtilsV2.java |  29 +-
 .../apache/atlas/type/TestAtlasEntityType.java  |   5 +-
 release-log.txt                                 |   1 +
 .../store/graph/v1/AtlasStructDefStoreV1.java   |   2 +-
 .../store/graph/v1/DeleteHandlerV1.java         |  25 +-
 .../store/graph/v1/EntityGraphMapper.java       |  94 ++++-
 .../graph/v1/AtlasDeleteHandlerV1Test.java      |  22 +-
 .../InverseReferenceUpdateHardDeleteV1Test.java |  80 ++++
 .../InverseReferenceUpdateSoftDeleteV1Test.java |  83 ++++
 .../graph/v1/InverseReferenceUpdateV1Test.java  | 392 +++++++++++++++++++
 11 files changed, 723 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/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 bb7eef8..0eeaf9c 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasStructType.java
@@ -28,6 +28,7 @@ import org.apache.atlas.model.typedef.AtlasStructDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
 import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef.Cardinality;
+import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.StringUtils;
@@ -164,6 +165,16 @@ public class AtlasStructType extends AtlasType {
     @Override
     public void resolveReferencesPhase2(AtlasTypeRegistry typeRegistry) throws AtlasBaseException {
         super.resolveReferencesPhase2(typeRegistry);
+        for (AtlasAttribute attribute : allAttributes.values()) {
+            if (attribute.getInverseRefAttributeName() == null) {
+                continue;
+            }
+            // Set the inverse reference attribute.
+            AtlasType referencedType = typeRegistry.getType(attribute.getAttributeDef().getTypeName());
+            AtlasEntityType referencedEntityType = getReferencedEntityType(referencedType);
+            AtlasAttribute inverseReference = referencedEntityType.getAttribute(attribute.getInverseRefAttributeName());
+            attribute.setInverseRefAttribute(inverseReference);
+         }
     }
 
     @Override
@@ -587,7 +598,8 @@ public class AtlasStructType extends AtlasType {
         private final String            qualifiedName;
         private final String            vertexPropertyName;
         private final boolean           isOwnedRef;
-        private final String            inverseRefAttribute;
+        private final String            inverseRefAttributeName;
+        private AtlasAttribute          inverseRefAttribute;
 
         public AtlasAttribute(AtlasStructType definedInType, AtlasAttributeDef attrDef, AtlasType attributeType) {
             this.definedInType      = definedInType;
@@ -616,7 +628,7 @@ public class AtlasStructType extends AtlasType {
             }
 
             this.isOwnedRef          = isOwnedRef;
-            this.inverseRefAttribute = inverseRefAttribute;
+            this.inverseRefAttributeName = inverseRefAttribute;
         }
 
         public AtlasStructType getDefinedInType() { return definedInType; }
@@ -641,7 +653,11 @@ public class AtlasStructType extends AtlasType {
 
         public boolean isOwnedRef() { return isOwnedRef; }
 
-        public String getInverseRefAttribute() { return inverseRefAttribute; }
+        public String getInverseRefAttributeName() { return inverseRefAttributeName; }
+
+        public AtlasAttribute getInverseRefAttribute() { return inverseRefAttribute; }
+
+        public void setInverseRefAttribute(AtlasAttribute inverseAttr) { inverseRefAttribute = inverseAttr; };
 
         public static String encodePropertyKey(String key) {
             if (StringUtils.isBlank(key)) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/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 0b28bcf..7b1f2ad 100755
--- a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
+++ b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
@@ -23,15 +23,16 @@ import com.google.common.collect.ImmutableSet;
 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.typedef.AtlasBaseTypeDef;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
 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;
@@ -151,6 +152,32 @@ public final class TestUtilsV2 {
                 ImmutableList.of(deptTypeDef, personTypeDef, employeeTypeDef, managerTypeDef));
     }
 
+    public static AtlasTypesDef defineInverseReferenceTestTypes() {
+        AtlasEntityDef aDef = AtlasTypeUtil.createClassTypeDef("A", ImmutableSet.<String>of(),
+            AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
+            new AtlasAttributeDef("b", "B", true, Cardinality.SINGLE, 0, 1, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-1
+            new AtlasAttributeDef("oneB", "B", true, Cardinality.SINGLE, 0, 1, false, false, Collections.<AtlasConstraintDef>emptyList()), // 1-*
+            new AtlasAttributeDef("manyB", AtlasBaseTypeDef.getArrayTypeName("B"), true, Cardinality.SINGLE, 0, 1, false, false, Collections.<AtlasConstraintDef>emptyList()),
+            new AtlasAttributeDef("mapToB", AtlasBaseTypeDef.getMapTypeName("string", "B"), true, Cardinality.SINGLE, 0, 1, false, false,
+                Collections.<AtlasConstraintDef>singletonList(new AtlasConstraintDef(
+                AtlasConstraintDef.CONSTRAINT_TYPE_INVERSE_REF, Collections.<String, Object>singletonMap(AtlasConstraintDef.CONSTRAINT_PARAM_ATTRIBUTE, "mappedFromA"))))); // *-*
+
+        AtlasEntityDef bDef = AtlasTypeUtil.createClassTypeDef("B", ImmutableSet.<String>of(),
+            AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
+            new AtlasAttributeDef("a", "A", true, Cardinality.SINGLE, 0, 1, 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,
+                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,
+                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, Collections.<AtlasConstraintDef>emptyList()));
+
+        return new AtlasTypesDef(ImmutableList.<AtlasEnumDef>of(), ImmutableList.<AtlasStructDef>of(), ImmutableList.<AtlasClassificationDef>of(), ImmutableList.<AtlasEntityDef>of(aDef, bDef));
+    }
+
     public static AtlasTypesDef defineValidUpdatedDeptEmployeeTypes() {
         String _description = "_description_updated";
         AtlasEnumDef orgLevelEnum =

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/intg/src/test/java/org/apache/atlas/type/TestAtlasEntityType.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/type/TestAtlasEntityType.java b/intg/src/test/java/org/apache/atlas/type/TestAtlasEntityType.java
index 2d71d41..3c53c02 100644
--- a/intg/src/test/java/org/apache/atlas/type/TestAtlasEntityType.java
+++ b/intg/src/test/java/org/apache/atlas/type/TestAtlasEntityType.java
@@ -149,9 +149,10 @@ public class TestAtlasEntityType {
             AtlasEntityType typeColumn = ttr.getEntityTypeByName(TYPE_COLUMN);
 
             assertTrue(typeTable.getAttribute(ATTR_COLUMNS).isOwnedRef());
-            assertNull(typeTable.getAttribute(ATTR_COLUMNS).getInverseRefAttribute());
+            assertNull(typeTable.getAttribute(ATTR_COLUMNS).getInverseRefAttributeName());
             assertFalse(typeColumn.getAttribute(ATTR_TABLE).isOwnedRef());
-            assertEquals(typeColumn.getAttribute(ATTR_TABLE).getInverseRefAttribute(), ATTR_COLUMNS);
+            assertEquals(typeColumn.getAttribute(ATTR_TABLE).getInverseRefAttributeName(), ATTR_COLUMNS);
+            assertEquals(typeColumn.getAttribute(ATTR_TABLE).getInverseRefAttribute(), typeTable.getAttribute(ATTR_COLUMNS));
 
             commit = true;
         } catch (AtlasBaseException excp) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 3148c70..b9d391f 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-1552: automatic update of inverse references in V2 code path (dkantor)
 ATLAS-1603: fix to handle null value for object_id type attributes (mneethiraj via kevalbhatt)
 ATLAS 1607: notify listeners on classification addition/deletion (sarathkumarsubramanian via mneethiraj)
 ATLAS-1606: introduced query provider to handle Gremlin version specific queries (apoorvnaik via mneethiraj)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
index 3f87e69..6803f1a 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasStructDefStoreV1.java
@@ -476,7 +476,7 @@ public class AtlasStructDefStoreV1 extends AtlasAbstractDefStoreV1 implements At
         attribInfo.put("isUnique", attributeDef.getIsUnique());
         attribInfo.put("isIndexable", attributeDef.getIsIndexable());
         attribInfo.put("isComposite", attribute.isOwnedRef());
-        attribInfo.put("reverseAttributeName", attribute.getInverseRefAttribute());
+        attribInfo.put("reverseAttributeName", attribute.getInverseRefAttributeName());
 
         final int lower;
         final int upper;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/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 f1efe46..d4fdc25 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
@@ -378,13 +378,13 @@ public abstract class DeleteHandlerV1 {
         }
     }
 
-    protected AtlasAttributeDef getAttributeForEdge(String edgeLabel) throws AtlasBaseException {
+    protected AtlasAttribute getAttributeForEdge(String edgeLabel) throws AtlasBaseException {
         AtlasEdgeLabel atlasEdgeLabel = new AtlasEdgeLabel(edgeLabel);
 
         AtlasType parentType = typeRegistry.getType(atlasEdgeLabel.getTypeName());
         AtlasStructType parentStructType = (AtlasStructType) parentType;
 
-        return parentStructType.getAttributeDef(atlasEdgeLabel.getAttributeName());
+        return parentStructType.getAttribute(atlasEdgeLabel.getAttributeName());
     }
 
     protected abstract void _deleteVertex(AtlasVertex instanceVertex, boolean force);
@@ -395,12 +395,12 @@ public abstract class DeleteHandlerV1 {
      * Deletes the edge between outvertex and inVertex. The edge is for attribute attributeName of outVertex
      * @param outVertex
      * @param inVertex
-     * @param attributeName
+     * @param attribute
      * @throws AtlasException
      */
-    protected void deleteEdgeBetweenVertices(AtlasVertex outVertex, AtlasVertex inVertex, String attributeName) throws AtlasBaseException {
+    protected void deleteEdgeBetweenVertices(AtlasVertex outVertex, AtlasVertex inVertex, AtlasAttribute attribute) throws AtlasBaseException {
         LOG.debug("Removing edge from {} to {} with attribute name {}", string(outVertex), string(inVertex),
-            attributeName);
+            attribute.getName());
         String typeName = GraphHelper.getTypeName(outVertex);
         String outId = GraphHelper.getGuid(outVertex);
 
@@ -413,12 +413,11 @@ public abstract class DeleteHandlerV1 {
         }
 
         AtlasStructType parentType = (AtlasStructType) typeRegistry.getType(typeName);
-        String propertyName = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(parentType, attributeName);
+        String propertyName = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(parentType, attribute.getName());
         String edgeLabel = EDGE_LABEL_PREFIX + propertyName;
         AtlasEdge edge = null;
 
-        AtlasAttribute attribute = parentType.getAttribute(attributeName);
-        AtlasAttributeDef attrDef = parentType.getAttributeDef(attributeName);
+        AtlasAttributeDef attrDef = attribute.getAttributeDef();
         AtlasType attrType = attribute.getAttributeType();
 
         switch (attrType.getTypeCategory()) {
@@ -466,7 +465,7 @@ public abstract class DeleteHandlerV1 {
                             //for example, when table is deleted, process still references the table
                             //but when column is deleted, table will not reference the deleted column
                             LOG.debug("Removing edge {} from the array attribute {}", string(elementEdge),
-                                attributeName);
+                                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.
@@ -505,7 +504,7 @@ public abstract class DeleteHandlerV1 {
                             if (shouldUpdateInverseReferences) {
                                 //remove this key
                                 LOG.debug("Removing edge {}, key {} from the map attribute {}", string(mapEdge), key,
-                                    attributeName);
+                                    attribute.getName());
                                 keys.remove(key);
                                 GraphHelper.setProperty(outVertex, propertyName, keys);
                                 GraphHelper.setProperty(outVertex, keyPropertyName, null);
@@ -523,7 +522,7 @@ public abstract class DeleteHandlerV1 {
 
         default:
             throw new IllegalStateException("There can't be an edge from " + GraphHelper.getVertexDetails(outVertex) + " to "
-                + GraphHelper.getVertexDetails(inVertex) + " with attribute name " + attributeName + " which is not class/array/map attribute. found " + attrType.getTypeCategory().name());
+                + GraphHelper.getVertexDetails(inVertex) + " with attribute name " + attribute.getName() + " which is not class/array/map attribute. found " + attrType.getTypeCategory().name());
         }
 
         if (edge != null) {
@@ -544,9 +543,9 @@ public abstract class DeleteHandlerV1 {
             AtlasEntity.Status edgeState = AtlasGraphUtilsV1.getState(edge);
             if (edgeState == AtlasEntity.Status.ACTIVE) {
                 //Delete only the active edge references
-                AtlasAttributeDef attribute = getAttributeForEdge(edge.getLabel());
+                AtlasAttribute attribute = getAttributeForEdge(edge.getLabel());
                 //TODO use delete edge instead??
-                deleteEdgeBetweenVertices(edge.getOutVertex(), edge.getInVertex(), attribute.getName());
+                deleteEdgeBetweenVertices(edge.getOutVertex(), edge.getInVertex(), attribute);
             }
         }
         _deleteVertex(instanceVertex, force);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/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 e2b82cc..29bda93 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
@@ -159,7 +159,12 @@ public class EntityGraphMapper {
         }
 
         for (AtlasObjectId id : req.getUpdatedEntityIds()) {
-            resp.addEntity(UPDATE, constructHeader(id));
+            if (isPartialUpdate) {
+                resp.addEntity(PARTIAL_UPDATE, constructHeader(id));
+            }
+            else {
+                resp.addEntity(UPDATE, constructHeader(id));
+            }
         }
 
         return resp;
@@ -294,8 +299,11 @@ public class EntityGraphMapper {
                     ctx.setExistingEdge(edge);
 
                     newEdge = mapObjectIdValue(ctx, context);
+                    if (ctx.getAttribute().getInverseRefAttribute() != null) {
+                        // Update the inverse reference on the target entity
+                        addInverseReference(ctx, ctx.getAttribute().getInverseRefAttribute(), newEdge);
+                    }
                 }
-
                 if (currentEdge != null && !currentEdge.equals(newEdge)) {
                     deleteHandler.deleteEdgeReference(currentEdge, ctx.getAttrType().getTypeCategory(), ctx.getAttribute().isOwnedRef(), true);
                 }
@@ -314,6 +322,68 @@ public class EntityGraphMapper {
         }
     }
 
+    private void addInverseReference(AttributeMutationContext ctx, AtlasAttribute inverseAttribute, AtlasEdge edge) throws AtlasBaseException {
+
+        AtlasStructType inverseType = inverseAttribute.getDefinedInType();
+        String propertyName = AtlasGraphUtilsV1.getQualifiedAttributePropertyKey(inverseType, inverseAttribute.getName());
+        AtlasVertex vertex = edge.getOutVertex();
+        AtlasVertex inverseVertex = edge.getInVertex();
+        String inverseEdgeLabel = AtlasGraphUtilsV1.getEdgeLabel(propertyName);
+        AtlasEdge inverseEdge = graphHelper.getEdgeForLabel(inverseVertex, inverseEdgeLabel);
+
+        AtlasEdge newEdge;
+        try {
+            newEdge = graphHelper.getOrCreateEdge(inverseVertex, vertex, inverseEdgeLabel);
+        } catch (RepositoryException e) {
+            throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, e);
+        }
+
+        boolean inverseUpdated = true;
+        switch (inverseAttribute.getAttributeType().getTypeCategory()) {
+        case OBJECT_ID_TYPE:
+            if (inverseEdge != null) {
+                if (!inverseEdge.equals(newEdge)) {
+                    // Disconnect old reference
+                    deleteHandler.deleteEdgeReference(inverseEdge, inverseAttribute.getAttributeType().getTypeCategory(),
+                        inverseAttribute.isOwnedRef(), true);
+                }
+                else {
+                    // Edge already exists for this attribute between these vertices.
+                    inverseUpdated = false;
+                }
+            }
+            break;
+        case ARRAY:
+            // Add edge ID to property value
+            List<String> elements = inverseVertex.getProperty(propertyName, List.class);
+            if (elements == null) {
+                elements = new ArrayList<>();
+                elements.add(newEdge.getId().toString());
+                inverseVertex.setProperty(propertyName, elements);
+            }
+            else {
+               if (!elements.contains(newEdge.getId().toString())) {
+                    elements.add(newEdge.getId().toString());
+                    inverseVertex.setProperty(propertyName, elements);
+               }
+               else {
+                   // Property value list already contains the edge ID.
+                   inverseUpdated = false;
+               }
+            }
+            break;
+        default:
+            break;
+        }
+
+        if (inverseUpdated) {
+            updateModificationMetadata(inverseVertex);
+            AtlasObjectId inverseEntityId = new AtlasObjectId(AtlasGraphUtilsV1.getIdFromVertex(inverseVertex), inverseType.getTypeName());
+            RequestContextV1.get().recordEntityUpdate(inverseEntityId);
+        }
+    }
+
+
     private Object mapPrimitiveValue(AttributeMutationContext ctx) {
         AtlasGraphUtilsV1.setProperty(ctx.getReferringVertex(), ctx.getVertexProperty(), ctx.getValue());
 
@@ -429,6 +499,8 @@ public class EntityGraphMapper {
             }
 
             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));
@@ -441,6 +513,13 @@ public class EntityGraphMapper {
                     setMapValueProperty(mapType.getValueType(), ctx.getReferringVertex(), propertyName, 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;
+                        addInverseReference(mapCtx, inverseRefAttribute, newEdge);
+                    }
                 }
             }
 
@@ -476,7 +555,8 @@ public class EntityGraphMapper {
         AtlasArrayType arrType         = (AtlasArrayType) attribute.getAttributeType();
         AtlasType      elementType     = arrType.getElementType();
         List<Object>   currentElements = getArrayElementsProperty(elementType, ctx.getReferringVertex(), ctx.getVertexProperty());
-
+        boolean isReference = AtlasGraphUtilsV1.isReference(elementType);
+        AtlasAttribute inverseRefAttribute = attribute.getInverseRefAttribute();
         List<Object> newElementsCreated = new ArrayList<>();
 
         if (CollectionUtils.isNotEmpty(newElements)) {
@@ -486,12 +566,16 @@ public class EntityGraphMapper {
                                                                                      ctx.getVertexProperty(), elementType, existingEdge);
 
                 Object newEntry = mapCollectionElementsToVertex(arrCtx, context);
-
+                if (isReference && newEntry instanceof AtlasEdge && inverseRefAttribute != null) {
+                    // Update the inverse reference value.
+                    AtlasEdge newEdge = (AtlasEdge) newEntry;
+                    addInverseReference(arrCtx, inverseRefAttribute, newEdge);
+                }
                 newElementsCreated.add(newEntry);
             }
         }
 
-        if (AtlasGraphUtilsV1.isReference(elementType)) {
+        if (isReference) {
             List<AtlasEdge> additionalEdges = removeUnusedArrayEntries(attribute, (List) currentElements, (List) newElementsCreated);
             newElementsCreated.addAll(additionalEdges);
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasDeleteHandlerV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasDeleteHandlerV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasDeleteHandlerV1Test.java
index aab0d3e..f1c4e48 100644
--- a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasDeleteHandlerV1Test.java
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/AtlasDeleteHandlerV1Test.java
@@ -406,20 +406,28 @@ public abstract class AtlasDeleteHandlerV1Test {
         Assert.assertTrue(modificationTimestampPostUpdate < modificationTimestampPost2ndUpdate);
 
         ITypedReferenceableInstance julius = metadataService.getEntityDefinition(juliusEmployeeCreated.getGuid());
-        Id juliusGuid = julius.getId();
+        Id juliusId = julius.getId();
 
         init();
         maxEmployee.setAttribute("manager", AtlasTypeUtil.getAtlasObjectId(juliusEmployeeCreated));
         entityResult = entityStore.createOrUpdate(new AtlasEntityStream(maxEmployee), false);
-        //TODO ATLAS-499 should have updated julius' subordinates
-        assertEquals(entityResult.getUpdatedEntities().size(), 2);
-        assertTrue(extractGuids(entityResult.getUpdatedEntities()).contains(maxGuid));
-        assertTrue(extractGuids(entityResult.getUpdatedEntities()).contains(janeEmployeeCreated.getGuid()));
+        assertEquals(entityResult.getUpdatedEntities().size(), 3);
+        List<String> updatedGuids = extractGuids(entityResult.getUpdatedEntities());
+        assertTrue(updatedGuids.contains(maxGuid));
+        assertTrue(updatedGuids.contains(janeEmployeeCreated.getGuid()));
+        // Should have updated julius to add max in subordinates list.
+        assertTrue(updatedGuids.contains(juliusEmployeeCreated.getGuid()));
 
-        // Verify the update was applied correctly - julius should now be max's manager.
+        // Verify the update was applied correctly - julius should now be max's manager and max should be julius' subordinate.
         max = metadataService.getEntityDefinition(maxGuid);
         refTarget = (ITypedReferenceableInstance) max.get("manager");
-        Assert.assertEquals(refTarget.getId()._getId(), juliusGuid._getId());
+        Assert.assertEquals(refTarget.getId()._getId(), juliusId._getId());
+        julius = metadataService.getEntityDefinition(juliusId._getId());
+        Object value = julius.get("subordinates");
+        Assert.assertTrue(value instanceof List);
+        List<ITypedReferenceableInstance> refList = (List<ITypedReferenceableInstance>) value;
+        Assert.assertEquals(refList.size(), 1);
+        Assert.assertEquals(refList.get(0).getId()._getId(), maxGuid);
 
         assertTestUpdateEntity_MultiplicityOneNonCompositeReference(janeEmployeeCreated.getGuid());
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateHardDeleteV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateHardDeleteV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateHardDeleteV1Test.java
new file mode 100644
index 0000000..8b5eaff
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateHardDeleteV1Test.java
@@ -0,0 +1,80 @@
+/**
+ * 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 static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.util.Map;
+
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.type.AtlasTypeUtil;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Inverse reference update test with {@link HardDeleteHandlerV1}
+ */
+public class InverseReferenceUpdateHardDeleteV1Test extends InverseReferenceUpdateV1Test {
+
+    @Override
+    protected DeleteHandlerV1 getDeleteHandler(AtlasTypeRegistry typeRegistry) {
+
+        return new HardDeleteHandlerV1(typeRegistry);
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonComposite_OneToMany(AtlasEntity jane) throws Exception {
+
+        // Max should have been removed from the subordinates list, leaving only John.
+        verifyReferenceList(jane, "subordinates", ImmutableList.of(nameIdMap.get("John")));
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonCompositeManyToOne(AtlasEntity a1, AtlasEntity a2, AtlasEntity a3, AtlasEntity b) {
+
+        verifyReferenceValue(a1, "oneB", null);
+
+        verifyReferenceValue(a2, "oneB", null);
+
+        verifyReferenceList(b, "manyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a3)));
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonComposite_OneToOne(AtlasEntity a1, AtlasEntity b) {
+
+        verifyReferenceValue(a1, "b", null);
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_Map(AtlasEntity a1, AtlasEntity b1,
+        AtlasEntity b2, AtlasEntity b3) {
+
+        Object value = a1.getAttribute("mapToB");
+        assertTrue(value instanceof Map);
+        Map<String, AtlasObjectId> refMap = (Map<String, AtlasObjectId>) value;
+        assertEquals(refMap.size(), 1);
+        AtlasObjectId referencedEntityId = refMap.get("b3");
+        assertEquals(referencedEntityId, AtlasTypeUtil.getAtlasObjectId(b3));
+        verifyReferenceValue(b1, "mappedFromA", null);
+        verifyReferenceValue(b2, "mappedFromA", null);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateSoftDeleteV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateSoftDeleteV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateSoftDeleteV1Test.java
new file mode 100644
index 0000000..0b3c5db
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateSoftDeleteV1Test.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.store.graph.v1;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.util.Map;
+
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.type.AtlasTypeUtil;
+
+import com.google.common.collect.ImmutableList;
+
+
+/**
+ * Inverse reference update test with {@link SoftDeleteHandlerV1}
+ */
+public class InverseReferenceUpdateSoftDeleteV1Test extends InverseReferenceUpdateV1Test {
+
+    @Override
+    protected DeleteHandlerV1 getDeleteHandler(AtlasTypeRegistry typeRegistry) {
+
+        return new SoftDeleteHandlerV1(typeRegistry);
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonComposite_OneToMany(AtlasEntity jane)
+        throws Exception {
+
+        // Max is still in the subordinates list, as the edge still exists with state DELETED
+        verifyReferenceList(jane, "subordinates", ImmutableList.of(nameIdMap.get("John"), nameIdMap.get("Max")));
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonCompositeManyToOne(AtlasEntity a1,
+        AtlasEntity a2, AtlasEntity a3, AtlasEntity b) {
+
+        verifyReferenceValue(a1, "oneB", b.getGuid());
+
+        verifyReferenceValue(a2, "oneB", b.getGuid());
+
+        verifyReferenceList(b, "manyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a1), AtlasTypeUtil.getAtlasObjectId(a2), AtlasTypeUtil.getAtlasObjectId(a3)));
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_NonComposite_OneToOne(AtlasEntity a1, AtlasEntity b) {
+
+        verifyReferenceValue(a1, "b", b.getGuid());
+    }
+
+    @Override
+    protected void verify_testInverseReferenceAutoUpdate_Map(AtlasEntity a1, AtlasEntity b1,
+        AtlasEntity b2, AtlasEntity b3) {
+
+        Object value = a1.getAttribute("mapToB");
+        assertTrue(value instanceof Map);
+        Map<String, AtlasObjectId> refMap = (Map<String, AtlasObjectId>) value;
+        assertEquals(refMap.size(), 3);
+        AtlasObjectId referencedEntityId = refMap.get("b3");
+        assertEquals(referencedEntityId, AtlasTypeUtil.getAtlasObjectId(b3));
+        verifyReferenceValue(b1, "mappedFromA", a1.getGuid());
+        verifyReferenceValue(b2, "mappedFromA", a1.getGuid());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/3860c95d/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateV1Test.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateV1Test.java b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateV1Test.java
new file mode 100644
index 0000000..a198e61
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/repository/store/graph/v1/InverseReferenceUpdateV1Test.java
@@ -0,0 +1,392 @@
+/**
+ * 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 static org.apache.atlas.TestUtils.NAME;
+import static org.mockito.Mockito.mock;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
+
+import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.RequestContextV1;
+import org.apache.atlas.TestUtils;
+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.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+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.services.MetadataService;
+import org.apache.atlas.store.AtlasTypeDefStore;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.type.AtlasTypeUtil;
+import org.testng.annotations.AfterClass;
+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.ImmutableMap;
+
+/**
+ * Test automatic inverse reference updating in V1 (V2?) code path.
+ *
+ */
+@Guice(modules = RepositoryMetadataModule.class)
+public abstract class InverseReferenceUpdateV1Test {
+
+    @Inject
+    AtlasTypeRegistry typeRegistry;
+
+    @Inject
+    AtlasTypeDefStore typeDefStore;
+
+    AtlasEntityStore entityStore;
+
+    @Inject
+    MetadataService metadataService;
+
+    @Inject
+    DeleteHandlerV1 deleteHandler;
+
+    private AtlasEntitiesWithExtInfo deptEntity;
+
+    AtlasEntityChangeNotifier mockChangeNotifier = mock(AtlasEntityChangeNotifier.class);
+
+    protected Map<String, AtlasObjectId> nameIdMap = new HashMap<>();
+
+    protected abstract DeleteHandlerV1 getDeleteHandler(AtlasTypeRegistry typeRegistry);
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        metadataService = TestUtils.addSessionCleanupWrapper(metadataService);
+        new GraphBackedSearchIndexer(typeRegistry);
+
+        AtlasTypesDef[] testTypesDefs = new AtlasTypesDef[] { TestUtilsV2.defineDeptEmployeeTypes(),
+                                                              TestUtilsV2.defineInverseReferenceTestTypes()
+                                                            };
+
+        for (AtlasTypesDef typesDef : testTypesDefs) {
+            AtlasTypesDef typesToCreate = AtlasTypeDefStoreInitializer.getTypesToCreate(typesDef, typeRegistry);
+
+            if (!typesToCreate.isEmpty()) {
+                typeDefStore.createTypesDef(typesToCreate);
+            }
+        }
+
+        deptEntity = TestUtilsV2.createDeptEg2();
+        DeleteHandlerV1 deleteHandler = getDeleteHandler(typeRegistry);
+        entityStore = new AtlasEntityStoreV1(deleteHandler, typeRegistry, mockChangeNotifier);
+        init();
+        EntityMutationResponse response = entityStore.createOrUpdate(new AtlasEntityStream(deptEntity), false);
+        for (AtlasEntityHeader entityHeader : response.getCreatedEntities()) {
+            nameIdMap.put((String)entityHeader.getAttribute(NAME), AtlasTypeUtil.getAtlasObjectId(entityHeader));
+        }
+    }
+
+    @AfterClass
+    public void clear() {
+        AtlasGraphProvider.cleanup();
+    }
+
+    @BeforeMethod
+    public void init() throws Exception {
+        RequestContextV1.clear();
+    }
+
+    @Test
+    public void testInverseReferenceAutoUpdate_NonComposite_OneToMany() throws Exception {
+        AtlasObjectId juliusId = nameIdMap.get("Julius");
+
+        // Change Max's Employee.manager reference to Julius and apply the change as a partial update.
+        // This should also update Julius to add Max to the inverse Manager.subordinates reference.
+        AtlasEntity maxEntityForUpdate = new AtlasEntity(TestUtilsV2.EMPLOYEE_TYPE);
+        maxEntityForUpdate.setAttribute("manager", juliusId);
+        AtlasEntityType employeeType = typeRegistry.getEntityTypeByName(TestUtilsV2.EMPLOYEE_TYPE);
+        Map<String, Object> uniqAttributes = Collections.<String, Object>singletonMap("name", "Max");
+        EntityMutationResponse updateResponse = entityStore.updateByUniqueAttributes(employeeType, uniqAttributes , maxEntityForUpdate);
+        List<AtlasEntityHeader> partialUpdatedEntities = updateResponse.getPartialUpdatedEntities();
+        // 3 entities should have been updated:
+        // * Max to change the Employee.manager reference
+        // * Julius to add Max to Manager.subordinates
+        // * Jane to remove Max from Manager.subordinates
+        assertEquals(partialUpdatedEntities.size(), 3);
+
+        AtlasObjectId maxId = nameIdMap.get("Max");
+        String janeGuid = nameIdMap.get("Jane").getGuid();
+        AtlasEntitiesWithExtInfo storedEntities = entityStore.getByIds(ImmutableList.of(maxId.getGuid(), juliusId.getGuid(), janeGuid));
+        AtlasEntity storedEntity = storedEntities.getEntity(maxId.getGuid());
+        verifyReferenceValue(storedEntity, "manager", juliusId.getGuid());
+        storedEntity = storedEntities.getEntity(juliusId.getGuid());
+        verifyReferenceList(storedEntity, "subordinates", ImmutableList.of(maxId));
+        storedEntity = storedEntities.getEntity(janeGuid);
+        verify_testInverseReferenceAutoUpdate_NonComposite_OneToMany(storedEntity);
+    }
+
+    protected abstract void verify_testInverseReferenceAutoUpdate_NonComposite_OneToMany(AtlasEntity jane) throws Exception;
+
+    @Test
+    public void testInverseReferenceAutoUpdate_NonCompositeManyToOne() throws Exception {
+        AtlasEntityType bType = typeRegistry.getEntityTypeByName("B");
+        AtlasEntity a1 = new AtlasEntity("A");
+        a1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity a2 = new AtlasEntity("A");
+        a2.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity a3 = new AtlasEntity("A");
+        a3.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b = new AtlasEntity("B");
+
+        b.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntitiesWithExtInfo atlasEntitiesWithExtInfo = new AtlasEntitiesWithExtInfo();
+        atlasEntitiesWithExtInfo.addEntity(a1);
+        atlasEntitiesWithExtInfo.addEntity(a2);
+        atlasEntitiesWithExtInfo.addEntity(a3);
+        atlasEntitiesWithExtInfo.addEntity(b);
+        AtlasEntityStream entityStream = new AtlasEntityStream(atlasEntitiesWithExtInfo);
+        EntityMutationResponse response = entityStore.createOrUpdate(entityStream , false);
+
+        AtlasEntity bForPartialUpdate = new AtlasEntity("B");
+        bForPartialUpdate.setAttribute("manyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a1), AtlasTypeUtil.getAtlasObjectId(a2)));
+        init();
+        response = entityStore.updateByUniqueAttributes(bType, Collections.<String, Object>singletonMap(NAME, b.getAttribute(NAME)), bForPartialUpdate);
+        List<AtlasEntityHeader> partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 3 entities were updated:
+        // * set b.manyA reference to a1 and a2
+        // * set inverse a1.oneB reference to b
+        // * set inverse a2.oneB reference to b
+        assertEquals(partialUpdatedEntities.size(), 3);
+        AtlasEntitiesWithExtInfo storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), a2.getGuid(), b.getGuid()));
+        AtlasEntity storedEntity = storedEntities.getEntity(a1.getGuid());
+        verifyReferenceValue(storedEntity, "oneB", b.getGuid());
+
+        storedEntity = storedEntities.getEntity(a2.getGuid());
+        verifyReferenceValue(storedEntity, "oneB", b.getGuid());
+
+        storedEntity = storedEntities.getEntity(b.getGuid());
+        verifyReferenceList(storedEntity, "manyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a1), AtlasTypeUtil.getAtlasObjectId(a2)));
+
+        bForPartialUpdate.setAttribute("manyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a3)));
+        init();
+        response = entityStore.updateByUniqueAttributes(bType, Collections.<String, Object>singletonMap(NAME, b.getAttribute(NAME)), bForPartialUpdate);
+        partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 4 entities were updated:
+        // * set b.manyA reference to a3
+        // * set inverse a3.oneB reference to b
+        // * disconnect inverse a1.oneB reference to b
+        // * disconnect inverse a2.oneB reference to b
+        assertEquals(partialUpdatedEntities.size(), 4);
+
+        init();
+        storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), a2.getGuid(), a3.getGuid(), b.getGuid()));
+        verifyReferenceValue(storedEntities.getEntity(a3.getGuid()), "oneB", b.getGuid());
+
+        verify_testInverseReferenceAutoUpdate_NonCompositeManyToOne(storedEntities.getEntity(a1.getGuid()), storedEntities.getEntity(a2.getGuid()),
+            storedEntities.getEntity(a3.getGuid()), storedEntities.getEntity(b.getGuid()));
+    }
+
+    protected abstract void verify_testInverseReferenceAutoUpdate_NonCompositeManyToOne(AtlasEntity a1, AtlasEntity a2, AtlasEntity a3, AtlasEntity b);
+
+    @Test
+    public void testInverseReferenceAutoUpdate_NonComposite_OneToOne() throws Exception {
+        AtlasEntityType bType = typeRegistry.getEntityTypeByName("B");
+        AtlasEntity a1 = new AtlasEntity("A");
+        a1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity a2 = new AtlasEntity("A");
+        a2.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b = new AtlasEntity("B");
+        b.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntitiesWithExtInfo atlasEntitiesWithExtInfo = new AtlasEntitiesWithExtInfo();
+        atlasEntitiesWithExtInfo.addEntity(a1);
+        atlasEntitiesWithExtInfo.addEntity(a2);
+        atlasEntitiesWithExtInfo.addEntity(b);
+        AtlasEntityStream entityStream = new AtlasEntityStream(atlasEntitiesWithExtInfo);
+        EntityMutationResponse response = entityStore.createOrUpdate(entityStream , false);
+
+        AtlasEntity bForPartialUpdate = new AtlasEntity("B");
+        bForPartialUpdate.setAttribute("a", AtlasTypeUtil.getAtlasObjectId(a1));
+        init();
+        response = entityStore.updateByUniqueAttributes(bType, Collections.<String, Object>singletonMap(NAME, b.getAttribute(NAME)), bForPartialUpdate);
+        List<AtlasEntityHeader> partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 2 entities were updated:
+        // * set b.a reference to a1
+        // * set inverse a1.b reference to b
+        assertEquals(partialUpdatedEntities.size(), 2);
+        AtlasEntitiesWithExtInfo storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), b.getGuid()));
+        AtlasEntity storedEntity = storedEntities.getEntity(a1.getGuid());
+        verifyReferenceValue(storedEntity, "b", b.getGuid());
+        storedEntity = storedEntities.getEntity(b.getGuid());
+        verifyReferenceValue(storedEntity, "a", a1.getGuid());
+
+        // Update b.a to reference a2.
+        bForPartialUpdate.setAttribute("a", AtlasTypeUtil.getAtlasObjectId(a2));
+        init();
+        response = entityStore.updateByUniqueAttributes(bType, Collections.<String, Object>singletonMap(NAME, b.getAttribute(NAME)), bForPartialUpdate);
+        partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 3 entities were updated:
+        // * set b.a reference to a2
+        // * set a2.b reference to b
+        // * disconnect a1.b reference
+        assertEquals(partialUpdatedEntities.size(), 3);
+        storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), a2.getGuid(), b.getGuid()));
+        storedEntity = storedEntities.getEntity(a2.getGuid());
+        verifyReferenceValue(storedEntity, "b", b.getGuid());
+        storedEntity = storedEntities.getEntity(b.getGuid());
+        verifyReferenceValue(storedEntity, "a", a2.getGuid());
+        storedEntity = storedEntities.getEntity(a1.getGuid());
+        Object refValue = storedEntity.getAttribute("b");
+        verify_testInverseReferenceAutoUpdate_NonComposite_OneToOne(storedEntities.getEntity(a1.getGuid()), storedEntities.getEntity(b.getGuid()));
+    }
+
+    protected abstract void verify_testInverseReferenceAutoUpdate_NonComposite_OneToOne(AtlasEntity a1, AtlasEntity b);
+
+    @Test
+    public void testInverseReferenceAutoUpdate_NonComposite_ManyToMany() throws Exception {
+        AtlasEntityType bType = typeRegistry.getEntityTypeByName("B");
+        AtlasEntity a1 = new AtlasEntity("A");
+        a1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity a2 = new AtlasEntity("A");
+        a2.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity a3 = new AtlasEntity("A");
+        a3.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b1 = new AtlasEntity("B");
+        b1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b2 = new AtlasEntity("B");
+        b2.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntitiesWithExtInfo atlasEntitiesWithExtInfo = new AtlasEntitiesWithExtInfo();
+        atlasEntitiesWithExtInfo.addEntity(a1);
+        atlasEntitiesWithExtInfo.addEntity(a2);
+        atlasEntitiesWithExtInfo.addEntity(a3);
+        atlasEntitiesWithExtInfo.addEntity(b1);
+        atlasEntitiesWithExtInfo.addEntity(b2);
+        AtlasEntityStream entityStream = new AtlasEntityStream(atlasEntitiesWithExtInfo);
+        EntityMutationResponse response = entityStore.createOrUpdate(entityStream , false);
+
+        AtlasEntity b1ForPartialUpdate = new AtlasEntity("B");
+        b1ForPartialUpdate.setAttribute("manyToManyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a1), AtlasTypeUtil.getAtlasObjectId(a2)));
+        init();
+        response = entityStore.updateByUniqueAttributes(bType, Collections.<String, Object>singletonMap(NAME, b1.getAttribute(NAME)), b1ForPartialUpdate);
+        List<AtlasEntityHeader> partialUpdatedEntities = response.getPartialUpdatedEntities();
+        assertEquals(partialUpdatedEntities.size(), 3);
+        AtlasEntitiesWithExtInfo storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), a2.getGuid(), b1.getGuid()));
+        AtlasEntity storedEntity = storedEntities.getEntity(b1.getGuid());
+        verifyReferenceList(storedEntity, "manyToManyA", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(a1), AtlasTypeUtil.getAtlasObjectId(a2)));
+        storedEntity = storedEntities.getEntity(a1.getGuid());
+        verifyReferenceList(storedEntity, "manyB", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(b1)));
+        storedEntity = storedEntities.getEntity(a2.getGuid());
+        verifyReferenceList(storedEntity, "manyB", ImmutableList.of(AtlasTypeUtil.getAtlasObjectId(b1)));
+    }
+
+    @Test
+    public void testInverseReferenceAutoUpdate_Map() throws Exception {
+        AtlasEntity a1 = new AtlasEntity("A");
+        a1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b1 = new AtlasEntity("B");
+        b1.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b2 = new AtlasEntity("B");
+        b2.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntity b3 = new AtlasEntity("B");
+        b3.setAttribute(NAME, TestUtils.randomString());
+        AtlasEntitiesWithExtInfo atlasEntitiesWithExtInfo = new AtlasEntitiesWithExtInfo();
+        atlasEntitiesWithExtInfo.addEntity(a1);
+        atlasEntitiesWithExtInfo.addEntity(b1);
+        atlasEntitiesWithExtInfo.addEntity(b2);
+        atlasEntitiesWithExtInfo.addEntity(b3);
+        AtlasEntityStream entityStream = new AtlasEntityStream(atlasEntitiesWithExtInfo);
+        EntityMutationResponse response = entityStore.createOrUpdate(entityStream , false);
+
+        AtlasEntityType aType = typeRegistry.getEntityTypeByName("A");
+        AtlasEntity aForPartialUpdate = new AtlasEntity("A");
+        aForPartialUpdate.setAttribute("mapToB", ImmutableMap.<String, AtlasObjectId>of("b1", AtlasTypeUtil.getAtlasObjectId(b1), "b2", AtlasTypeUtil.getAtlasObjectId(b2)));
+        init();
+        response = entityStore.updateByUniqueAttributes(aType, Collections.<String, Object>singletonMap(NAME, a1.getAttribute(NAME)), aForPartialUpdate);
+        List<AtlasEntityHeader> partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 3 entities were updated:
+        // * set a1.mapToB to "b1"->b1, "b2"->b2
+        // * set b1.mappedFromA to a1
+        // * set b2.mappedFromA to a1
+        assertEquals(partialUpdatedEntities.size(), 3);
+        AtlasEntitiesWithExtInfo storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), b2.getGuid(), b1.getGuid()));
+        AtlasEntity storedEntity = storedEntities.getEntity(a1.getGuid());
+        Object value = storedEntity.getAttribute("mapToB");
+        assertTrue(value instanceof Map);
+        Map<String, AtlasObjectId> refMap = (Map<String, AtlasObjectId>) value;
+        assertEquals(refMap.size(), 2);
+        AtlasObjectId referencedEntityId = refMap.get("b1");
+        assertEquals(referencedEntityId, AtlasTypeUtil.getAtlasObjectId(b1));
+        referencedEntityId = refMap.get("b2");
+        assertEquals(referencedEntityId, AtlasTypeUtil.getAtlasObjectId(b2));
+        storedEntity = storedEntities.getEntity(b1.getGuid());
+        verifyReferenceValue(storedEntity, "mappedFromA", a1.getGuid());
+        storedEntity = storedEntities.getEntity(b2.getGuid());
+        verifyReferenceValue(storedEntity, "mappedFromA", a1.getGuid());
+
+        aForPartialUpdate.setAttribute("mapToB", ImmutableMap.<String, AtlasObjectId>of("b3", AtlasTypeUtil.getAtlasObjectId(b3)));
+        init();
+        response = entityStore.updateByUniqueAttributes(aType, Collections.<String, Object>singletonMap(NAME, a1.getAttribute(NAME)), aForPartialUpdate);
+        partialUpdatedEntities = response.getPartialUpdatedEntities();
+        // Verify 4 entities were updated:
+        // * set a1.mapToB to "b3"->b3
+        // * set b3.mappedFromA to a1
+        // * disconnect b1.mappedFromA
+        // * disconnect b2.mappedFromA
+        assertEquals(partialUpdatedEntities.size(), 4);
+        storedEntities = entityStore.getByIds(ImmutableList.of(a1.getGuid(), b2.getGuid(), b1.getGuid(), b3.getGuid()));
+        AtlasEntity storedB3 = storedEntities.getEntity(b3.getGuid());
+        verifyReferenceValue(storedB3, "mappedFromA", a1.getGuid());
+        verify_testInverseReferenceAutoUpdate_Map(storedEntities.getEntity(a1.getGuid()), storedEntities.getEntity(b1.getGuid()), storedEntities.getEntity(b2.getGuid()), storedB3);
+    }
+
+    protected abstract void verify_testInverseReferenceAutoUpdate_Map(AtlasEntity a1, AtlasEntity b1, AtlasEntity b2, AtlasEntity b3);
+
+    protected void verifyReferenceValue(AtlasEntity entity, String refName, String expectedGuid) {
+        Object refValue = entity.getAttribute(refName);
+        if (expectedGuid == null) {
+            assertNull(refValue);
+        }
+        else {
+            assertTrue(refValue instanceof AtlasObjectId);
+            AtlasObjectId referencedObjectId = (AtlasObjectId) refValue;
+            assertEquals(referencedObjectId.getGuid(), expectedGuid);
+        }
+    }
+
+    protected void verifyReferenceList(AtlasEntity entity, String refName, List<AtlasObjectId> expectedValues) {
+        Object refValue = entity.getAttribute(refName);
+        assertTrue(refValue instanceof List);
+        List<AtlasObjectId> refList = (List<AtlasObjectId>) refValue;
+        assertEquals(refList.size(), expectedValues.size());
+        if (expectedValues.size() > 0) {
+            assertTrue(refList.containsAll(expectedValues));
+        }
+    }
+}