You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/02/25 00:21:29 UTC

incubator-atlas git commit: ATLAS-1564: EntityResource v1 updated to route its calls to v2 EntityREST

Repository: incubator-atlas
Updated Branches:
  refs/heads/master c7900f255 -> 765d556cc


ATLAS-1564: EntityResource v1 updated to route its calls to v2 EntityREST

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


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

Branch: refs/heads/master
Commit: 765d556cce4f276d295d06cdb513fc3b9b00dc51
Parents: c7900f2
Author: Sarath Subramanian <ss...@hortonworks.com>
Authored: Fri Feb 17 16:04:00 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Fri Feb 24 16:21:19 2017 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/atlas/AtlasClient.java |   8 +-
 .../java/org/apache/atlas/AtlasErrorCode.java   |   1 +
 .../converters/AtlasFormatConverter.java        |   6 +-
 .../converters/AtlasInstanceConverter.java      | 113 ++++++++++++++--
 .../store/graph/AtlasEntityStore.java           |  20 +++
 .../store/graph/v1/AtlasEntityStoreV1.java      |  79 ++++++++++-
 .../atlas/services/DefaultMetadataService.java  |   3 +-
 .../apache/atlas/services/MetadataService.java  |   8 ++
 .../atlas/web/resources/EntityResource.java     | 134 ++++++++++++++-----
 .../org/apache/atlas/web/rest/EntityREST.java   |  19 ++-
 .../atlas/web/resources/EntityResourceTest.java |  53 +++++---
 11 files changed, 377 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/client/src/main/java/org/apache/atlas/AtlasClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/atlas/AtlasClient.java b/client/src/main/java/org/apache/atlas/AtlasClient.java
index 13896ce..909bd23 100755
--- a/client/src/main/java/org/apache/atlas/AtlasClient.java
+++ b/client/src/main/java/org/apache/atlas/AtlasClient.java
@@ -270,12 +270,12 @@ public class AtlasClient extends AtlasBaseClient {
         }
 
         public EntityResult(List<String> created, List<String> updated, List<String> deleted) {
-            add(OP_CREATED, created);
-            add(OP_UPDATED, updated);
-            add(OP_DELETED, deleted);
+            set(OP_CREATED, created);
+            set(OP_UPDATED, updated);
+            set(OP_DELETED, deleted);
         }
 
-        private void add(String type, List<String> list) {
+        public void set(String type, List<String> list) {
             if (list != null && list.size() > 0) {
                 entities.put(type, list);
             }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
index ca3023a..542b659 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -66,6 +66,7 @@ public enum AtlasErrorCode {
     SYSTEM_TYPE(400, "ATLAS40035E", "{0} is a System-type"),
     INVALID_STRUCT_VALUE(400, "ATLAS40036E", "not a valid struct value {0}"),
     INSTANCE_LINEAGE_INVALID_PARAMS(400, "ATLAS40037E", "Invalid lineage query parameters passed {0}: {1}"),
+    ATTRIBUTE_UPDATE_NOT_SUPPORTED(400, "ATLAS40038E", "{0}.{1} : attribute update not supported"),
 
     // All Not found enums go here
     TYPE_NAME_NOT_FOUND(404, "ATLAS4041E", "Given typename {0} was invalid"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/repository/src/main/java/org/apache/atlas/repository/converters/AtlasFormatConverter.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/converters/AtlasFormatConverter.java b/repository/src/main/java/org/apache/atlas/repository/converters/AtlasFormatConverter.java
index 9d0d7f4..a6d43da 100644
--- a/repository/src/main/java/org/apache/atlas/repository/converters/AtlasFormatConverter.java
+++ b/repository/src/main/java/org/apache/atlas/repository/converters/AtlasFormatConverter.java
@@ -34,7 +34,7 @@ public interface AtlasFormatConverter {
 
     class ConverterContext {
 
-        private AtlasEntity.AtlasEntitiesWithExtInfo entities = null;
+        private AtlasEntitiesWithExtInfo entities = null;
 
         public void addEntity(AtlasEntity entity) {
             if (entities == null) {
@@ -61,6 +61,10 @@ public interface AtlasFormatConverter {
         public boolean entityExists(String guid) { return entities != null && entities.hasEntity(guid); }
 
         public AtlasEntitiesWithExtInfo getEntities() {
+            if (entities != null) {
+                entities.compact();
+            }
+
             return entities;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/repository/src/main/java/org/apache/atlas/repository/converters/AtlasInstanceConverter.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/converters/AtlasInstanceConverter.java b/repository/src/main/java/org/apache/atlas/repository/converters/AtlasInstanceConverter.java
index e14fafb..621b32f 100644
--- a/repository/src/main/java/org/apache/atlas/repository/converters/AtlasInstanceConverter.java
+++ b/repository/src/main/java/org/apache/atlas/repository/converters/AtlasInstanceConverter.java
@@ -20,6 +20,7 @@ package org.apache.atlas.repository.converters;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasClient.EntityResult;
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.CreateUpdateEntitiesResult;
@@ -27,9 +28,11 @@ import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.TypeCategory;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.atlas.model.instance.EntityMutationResponse;
 import org.apache.atlas.model.instance.EntityMutations;
+import org.apache.atlas.model.instance.EntityMutations.EntityOperation;
 import org.apache.atlas.model.instance.GuidMapping;
 import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.type.AtlasClassificationType;
@@ -45,12 +48,17 @@ import org.apache.atlas.typesystem.Struct;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.exception.TraitNotFoundException;
 import org.apache.atlas.typesystem.exception.TypeNotFoundException;
+import org.apache.atlas.repository.converters.AtlasFormatConverter.ConverterContext;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 @Singleton
 public class AtlasInstanceConverter {
@@ -100,10 +108,10 @@ public class AtlasInstanceConverter {
         }
     }
 
-    public Referenceable getReferenceable(AtlasEntity entity, final AtlasFormatConverter.ConverterContext ctx) throws AtlasBaseException {
+    public Referenceable getReferenceable(AtlasEntity entity, final ConverterContext ctx) throws AtlasBaseException {
         AtlasFormatConverter converter  = instanceFormatters.getConverter(TypeCategory.ENTITY);
         AtlasType            entityType = typeRegistry.getType(entity.getTypeName());
-        Referenceable        ref        = (Referenceable)converter.fromV2ToV1(entity, entityType, ctx);
+        Referenceable        ref        = (Referenceable) converter.fromV2ToV1(entity, entityType, ctx);
 
         return ref;
     }
@@ -111,7 +119,7 @@ public class AtlasInstanceConverter {
     public ITypedStruct getTrait(AtlasClassification classification) throws AtlasBaseException {
         AtlasFormatConverter converter          = instanceFormatters.getConverter(TypeCategory.CLASSIFICATION);
         AtlasType            classificationType = typeRegistry.getType(classification.getTypeName());
-        Struct               trait               = (Struct)converter.fromV2ToV1(classification, classificationType, new AtlasFormatConverter.ConverterContext());
+        Struct               trait               = (Struct)converter.fromV2ToV1(classification, classificationType, new ConverterContext());
 
         try {
             return metadataService.createTraitInstance(trait);
@@ -132,18 +140,17 @@ public class AtlasInstanceConverter {
         return ret;
     }
 
-    public AtlasEntity.AtlasEntitiesWithExtInfo toAtlasEntity(IReferenceableInstance referenceable) throws AtlasBaseException {
-
+    public AtlasEntitiesWithExtInfo toAtlasEntity(IReferenceableInstance referenceable) throws AtlasBaseException {
         AtlasEntityFormatConverter converter  = (AtlasEntityFormatConverter) instanceFormatters.getConverter(TypeCategory.ENTITY);
-        AtlasEntityType      entityType = typeRegistry.getEntityTypeByName(referenceable.getTypeName());
+        AtlasEntityType            entityType = typeRegistry.getEntityTypeByName(referenceable.getTypeName());
 
         if (entityType == null) {
             throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), referenceable.getTypeName());
         }
 
-        AtlasFormatConverter.ConverterContext ctx = new AtlasFormatConverter.ConverterContext();
+        ConverterContext ctx    = new ConverterContext();
+        AtlasEntity      entity = converter.fromV1ToV2(referenceable, entityType, ctx);
 
-        AtlasEntity entity = converter.fromV1ToV2(referenceable, entityType, ctx);
         ctx.addEntity(entity);
 
         return ctx.getEntities();
@@ -212,6 +219,31 @@ public class AtlasInstanceConverter {
         return context.getEntities();
     }
 
+    public AtlasEntitiesWithExtInfo toAtlasEntities(String entitiesJson) throws AtlasBaseException, AtlasException {
+        ITypedReferenceableInstance[] referenceables = metadataService.deserializeClassInstances(entitiesJson);
+        AtlasEntityFormatConverter    converter      = (AtlasEntityFormatConverter) instanceFormatters.getConverter(TypeCategory.ENTITY);
+        ConverterContext              context        = new ConverterContext();
+        AtlasEntitiesWithExtInfo      ret            = null;
+
+        if (referenceables != null) {
+            for (IReferenceableInstance referenceable : referenceables) {
+                AtlasEntityType entityType = typeRegistry.getEntityTypeByName(referenceable.getTypeName());
+
+                if (entityType == null) {
+                    throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), referenceable.getTypeName());
+                }
+
+                AtlasEntity entity = converter.fromV1ToV2(referenceable, entityType, context);
+
+                context.addEntity(entity);
+            }
+
+            ret = context.getEntities();
+        }
+
+        return ret;
+    }
+
     private AtlasEntity fromV1toV2Entity(Referenceable referenceable, AtlasFormatConverter.ConverterContext context) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> fromV1toV2Entity");
@@ -227,4 +259,69 @@ public class AtlasInstanceConverter {
         return entity;
     }
 
+    public CreateUpdateEntitiesResult toCreateUpdateEntitiesResult(EntityMutationResponse reponse) {
+        CreateUpdateEntitiesResult ret = null;
+
+        if (reponse != null) {
+            Map<EntityOperation, List<AtlasEntityHeader>> mutatedEntities = reponse.getMutatedEntities();
+            Map<String, String>                           guidAssignments = reponse.getGuidAssignments();
+
+            ret = new CreateUpdateEntitiesResult();
+
+            if (MapUtils.isNotEmpty(guidAssignments)) {
+                ret.setGuidMapping(new GuidMapping(guidAssignments));
+            }
+
+            if (MapUtils.isNotEmpty(mutatedEntities)) {
+                EntityResult entityResult = new EntityResult();
+
+                for (Map.Entry<EntityOperation, List<AtlasEntityHeader>> e : mutatedEntities.entrySet()) {
+                    switch (e.getKey()) {
+                        case CREATE:
+                            List<AtlasEntityHeader> createdEntities = mutatedEntities.get(EntityOperation.CREATE);
+                            if (CollectionUtils.isNotEmpty(createdEntities)) {
+                                entityResult.set(EntityResult.OP_CREATED, getGuids(createdEntities));
+                            }
+                            break;
+                        case UPDATE:
+                            List<AtlasEntityHeader> updatedEntities = mutatedEntities.get(EntityOperation.UPDATE);
+                            if (CollectionUtils.isNotEmpty(updatedEntities)) {
+                                entityResult.set(EntityResult.OP_UPDATED, getGuids(updatedEntities));
+                            }
+                            break;
+                        case PARTIAL_UPDATE:
+                            List<AtlasEntityHeader> partialUpdatedEntities = mutatedEntities.get(EntityOperation.PARTIAL_UPDATE);
+                            if (CollectionUtils.isNotEmpty(partialUpdatedEntities)) {
+                                entityResult.set(EntityResult.OP_UPDATED, getGuids(partialUpdatedEntities));
+                            }
+                            break;
+                        case DELETE:
+                            List<AtlasEntityHeader> deletedEntities = mutatedEntities.get(EntityOperation.DELETE);
+                            if (CollectionUtils.isNotEmpty(deletedEntities)) {
+                                entityResult.set(EntityResult.OP_DELETED, getGuids(deletedEntities));
+                            }
+                            break;
+                    }
+
+                }
+
+                ret.setEntityResult(entityResult);
+            }
+        }
+
+        return ret;
+    }
+
+    public List<String> getGuids(List<AtlasEntityHeader> entities) {
+        List<String> ret = null;
+
+        if (CollectionUtils.isNotEmpty(entities)) {
+            ret = new ArrayList<>();
+            for (AtlasEntityHeader entity : entities) {
+                ret.add(entity.getGuid());
+            }
+        }
+
+        return ret;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityStore.java
index 3a037cc..6c372b3 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityStore.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasEntityStore.java
@@ -89,6 +89,26 @@ public interface AtlasEntityStore {
                                                     AtlasEntity entity) throws AtlasBaseException;
 
     /**
+     * Partial update a single entity using its guid.
+     * @param entityType type of the entity
+     * @param guid Entity guid
+     * @return EntityMutationResponse details of the updates performed by this call
+     * @throws AtlasBaseException
+     *
+     */
+    EntityMutationResponse updateByGuid(AtlasEntityType entityType, String guid, AtlasEntity entity) throws AtlasBaseException;
+
+    /**
+     * Partial update entities attribute using its guid.
+     * @param guid Entity guid
+     * @param attrName attribute name to be updated
+     * @param attrValue updated attribute value
+     * @return EntityMutationResponse details of the updates performed by this call
+     * @throws AtlasBaseException
+     */
+    EntityMutationResponse updateEntityAttributeByGuid(String guid, String attrName, Object attrValue) throws AtlasBaseException;
+
+    /**
      * Delete an entity by its guid
      * @param guid
      * @return

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
index bbfc3e5..c0355d9 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasEntityStoreV1.java
@@ -21,15 +21,16 @@ package org.apache.atlas.repository.store.graph.v1;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.AtlasException;
 import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.RequestContextV1;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.impexp.AtlasImportResult;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
-import org.apache.atlas.model.instance.AtlasEntityHeader;
-import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
 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.EntityMutationResponse;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
@@ -37,17 +38,28 @@ import org.apache.atlas.repository.store.graph.AtlasEntityStore;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscovery;
 import org.apache.atlas.repository.store.graph.EntityGraphDiscoveryContext;
 import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
+import org.apache.atlas.type.AtlasType;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.type.AtlasTypeUtil;
+import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
-import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.*;
+import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.DELETE;
+import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.UPDATE;
 
 
 @Singleton
@@ -245,6 +257,65 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
         return createOrUpdate(new AtlasEntityStream(updatedEntity), true);
     }
 
+    @Override
+    @GraphTransaction
+    public EntityMutationResponse updateByGuid(AtlasEntityType entityType, String guid, AtlasEntity updatedEntity)
+            throws AtlasBaseException {
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> updateByUniqueAttributes({}, {})", entityType.getTypeName(), guid);
+        }
+
+        if (updatedEntity == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "no entity to update.");
+        }
+
+        updatedEntity.setGuid(guid);
+
+        return createOrUpdate(new AtlasEntityStream(updatedEntity), true);
+    }
+
+    @Override
+    @GraphTransaction
+    public EntityMutationResponse updateEntityAttributeByGuid(String guid, String attrName, Object attrValue)
+                                                              throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> updateEntityAttributeByGuid({}, {}, {})", guid, attrName, attrValue);
+        }
+
+        AtlasEntityWithExtInfo entityInfo = getById(guid);
+
+        if (entityInfo == null || entityInfo.getEntity() == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+        }
+
+        AtlasEntity     entity     = entityInfo.getEntity();
+        AtlasEntityType entityType = (AtlasEntityType) typeRegistry.getType(entity.getTypeName());
+        AtlasAttribute  attr       = entityType.getAttribute(attrName);
+
+        if (attr == null) {
+            throw new AtlasBaseException(AtlasErrorCode.UNKNOWN_ATTRIBUTE, attrName, entity.getTypeName());
+        }
+
+        AtlasType   attrType     = attr.getAttributeType();
+        AtlasEntity updateEntity = new AtlasEntity();
+
+        updateEntity.setGuid(guid);
+        updateEntity.setTypeName(entity.getTypeName());
+
+        switch (attrType.getTypeCategory()) {
+            case PRIMITIVE:
+            case OBJECT_ID_TYPE:
+                updateEntity.setAttribute(attrName, attrValue);
+                break;
+
+            default:
+                throw new AtlasBaseException(AtlasErrorCode.ATTRIBUTE_UPDATE_NOT_SUPPORTED, attrName, attrType.getTypeName());
+        }
+
+        return createOrUpdate(new AtlasEntityStream(updateEntity), true);
+    }
+
     @GraphTransaction
     public EntityMutationResponse deleteById(final String guid) throws AtlasBaseException {
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
index 5127f74..993cf61 100755
--- a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
@@ -308,7 +308,8 @@ public class DefaultMetadataService implements MetadataService, ActiveStateChang
         return result;
     }
 
-    private ITypedReferenceableInstance[] deserializeClassInstances(String entityInstanceDefinition) throws AtlasException {
+    @Override
+    public ITypedReferenceableInstance[] deserializeClassInstances(String entityInstanceDefinition) throws AtlasException {
         return GraphHelper.deserializeClassInstances(typeSystem, entityInstanceDefinition);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/server-api/src/main/java/org/apache/atlas/services/MetadataService.java
----------------------------------------------------------------------
diff --git a/server-api/src/main/java/org/apache/atlas/services/MetadataService.java b/server-api/src/main/java/org/apache/atlas/services/MetadataService.java
index e0fb66c..45b35b3 100644
--- a/server-api/src/main/java/org/apache/atlas/services/MetadataService.java
+++ b/server-api/src/main/java/org/apache/atlas/services/MetadataService.java
@@ -301,4 +301,12 @@ public interface MetadataService {
      * @return
      */
     List<EntityAuditEvent> getAuditEvents(String guid, String startKey, short count) throws AtlasException;
+
+    /**
+     * Deserializes entity instances into ITypedReferenceableInstance array.
+     * @param entityInstanceDefinition
+     * @return ITypedReferenceableInstance[]
+     * @throws AtlasException
+     */
+    ITypedReferenceableInstance[] deserializeClassInstances(String entityInstanceDefinition) throws AtlasException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
index f59cd9d..d7adb3a 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
@@ -20,16 +20,28 @@ package org.apache.atlas.web.resources;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.sun.jersey.api.core.ResourceContext;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasConstants;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.CreateUpdateEntitiesResult;
 import org.apache.atlas.EntityAuditEvent;
 import org.apache.atlas.AtlasClient.EntityResult;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
+import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
+import org.apache.atlas.model.instance.EntityMutationResponse;
 import org.apache.atlas.model.instance.GuidMapping;
+import org.apache.atlas.repository.converters.AtlasFormatConverter.ConverterContext;
+import org.apache.atlas.repository.converters.AtlasInstanceConverter;
+import org.apache.atlas.repository.store.graph.AtlasEntityStore;
 import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.IStruct;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.exception.EntityExistsException;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
@@ -39,6 +51,7 @@ import org.apache.atlas.typesystem.json.InstanceSerialization;
 import org.apache.atlas.typesystem.types.ValueConversionException;
 import org.apache.atlas.utils.AtlasPerfTracer;
 import org.apache.atlas.utils.ParamChecker;
+import org.apache.atlas.web.rest.EntityREST;
 import org.apache.atlas.web.util.Servlets;
 import org.apache.commons.lang.StringUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -59,7 +72,9 @@ import javax.ws.rs.core.UriInfo;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 
 /**
@@ -77,11 +92,17 @@ public class EntityResource {
 
     private static final String TRAIT_NAME = "traitName";
 
-    private final MetadataService metadataService;
+    private final MetadataService        metadataService;
+    private final AtlasInstanceConverter restAdapters;
+    private final AtlasEntityStore       entitiesStore;
+    private final AtlasTypeRegistry      typeRegistry;
 
     @Context
     UriInfo uriInfo;
 
+    @Context
+    private ResourceContext resourceContext;
+
     /**
      * Created by the Guice ServletModule and injected with the
      * configured MetadataService.
@@ -89,8 +110,11 @@ public class EntityResource {
      * @param metadataService metadata service handle
      */
     @Inject
-    public EntityResource(MetadataService metadataService) {
+    public EntityResource(MetadataService metadataService, AtlasInstanceConverter restAdapters, AtlasEntityStore entitiesStore, AtlasTypeRegistry typeRegistry) {
         this.metadataService = metadataService;
+        this.restAdapters    = restAdapters;
+        this.entitiesStore   = entitiesStore;
+        this.typeRegistry    = typeRegistry;
     }
 
     /**
@@ -131,15 +155,20 @@ public class EntityResource {
                 LOG.debug("submitting entities {} ", entityJson);
             }
 
-            final CreateUpdateEntitiesResult result = metadataService.createEntities(entities);
-            final List<String> guids = result.getEntityResult().getCreatedEntities();
+            EntityREST               entityREST       = resourceContext.getResource(EntityREST.class);
+            AtlasEntitiesWithExtInfo entitiesInfo     = restAdapters.toAtlasEntities(entities);
+            EntityMutationResponse   mutationResponse = entityREST.createOrUpdate(entitiesInfo);
+
+            final List<String> guids = restAdapters.getGuids(mutationResponse.getCreatedEntities());
+
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Created entities {}", guids);
             }
 
-            JSONObject response = getResponse(result);
+            final CreateUpdateEntitiesResult result = restAdapters.toCreateUpdateEntitiesResult(mutationResponse);
 
-            URI locationURI = getLocationURI(guids);
+            JSONObject response    = getResponse(result);
+            URI        locationURI = getLocationURI(guids);
 
             return Response.created(locationURI).entity(response).build();
 
@@ -165,7 +194,6 @@ public class EntityResource {
         }
     }
 
-
     @VisibleForTesting
     public URI getLocationURI(List<String> guids) {
         URI locationURI = null;
@@ -235,7 +263,10 @@ public class EntityResource {
                 LOG.info("updating entities {} ", entityJson);
             }
 
-            CreateUpdateEntitiesResult result = metadataService.updateEntities(entities);
+            EntityREST                 entityREST       = resourceContext.getResource(EntityREST.class);
+            AtlasEntitiesWithExtInfo   entitiesInfo     = restAdapters.toAtlasEntities(entities);
+            EntityMutationResponse     mutationResponse = entityREST.createOrUpdate(entitiesInfo);
+            CreateUpdateEntitiesResult result           = restAdapters.toCreateUpdateEntitiesResult(mutationResponse);
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Updated entities: {}", result.getEntityResult());
@@ -317,11 +348,19 @@ public class EntityResource {
                 LOG.debug("Partially updating entity by unique attribute {} {} {} {} ", entityType, attribute, value, entityJson);
             }
 
-            Referenceable updatedEntity =
-                InstanceSerialization.fromJsonReferenceable(entityJson, true);
+            Referenceable updatedEntity = InstanceSerialization.fromJsonReferenceable(entityJson, true);
+
+            entityType = ParamChecker.notEmpty(entityType, "Entity type cannot be null");
+            attribute  = ParamChecker.notEmpty(attribute, "attribute name cannot be null");
+            value      = ParamChecker.notEmpty(value, "attribute value cannot be null");
+
+            Map<String, Object> attributes = new HashMap<>();
+            attributes.put(attribute, value);
 
-            CreateUpdateEntitiesResult result =
-                    metadataService.updateEntityByUniqueAttribute(entityType, attribute, value, updatedEntity);
+            AtlasEntitiesWithExtInfo   entitiesInfo     = restAdapters.toAtlasEntity(updatedEntity);
+            AtlasEntity                entity           = entitiesInfo.getEntity(updatedEntity.getId()._getId());
+            EntityMutationResponse     mutationResponse = entitiesStore.updateByUniqueAttributes(getEntityType(entityType), attributes, entity);
+            CreateUpdateEntitiesResult result           = restAdapters.toCreateUpdateEntitiesResult(mutationResponse);
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Updated entities: {}", result.getEntityResult());
@@ -379,9 +418,9 @@ public class EntityResource {
             }
 
             if (StringUtils.isEmpty(attribute)) {
-                return updateEntityPartialByGuid(guid, request);
+                return partialUpdateEntityByGuid(guid, request);
             } else {
-                return updateEntityAttributeByGuid(guid, attribute, request);
+                return partialUpdateEntityAttrByGuid(guid, attribute, request);
             }
         } finally {
             AtlasPerfTracer.log(perf);
@@ -392,7 +431,7 @@ public class EntityResource {
         }
     }
 
-    private Response updateEntityPartialByGuid(String guid, HttpServletRequest request) {
+    private Response partialUpdateEntityByGuid(String guid, HttpServletRequest request) {
         String entityJson = null;
         try {
             guid = ParamChecker.notEmpty(guid, "Guid property cannot be null");
@@ -402,10 +441,11 @@ public class EntityResource {
                 LOG.debug("partially updating entity for guid {} : {} ", guid, entityJson);
             }
 
-            Referenceable updatedEntity =
-                    InstanceSerialization.fromJsonReferenceable(entityJson, true);
-
-            CreateUpdateEntitiesResult result = metadataService.updateEntityPartialByGuid(guid, updatedEntity);
+            Referenceable              updatedEntity    = InstanceSerialization.fromJsonReferenceable(entityJson, true);
+            AtlasEntitiesWithExtInfo   entitiesInfo     = restAdapters.toAtlasEntity(updatedEntity);
+            AtlasEntity                entity           = entitiesInfo.getEntity(updatedEntity.getId()._getId());
+            EntityMutationResponse     mutationResponse = entitiesStore.updateByGuid(getEntityType(updatedEntity.getTypeName()), guid, entity);
+            CreateUpdateEntitiesResult result           = restAdapters.toCreateUpdateEntitiesResult(mutationResponse);
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Updated entities: {}", result.getEntityResult());
@@ -435,7 +475,7 @@ public class EntityResource {
      * @postbody property's value
      * @return response payload as json
      */
-    private Response updateEntityAttributeByGuid(String guid, String property, HttpServletRequest request) {
+    private Response partialUpdateEntityAttrByGuid(String guid, String property, HttpServletRequest request) {
         String value = null;
         try {
             Preconditions.checkNotNull(property, "Entity property cannot be null");
@@ -446,7 +486,8 @@ public class EntityResource {
                 LOG.debug("Updating entity {} for property {} = {}", guid, property, value);
             }
 
-            CreateUpdateEntitiesResult result = metadataService.updateEntityAttributeByGuid(guid, property, value);
+            EntityMutationResponse     mutationResponse = entitiesStore.updateEntityAttributeByGuid(guid, property, value);
+            CreateUpdateEntitiesResult result           = restAdapters.toCreateUpdateEntitiesResult(mutationResponse);
 
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Updated entities: {}", result.getEntityResult());
@@ -481,9 +522,9 @@ public class EntityResource {
     @DELETE
     @Produces(Servlets.JSON_MEDIA_TYPE)
     public Response deleteEntities(@QueryParam("guid") List<String> guids,
-        @QueryParam("type") String entityType,
-        @QueryParam("property") String attribute,
-        @QueryParam("value") String value) {
+                                   @QueryParam("type") String entityType,
+                                   @QueryParam("property") final String attribute,
+                                   @QueryParam("value") final String value) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> EntityResource.deleteEntities({}, {}, {}, {})", guids, entityType, attribute, value);
         }
@@ -494,19 +535,26 @@ public class EntityResource {
                 perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "EntityResource.deleteEntities(" + guids + ", " + entityType + ", " + attribute + ", " + value + ")");
             }
 
-            AtlasClient.EntityResult entityResult;
+            EntityResult entityResult;
+            EntityREST entityREST = resourceContext.getResource(EntityREST.class);
+
             if (guids != null && !guids.isEmpty()) {
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Deleting entities {}", guids);
                 }
 
-                entityResult = metadataService.deleteEntities(guids);
+                EntityMutationResponse mutationResponse = entityREST.deleteByGuids(guids);
+                entityResult = restAdapters.toCreateUpdateEntitiesResult(mutationResponse).getEntityResult();
             } else {
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Deleting entity type={} with property {}={}", entityType, attribute, value);
                 }
 
-                entityResult = metadataService.deleteEntityByUniqueAttribute(entityType, attribute, value);
+                Map<String, Object> attributes = new HashMap<>();
+                attributes.put(attribute, value);
+
+                EntityMutationResponse mutationResponse = entitiesStore.deleteByUniqueAttributes(getEntityType(entityType), attributes);
+                entityResult = restAdapters.toCreateUpdateEntitiesResult(mutationResponse).getEntityResult();
             }
 
             if (LOG.isDebugEnabled()) {
@@ -634,7 +682,7 @@ public class EntityResource {
     @Produces(Servlets.JSON_MEDIA_TYPE)
     public Response getEntity(@QueryParam("type") String entityType,
                               @QueryParam("property") String attribute,
-                              @QueryParam("value") String value) {
+                              @QueryParam("value") final String value) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> EntityResource.getEntity({}, {}, {})", entityType, attribute, value);
         }
@@ -678,7 +726,26 @@ public class EntityResource {
             attribute  = ParamChecker.notEmpty(attribute, "attribute name cannot be null");
             value      = ParamChecker.notEmpty(value, "attribute value cannot be null");
 
-            final String entityDefinition = metadataService.getEntityDefinition(entityType, attribute, value);
+            Map<String, Object> attributes = new HashMap<>();
+            attributes.put(attribute, value);
+
+            AtlasEntityWithExtInfo entityInfo;
+
+            try {
+                entityInfo = entitiesStore.getByUniqueAttributes(getEntityType(entityType), attributes);
+            } catch (AtlasBaseException e) {
+                LOG.error("Cannot find entity with type: {0}, attribute: {1} and value: {2}", entityType, attribute, value);
+                throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+            }
+
+            String entityDefinition = null;
+
+            if (entityInfo != null) {
+                AtlasEntity entity = entityInfo.getEntity();
+                final ITypedReferenceableInstance instance = restAdapters.getITypedReferenceable(entity);
+
+                entityDefinition = InstanceSerialization.toJson(instance, true);
+            }
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
@@ -694,10 +761,7 @@ public class EntityResource {
 
             return Response.status(status).entity(response).build();
 
-        } catch (EntityNotFoundException e) {
-            LOG.error("An entity with type={} and qualifiedName={} does not exist", entityType, value, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
+        } catch (IllegalArgumentException e) {
             LOG.error("Bad type={}, qualifiedName={}", entityType, value, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
@@ -1032,4 +1096,8 @@ public class EntityResource {
         }
         return jsonArray;
     }
+
+    private AtlasEntityType getEntityType(String typeName) {
+        return typeRegistry.getEntityTypeByName(typeName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/webapp/src/main/java/org/apache/atlas/web/rest/EntityREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/EntityREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/EntityREST.java
index 92ea93e..d1bef78 100644
--- a/webapp/src/main/java/org/apache/atlas/web/rest/EntityREST.java
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/EntityREST.java
@@ -155,13 +155,30 @@ public class EntityREST {
         return entitiesStore.updateByUniqueAttributes(entityType, uniqueAttributes, entity);
     }
 
+    /*******
+     * Entity Partial Update - Add/Update entity attribute identified by its GUID.
+     * Supports only uprimitive attribute type and entity references.
+     * does not support updation of complex types like arrays, maps
+     * Null updates are not possible
+     *******/
+    @PUT
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    @Path("/guid/{guid}")
+    public EntityMutationResponse partialUpdateByGuid(@PathParam("guid") String guid,
+                                                      @QueryParam("name") String attrName,
+                                                      Object attrValue) throws Exception {
+
+        return entitiesStore.updateEntityAttributeByGuid(guid, attrName, attrValue);
+    }
+
     /**
      * Delete an entity identified by its GUID.
      * @param  guid GUID for the entity
      * @return EntityMutationResponse
      */
     @DELETE
-    @Path("guid/{guid}")
+    @Path("/guid/{guid}")
     @Consumes({Servlets.JSON_MEDIA_TYPE, MediaType.APPLICATION_JSON})
     @Produces(Servlets.JSON_MEDIA_TYPE)
     public EntityMutationResponse deleteByGuid(@PathParam("guid") final String guid) throws AtlasBaseException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/765d556c/webapp/src/test/java/org/apache/atlas/web/resources/EntityResourceTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/EntityResourceTest.java b/webapp/src/test/java/org/apache/atlas/web/resources/EntityResourceTest.java
index 3fe8e11..2f71378 100644
--- a/webapp/src/test/java/org/apache/atlas/web/resources/EntityResourceTest.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/EntityResourceTest.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -21,13 +21,22 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
 import javax.ws.rs.core.Response;
 
+import com.vividsolutions.jts.util.CollectionUtil;
 import org.apache.atlas.AtlasClient.EntityResult;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+import org.apache.atlas.model.instance.EntityMutations;
+import org.apache.atlas.repository.converters.AtlasInstanceConverter;
+import org.apache.atlas.repository.store.graph.AtlasEntityStore;
 import org.apache.atlas.services.MetadataService;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.commons.collections.CollectionUtils;
 import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
@@ -42,8 +51,9 @@ public class EntityResourceTest {
 
     private static final String DELETED_GUID = "deleted_guid";
 
+
     @Mock
-    MetadataService mockService;
+    AtlasEntityStore entitiesStore;
 
     @BeforeMethod
     public void setUp() {
@@ -53,22 +63,35 @@ public class EntityResourceTest {
     @Test
     public void testDeleteEntitiesDoesNotLookupDeletedEntity() throws Exception {
         List<String> guids = Collections.singletonList(DELETED_GUID);
+        List<AtlasEntityHeader> deletedEntities = Collections.singletonList(new AtlasEntityHeader(null, DELETED_GUID, null));
 
         // Create EntityResult with a deleted guid and no other guids.
-        EntityResult entityResult = new EntityResult(Collections.<String>emptyList(),
-            Collections.<String>emptyList(), guids);
-        when(mockService.deleteEntities(guids)).thenReturn(entityResult);
+        EntityMutationResponse  resp    = new EntityMutationResponse();
+        List<AtlasEntityHeader> headers = toAtlasEntityHeaders(guids);
+
+        for (AtlasEntityHeader entity : headers) {
+            resp.addEntity(EntityMutations.EntityOperation.DELETE, entity);
+        }
+
+        when(entitiesStore.deleteByIds(guids)).thenReturn(resp);
 
-        // Create EntityResource with mock MetadataService.
-        EntityResource entityResource = new EntityResource(mockService);
+        EntityMutationResponse response = entitiesStore.deleteByIds(guids);
+
+        List<AtlasEntityHeader> responseDeletedEntities = response.getDeletedEntities();
+
+        Assert.assertEquals(responseDeletedEntities, deletedEntities);
+    }
 
-        Response response = entityResource.deleteEntities(guids, null, null, null);
+    private List<AtlasEntityHeader> toAtlasEntityHeaders(List<String> guids) {
+        List<AtlasEntityHeader> ret = null;
 
-        // Verify that if the EntityResult returned by MetadataService includes only deleted guids,
-        // deleteEntities() does not perform any entity lookup.
-        verify(mockService, never()).getEntityDefinition(Matchers.anyString());
+        if (CollectionUtils.isNotEmpty(guids)) {
+            ret = new ArrayList<>(guids.size());
+            for (String guid : guids) {
+                ret.add(new AtlasEntityHeader(null, guid, null));
+            }
+        }
 
-        EntityResult resultFromEntityResource = EntityResult.fromString(response.getEntity().toString());
-        Assert.assertTrue(resultFromEntityResource.getDeletedEntities().contains(DELETED_GUID));
+        return ret;
     }
 }