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

incubator-atlas git commit: ATLAS-1601 Added support for classifications(sumasai)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 8244f8bf9 -> 101abe6ee


ATLAS-1601 Added support for classifications(sumasai)


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

Branch: refs/heads/master
Commit: 101abe6eedb18a0ef3e30c6ad7926b02fb89605a
Parents: 8244f8b
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Fri Feb 24 13:29:54 2017 -0800
Committer: Suma Shivaprasad <su...@gmail.com>
Committed: Tue Feb 28 13:11:47 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/atlas/AtlasErrorCode.java   |   1 +
 .../converters/AtlasInstanceConverter.java      |   1 +
 .../store/graph/AtlasEntityStore.java           |  14 +-
 .../store/graph/v1/AtlasEntityStoreV1.java      |  89 +++++++++++--
 .../store/graph/v1/EntityGraphMapper.java       | 132 ++++++++++++++++++-
 .../store/graph/v1/EntityGraphRetriever.java    |  77 +++++++++--
 .../store/graph/v1/EntityMutationContext.java   |   5 +-
 .../atlas/web/resources/EntityResource.java     |  92 ++++++++-----
 .../org/apache/atlas/web/rest/EntityREST.java   |  86 +++---------
 9 files changed, 359 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 d58c514..edcef4b 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -74,6 +74,7 @@ public enum AtlasErrorCode {
     INSTANCE_GUID_NOT_FOUND(404, "ATLAS4045E", "Given instance guid {0} is invalid/not found"),
     INSTANCE_LINEAGE_QUERY_FAILED(404, "ATLAS4047E", "Instance lineage query failed {0}"),
     INSTANCE_CRUD_INVALID_PARAMS(404, "ATLAS4049E", "Invalid instance creation/updation parameters passed : {0}"),
+    CLASSIFICATION_NOT_FOUND(404, "ATLAS4048E", "Given classification {0} was invalid"),
 
     INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND(404, "ATLAS40410E", "Instance {0} with unique attribute {1} does not exist"),
     REFERENCED_ENTITY_NOT_FOUND(404, "ATLAS40411E", "Referenced entity {0} is not found"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 621b32f..8f246e0 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
@@ -201,6 +201,7 @@ public class AtlasInstanceConverter {
         return new AtlasBaseException(e);
     }
 
+
     public AtlasEntity.AtlasEntitiesWithExtInfo toAtlasEntities(List<Referenceable> referenceables) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> toAtlasEntities");

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 6c372b3..61657a1 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
@@ -18,6 +18,7 @@
 package org.apache.atlas.repository.store.graph;
 
 
+import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.impexp.AtlasImportResult;
 import org.apache.atlas.model.instance.AtlasClassification;
@@ -135,14 +136,17 @@ public interface AtlasEntityStore {
      */
     void addClassifications(String guid, List<AtlasClassification> classification) throws AtlasBaseException;
 
-
-    /**
-     * Update classification(s)
-     */
-    void updateClassifications(String guid, List<AtlasClassification> classification) throws AtlasBaseException;
+    @GraphTransaction
+    void addClassification(List<String> guids, AtlasClassification classification) throws AtlasBaseException;
 
     /**
      * Delete classification(s)
      */
     void deleteClassifications(String guid, List<String> classificationNames) throws AtlasBaseException;
+
+    @GraphTransaction
+    List<AtlasClassification> getClassifications(String guid) throws AtlasBaseException;
+
+    @GraphTransaction
+    AtlasClassification getClassification(String guid, String classificationName) throws AtlasBaseException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 c84f169..af1066d 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
@@ -49,6 +49,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -118,7 +119,6 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
     @GraphTransaction
     public AtlasEntityWithExtInfo getByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> uniqAttributes)
                                                                                             throws AtlasBaseException {
-
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> getByUniqueAttribute({}, {})", entityType.getTypeName(), uniqAttributes);
         }
@@ -166,7 +166,7 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
 
             AtlasEntityStreamForImport oneEntityStream = new AtlasEntityStreamForImport(entity, entityStream);
 
-            EntityMutationResponse resp = createOrUpdate(oneEntityStream, false);
+            EntityMutationResponse resp = createOrUpdate(oneEntityStream, false, true);
 
             updateImportMetrics("entity:%s:created", resp.getCreatedEntities(), processedGuids, importResult);
             updateImportMetrics("entity:%s:updated", resp.getUpdatedEntities(), processedGuids, importResult);
@@ -204,9 +204,8 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
         }
     }
 
-    @Override
     @GraphTransaction
-    public EntityMutationResponse createOrUpdate(EntityStream entityStream, boolean isPartialUpdate) throws AtlasBaseException {
+    private EntityMutationResponse createOrUpdate(EntityStream entityStream, boolean isPartialUpdate, boolean replaceClassifications) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> createOrUpdate()");
         }
@@ -220,7 +219,7 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
 
         EntityMutationContext context = preCreateOrUpdate(entityStream, entityGraphMapper, isPartialUpdate);
 
-        EntityMutationResponse ret = entityGraphMapper.mapAttributes(context, isPartialUpdate);
+        EntityMutationResponse ret = entityGraphMapper.mapAttributesAndClassifications(context, isPartialUpdate, replaceClassifications);
 
         ret.setGuidAssignments(context.getGuidAssignments());
 
@@ -235,6 +234,11 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
     }
 
     @Override
+    public EntityMutationResponse createOrUpdate(EntityStream entityStream, boolean isPartialUpdate) throws AtlasBaseException {
+        return createOrUpdate(entityStream, isPartialUpdate, false);
+    }
+
+    @Override
     @GraphTransaction
     public EntityMutationResponse updateByUniqueAttributes(AtlasEntityType entityType, Map<String, Object> uniqAttributes,
                                                            AtlasEntity updatedEntity) throws AtlasBaseException {
@@ -411,20 +415,83 @@ public class AtlasEntityStoreV1 implements AtlasEntityStore {
 
     @Override
     @GraphTransaction
-    public void addClassifications(String guid, List<AtlasClassification> classification) throws AtlasBaseException {
-        throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, "addClassifications() not implemented yet");
+    public void addClassifications(final String guid, final List<AtlasClassification> classifications) throws AtlasBaseException {
+        if (StringUtils.isEmpty(guid)) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "Guid(s) not specified");
+        }
+        if (CollectionUtils.isEmpty(classifications)) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "classifications(s) not specified");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Adding classifications={} to entity={}", classifications, guid);
+        }
+
+        EntityGraphMapper graphMapper = new EntityGraphMapper(deleteHandler, typeRegistry);
+        graphMapper.addClassifications(new EntityMutationContext(), guid, classifications);
+
     }
 
     @Override
     @GraphTransaction
-    public void updateClassifications(String guid, List<AtlasClassification> classification) throws AtlasBaseException {
-        throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, "updateClassifications() not implemented yet");
+    public void addClassification(final List<String> guids, final AtlasClassification classification) throws AtlasBaseException {
+        if (CollectionUtils.isEmpty(guids)) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "Guid(s) not specified");
+        }
+        if (classification == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "classification not specified");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Adding classification={} to entities={}", classification, guids);
+        }
+
+        EntityGraphMapper graphMapper = new EntityGraphMapper(deleteHandler, typeRegistry);
+
+        for (String guid : guids) {
+            graphMapper.addClassifications(new EntityMutationContext(), guid, Collections.singletonList(classification));
+        }
+
     }
 
     @Override
     @GraphTransaction
-    public void deleteClassifications(String guid, List<String> classificationNames) throws AtlasBaseException {
-        throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, "deleteClassifications() not implemented yet");
+    public void deleteClassifications(final String guid, final List<String> classificationNames) throws AtlasBaseException {
+        if (StringUtils.isEmpty(guid)) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "Guid(s) not specified");
+        }
+        if (CollectionUtils.isEmpty(classificationNames)) {
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "classifications(s) not specified");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Deleting classifications={} from entity={}", classificationNames, guid);
+        }
+
+        EntityGraphMapper entityGraphMapper = new EntityGraphMapper(deleteHandler, typeRegistry);
+        entityGraphMapper.deleteClassifications(guid, classificationNames);
+    }
+
+    @Override
+    @GraphTransaction
+    public List<AtlasClassification> getClassifications(String guid) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Getting classifications for entities={}", guid);
+        }
+
+        EntityGraphRetriever graphRetriever = new EntityGraphRetriever(typeRegistry);
+        return graphRetriever.getClassifications(guid);
+    }
+
+    @Override
+    @GraphTransaction
+    public AtlasClassification getClassification(String guid, String classificationName) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Getting classifications for entities={}", guid);
+        }
+
+        EntityGraphRetriever graphRetriever = new EntityGraphRetriever(typeRegistry);
+        return graphRetriever.getClassification(guid, classificationName);
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 09f69db..a5abac2 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
@@ -24,6 +24,7 @@ import org.apache.atlas.AtlasException;
 import org.apache.atlas.RequestContextV1;
 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.AtlasEntityHeader;
 import org.apache.atlas.model.instance.AtlasObjectId;
@@ -40,6 +41,7 @@ import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.type.AtlasArrayType;
+import org.apache.atlas.type.AtlasClassificationType;
 import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasMapType;
 import org.apache.atlas.type.AtlasStructType;
@@ -66,6 +68,7 @@ import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.CR
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.PARTIAL_UPDATE;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.UPDATE;
 import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.DELETE;
+import static org.apache.atlas.repository.graph.GraphHelper.string;
 
 
 public class EntityGraphMapper {
@@ -107,7 +110,7 @@ public class EntityGraphMapper {
         return ret;
     }
 
-    public EntityMutationResponse mapAttributes(EntityMutationContext context, boolean isPartialUpdate) throws AtlasBaseException {
+    public EntityMutationResponse mapAttributesAndClassifications(EntityMutationContext context, final boolean isPartialUpdate, final boolean replaceClassifications) throws AtlasBaseException {
         EntityMutationResponse resp = new EntityMutationResponse();
 
         Collection<AtlasEntity> createdEntities = context.getCreatedEntities();
@@ -122,6 +125,7 @@ public class EntityGraphMapper {
                 mapAttributes(createdEntity, vertex, CREATE, context);
 
                 resp.addEntity(CREATE, constructHeader(createdEntity, entityType, vertex));
+                addClassifications(context, guid, createdEntity.getClassifications());
             }
         }
 
@@ -139,6 +143,10 @@ public class EntityGraphMapper {
                     resp.addEntity(UPDATE, constructHeader(updatedEntity, entityType, vertex));
                 }
 
+                if ( replaceClassifications ) {
+                    deleteClassifications(guid);
+                    addClassifications(context, guid, updatedEntity.getClassifications());
+                }
             }
         }
 
@@ -176,6 +184,21 @@ public class EntityGraphMapper {
         return ret;
     }
 
+    private AtlasVertex createClassificationVertex(AtlasClassification classification) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> createVertex({})", classification.getTypeName());
+        }
+
+        AtlasClassificationType classificationType = typeRegistry.getClassificationTypeByName(classification.getTypeName());
+
+        AtlasVertex ret = createStructVertex(classification);
+
+        AtlasGraphUtilsV1.addProperty(ret, Constants.SUPER_TYPES_PROPERTY_KEY, classificationType.getAllSuperTypes());
+
+        return ret;
+    }
+
+
     private void mapAttributes(AtlasStruct struct, AtlasVertex vertex, EntityOperation op, EntityMutationContext context) throws AtlasBaseException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> mapAttributes({}, {})", op, struct.getTypeName());
@@ -346,7 +369,6 @@ public class EntityGraphMapper {
 
         if (entityVertex == null) {
             AtlasObjectId objId = getObjectId(ctx.getValue());
-
             entityVertex = context.getDiscoveryContext().getResolvedEntityVertex(objId);
         }
 
@@ -753,4 +775,110 @@ public class EntityGraphMapper {
     public static AtlasEntityHeader constructHeader(AtlasObjectId id) {
         return new AtlasEntityHeader(id.getTypeName(), id.getGuid(), id.getUniqueAttributes());
     }
+
+    public void addClassifications(final EntityMutationContext context, String guid, List<AtlasClassification> classifications)
+        throws AtlasBaseException {
+
+        if ( CollectionUtils.isNotEmpty(classifications)) {
+
+            AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
+            if (instanceVertex == null) {
+                throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+            }
+
+            String entityTypeName = AtlasGraphUtilsV1.getTypeName(instanceVertex);
+
+            final AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entityTypeName);
+
+            for (AtlasClassification classification : classifications) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("mapping classification {}", classification);
+                }
+
+                GraphHelper.addProperty(instanceVertex, Constants.TRAIT_NAMES_PROPERTY_KEY, classification.getTypeName());
+                // add a new AtlasVertex for the struct or trait instance
+                AtlasVertex classificationVertex = createClassificationVertex(classification);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("created vertex {} for trait {}", string(classificationVertex), classification.getTypeName());
+                }
+
+                // add the attributes for the trait instance
+                mapClassification(EntityOperation.CREATE, context, classification, entityType, instanceVertex, classificationVertex);
+            }
+        }
+    }
+
+    private AtlasEdge mapClassification(EntityOperation operation,  final EntityMutationContext context, AtlasClassification classification, AtlasEntityType entityType, AtlasVertex parentInstanceVertex, AtlasVertex traitInstanceVertex)
+        throws AtlasBaseException {
+
+        // map all the attributes to this newly created AtlasVertex
+        mapAttributes(classification, traitInstanceVertex, operation, context);
+
+        // add an edge to the newly created AtlasVertex from the parent
+        String relationshipLabel = GraphHelper.getTraitLabel(entityType.getTypeName(), classification.getTypeName());
+        try {
+           return graphHelper.getOrCreateEdge(parentInstanceVertex, traitInstanceVertex, relationshipLabel);
+        } catch (RepositoryException e) {
+            throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, e);
+        }
+    }
+
+    public void deleteClassifications(String guid) throws AtlasBaseException {
+
+        AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
+        if (instanceVertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+        }
+
+        List<String> traitNames = GraphHelper.getTraitNames(instanceVertex);
+
+        deleteClassifications(guid, traitNames);
+    }
+
+    public void deleteClassifications(String guid, List<String> classificationNames) throws AtlasBaseException {
+
+        AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
+        if (instanceVertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+        }
+
+        List<String> traitNames = GraphHelper.getTraitNames(instanceVertex);
+
+        validateClassificationExists(traitNames, classificationNames);
+
+        for (String classificationName : classificationNames) {
+            try {
+                final String entityTypeName = GraphHelper.getTypeName(instanceVertex);
+                String relationshipLabel = GraphHelper.getTraitLabel(entityTypeName, classificationName);
+                AtlasEdge edge = graphHelper.getEdgeForLabel(instanceVertex, relationshipLabel);
+                if (edge != null) {
+                    deleteHandler.deleteEdgeReference(edge, TypeCategory.CLASSIFICATION, false, true);
+
+                    // update the traits in entity once trait removal is successful
+                    traitNames.remove(classificationName);
+
+                }
+            } catch (Exception e) {
+                throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, e);
+            }
+        }
+
+        // remove the key
+        instanceVertex.removeProperty(Constants.TRAIT_NAMES_PROPERTY_KEY);
+
+        // add it back again
+        for (String traitName : traitNames) {
+            GraphHelper.addProperty(instanceVertex, Constants.TRAIT_NAMES_PROPERTY_KEY, traitName);
+        }
+        updateModificationMetadata(instanceVertex);
+    }
+
+    private void validateClassificationExists(List<String> existingClassifications, List<String> suppliedClassifications) throws AtlasBaseException {
+        Set<String> existingNames = new HashSet<>(existingClassifications);
+        for (String classificationName : suppliedClassifications) {
+            if (!existingNames.contains(classificationName)) {
+                throw new AtlasBaseException(AtlasErrorCode.CLASSIFICATION_NOT_FOUND, classificationName);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
index 3ba2190..44ca854 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
@@ -17,6 +17,8 @@
  */
 package org.apache.atlas.repository.store.graph.v1;
 
+import com.google.common.base.Optional;
+import com.sun.istack.Nullable;
 import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.exception.AtlasBaseException;
@@ -35,6 +37,7 @@ import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.type.*;
 import org.apache.atlas.type.AtlasStructType.AtlasAttribute;
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -203,31 +206,77 @@ public final class EntityGraphRetriever {
         }
     }
 
-    private void mapClassifications(AtlasVertex entityVertex, AtlasEntity entity, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {
-        List<String> classificationNames = GraphHelper.getTraitNames(entityVertex);
+    public List<AtlasClassification> getClassifications(String guid) throws AtlasBaseException {
 
-        if (CollectionUtils.isNotEmpty(classificationNames)) {
-            List<AtlasClassification> classifications = new ArrayList<>(classificationNames.size());
+        AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
+        if (instanceVertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+        }
 
-            for (String classficationName : classificationNames) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("mapping classification {} to atlas entity", classficationName);
-                }
+        return getClassifications(instanceVertex, null);
+    }
+
+    public AtlasClassification getClassification(String guid, String classificationName) throws AtlasBaseException {
 
-                Iterable<AtlasEdge> edges = entityVertex.getEdges(AtlasEdgeDirection.OUT, classficationName);
-                AtlasEdge           edge  = (edges != null && edges.iterator().hasNext()) ? edges.iterator().next() : null;
+        AtlasVertex instanceVertex = AtlasGraphUtilsV1.findByGuid(guid);
+        if (instanceVertex == null) {
+            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
+        }
+
+        List<AtlasClassification> classifications = getClassifications(instanceVertex, classificationName);
+        return classifications.get(0);
+    }
 
-                if (edge != null) {
-                    AtlasClassification classification = new AtlasClassification(classficationName);
 
-                    mapAttributes(edge.getInVertex(), classification, entityExtInfo);
+    private List<AtlasClassification> getClassifications(AtlasVertex instanceVertex, @Nullable String classificationNameFilter) throws AtlasBaseException {
+        List<AtlasClassification> classifications = new ArrayList<>();
+        List<String> classificationNames = GraphHelper.getTraitNames(instanceVertex);
 
+        if (CollectionUtils.isNotEmpty(classificationNames)) {
+            for (String classficationName : classificationNames) {
+                AtlasClassification classification = null;
+                if (StringUtils.isNotEmpty(classificationNameFilter)) {
+                    if (classficationName.equals(classificationNameFilter)) {
+                        classification = getClassification(instanceVertex, classficationName);
+                        classifications.add(classification);
+                        return classifications;
+                    }
+                } else {
+                    classification = getClassification(instanceVertex, classficationName);
                     classifications.add(classification);
                 }
             }
 
-            entity.setClassifications(classifications);
+
+            if (StringUtils.isNotEmpty(classificationNameFilter)) {
+                //Should not reach here if classification present
+                throw new AtlasBaseException(AtlasErrorCode.CLASSIFICATION_NOT_FOUND, classificationNameFilter);
+            }
+        }
+        return classifications;
+    }
+
+    private AtlasClassification getClassification(AtlasVertex instanceVertex, String classificationName) throws AtlasBaseException {
+
+        AtlasClassification ret = null;
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("mapping classification {} to atlas entity", classificationName);
         }
+
+        Iterable<AtlasEdge> edges = instanceVertex.getEdges(AtlasEdgeDirection.OUT, classificationName);
+        AtlasEdge           edge  = (edges != null && edges.iterator().hasNext()) ? edges.iterator().next() : null;
+
+        if (edge != null) {
+            ret = new AtlasClassification(classificationName);
+            mapAttributes(edge.getInVertex(), ret, null);
+        }
+
+        return ret;
+    }
+
+    private void mapClassifications(AtlasVertex entityVertex, AtlasEntity entity, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {
+        final List<AtlasClassification> classifications = getClassifications(entityVertex, null);
+        entity.setClassifications(classifications);
     }
 
     private Object mapVertexToAttribute(AtlasVertex entityVertex, AtlasAttribute attribute, AtlasEntityExtInfo entityExtInfo) throws AtlasBaseException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
index 23e825e..8a6a0e3 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityMutationContext.java
@@ -29,7 +29,7 @@ import org.apache.commons.lang.StringUtils;
 import java.util.*;
 
 public class EntityMutationContext {
-    private final EntityGraphDiscoveryContext  context;
+    private EntityGraphDiscoveryContext  context = null;
     private final List<AtlasEntity>            entitiesCreated = new ArrayList<>();
     private final List<AtlasEntity>            entitiesUpdated = new ArrayList<>();
     private final Map<String, AtlasEntityType> entityVsType    = new HashMap<>();
@@ -40,6 +40,9 @@ public class EntityMutationContext {
         this.context = context;
     }
 
+    public EntityMutationContext() {
+    }
+
     public void addCreated(String internalGuid, AtlasEntity entity, AtlasEntityType type, AtlasVertex atlasVertex) {
         entitiesCreated.add(entity);
         entityVsType.put(entity.getGuid(), type);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 d7adb3a..6a46fca 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
@@ -23,11 +23,13 @@ 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.AtlasErrorCode;
 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.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
@@ -797,7 +799,12 @@ public class EntityResource {
                 LOG.debug("Fetching trait names for entity={}", guid);
             }
 
-            final List<String> traitNames = metadataService.getTraitNames(guid);
+            final List<AtlasClassification> classifications = entitiesStore.getClassifications(guid);
+
+            List<String> traitNames = new ArrayList<>();
+            for (AtlasClassification classification : classifications) {
+                traitNames.add(classification.getTypeName());
+            }
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
@@ -805,11 +812,11 @@ public class EntityResource {
             response.put(AtlasClient.COUNT, traitNames.size());
 
             return Response.ok(response).build();
-        } catch (EntityNotFoundException e) {
-            LOG.error("An entity with GUID={} does not exist", guid, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
-            LOG.error("Unable to get trait names for entity {}", guid, e);
+        } catch (AtlasBaseException e) {
+            LOG.error("Unable to get trait definition for entity {}", guid, e);
+            throw toWebApplicationException(e);
+        } catch (IllegalArgumentException e) {
+            LOG.error("Unable to get trait definition for entity {}", guid, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
             LOG.error("Unable to get trait names for entity {}", guid, e);
@@ -845,12 +852,11 @@ public class EntityResource {
                 LOG.debug("Fetching all trait definitions for entity={}", guid);
             }
 
-            final String entityDefinition = metadataService.getEntityDefinitionJson(guid);
+            final List<AtlasClassification> classifications = entitiesStore.getClassifications(guid);
 
-            Referenceable entity = InstanceSerialization.fromJsonReferenceable(entityDefinition, true);
             JSONArray traits = new JSONArray();
-            for (String traitName : entity.getTraits()) {
-                IStruct trait = entity.getTrait(traitName);
+            for (AtlasClassification classification : classifications) {
+                IStruct trait = restAdapters.getTrait(classification);
                 traits.put(new JSONObject(InstanceSerialization.toJson(trait, true)));
             }
 
@@ -860,11 +866,11 @@ public class EntityResource {
             response.put(AtlasClient.COUNT, traits.length());
 
             return Response.ok(response).build();
-        } catch (EntityNotFoundException e){
-            LOG.error("An entity with GUID={} does not exist", guid, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
-            LOG.error("Unable to get trait definitions for entity {}", guid, e);
+        } catch (AtlasBaseException e) {
+            LOG.error("Unable to get trait definition for entity {}", guid, e);
+            throw toWebApplicationException(e);
+        } catch (IllegalArgumentException e) {
+            LOG.error("Unable to get trait definition for entity {}", guid, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
             LOG.error("Unable to get trait definitions for entity {}", guid, e);
@@ -902,7 +908,10 @@ public class EntityResource {
                 LOG.debug("Fetching trait definition for entity {} and trait name {}", guid, traitName);
             }
 
-            final IStruct traitDefinition = metadataService.getTraitDefinition(guid, traitName);
+
+            final AtlasClassification classification = entitiesStore.getClassification(guid, traitName);
+
+            IStruct traitDefinition = restAdapters.getTrait(classification);
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
@@ -910,10 +919,10 @@ public class EntityResource {
 
             return Response.ok(response).build();
 
-        } catch (EntityNotFoundException e){
-            LOG.error("An entity with GUID={} does not exist", guid, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
+        } catch (AtlasBaseException e) {
+            LOG.error("Unable to get trait definition for entity {} and trait {}", guid, traitName, e);
+            throw toWebApplicationException(e);
+        } catch (IllegalArgumentException e) {
             LOG.error("Unable to get trait definition for entity {} and trait {}", guid, traitName, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
@@ -955,7 +964,11 @@ public class EntityResource {
                 LOG.debug("Adding trait={} for entity={} ", traitDefinition, guid);
             }
 
-            metadataService.addTrait(guid, traitDefinition);
+            List<String> guids = new ArrayList<String>() {{
+                add(guid);
+            }};
+
+            entitiesStore.addClassification(guids, restAdapters.getClassification(InstanceSerialization.fromJsonStruct(traitDefinition, true)));
 
             URI locationURI = getLocationURI(new ArrayList<String>() {{
                 add(guid);
@@ -965,10 +978,10 @@ public class EntityResource {
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
 
             return Response.created(locationURI).entity(response).build();
-        } catch (EntityNotFoundException | TypeNotFoundException e) {
-            LOG.error("An entity with GUID={} does not exist traitDef={} ", guid, traitDefinition, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
+        } catch (AtlasBaseException e) {
+            LOG.error("Unable to add trait for entity={} traitDef={}", guid, traitDefinition, e);
+            throw toWebApplicationException(e);
+        } catch  (IllegalArgumentException e) {
             LOG.error("Unable to add trait for entity={} traitDef={}", guid, traitDefinition, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
@@ -994,7 +1007,7 @@ public class EntityResource {
     @Consumes({Servlets.JSON_MEDIA_TYPE, MediaType.APPLICATION_JSON})
     @Produces(Servlets.JSON_MEDIA_TYPE)
     public Response deleteTrait(@Context HttpServletRequest request, @PathParam("guid") String guid,
-            @PathParam(TRAIT_NAME) String traitName) {
+            @PathParam(TRAIT_NAME) final String traitName) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> EntityResource.deleteTrait({}, {})", guid, traitName);
         }
@@ -1010,20 +1023,17 @@ public class EntityResource {
                 perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "EntityResource.deleteTrait(" + guid + ", " + traitName + ")");
             }
 
-            metadataService.deleteTrait(guid, traitName);
+            entitiesStore.deleteClassifications(guid, new ArrayList<String>() {{ add(traitName); }});
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
             response.put(TRAIT_NAME, traitName);
 
             return Response.ok(response).build();
-        } catch (EntityNotFoundException | TypeNotFoundException e) {
-            LOG.error("An entity with GUID={} does not exist traitName={} ", guid, traitName, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (TraitNotFoundException e) {
-            LOG.error("The trait name={} for entity={} does not exist.", traitName, guid, e);
-            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
-        } catch (AtlasException | IllegalArgumentException e) {
+        } catch (AtlasBaseException e) {
+            LOG.error("Unable to delete trait name={} for entity={}", traitName, guid, e);
+            throw toWebApplicationException(e);
+        } catch (IllegalArgumentException e) {
             LOG.error("Unable to delete trait name={} for entity={}", traitName, guid, e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
@@ -1100,4 +1110,18 @@ public class EntityResource {
     private AtlasEntityType getEntityType(String typeName) {
         return typeRegistry.getEntityTypeByName(typeName);
     }
+
+    public static WebApplicationException toWebApplicationException(AtlasBaseException e) {
+        if (e.getAtlasErrorCode() == AtlasErrorCode.CLASSIFICATION_NOT_FOUND
+            || e.getAtlasErrorCode() == AtlasErrorCode.INSTANCE_GUID_NOT_FOUND) {
+            return new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.NOT_FOUND));
+        }
+
+        if (e.getAtlasErrorCode() == AtlasErrorCode.INVALID_PARAMETERS
+            || e.getAtlasErrorCode() == AtlasErrorCode.INSTANCE_CRUD_INVALID_PARAMS) {
+            return new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+        }
+
+        return new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/101abe6e/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 d1bef78..288ef82 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
@@ -206,26 +206,19 @@ public class EntityREST {
     /**
      * Gets the list of classifications for a given entity represented by a guid.
      * @param guid globally unique identifier for the entity
-     * @return a list of classifications for the given entity guid
+     * @return classification for the given entity guid
      */
     @GET
     @Path("/guid/{guid}/classification/{classificationName}")
     @Produces(Servlets.JSON_MEDIA_TYPE)
-    public AtlasClassification getClassification(@PathParam("guid") String guid, @PathParam("classificationName") String typeName) throws AtlasBaseException {
+    public AtlasClassification getClassification(@PathParam("guid") String guid, @PathParam("classificationName") final String classificationName) throws AtlasBaseException {
 
         if (StringUtils.isEmpty(guid)) {
             throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
         }
 
-        ensureClassificationType(typeName);
-
-        try {
-            IStruct trait = metadataService.getTraitDefinition(guid, typeName);
-            return instanceConverter.getClassification(trait);
-
-        } catch (AtlasException e) {
-            throw toAtlasBaseException(e);
-        }
+        ensureClassificationType(classificationName);
+        return entitiesStore.getClassification(guid, classificationName);
     }
 
     /**
@@ -242,22 +235,12 @@ public class EntityREST {
             throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
         }
 
-        AtlasClassification.AtlasClassifications clss = new AtlasClassification.AtlasClassifications();
+        AtlasClassification.AtlasClassifications classifications = new AtlasClassification.AtlasClassifications();
 
-        try {
-            List<AtlasClassification> clsList = new ArrayList<>();
-            for ( String traitName : metadataService.getTraitNames(guid) ) {
-                IStruct trait = metadataService.getTraitDefinition(guid, traitName);
-                AtlasClassification cls = instanceConverter.getClassification(trait);
-                clsList.add(cls);
-            }
-
-            clss.setList(clsList);
+        final List<AtlasClassification> classificationList = entitiesStore.getClassifications(guid);
+        classifications.setList(classificationList);
 
-        } catch (AtlasException e) {
-            throw toAtlasBaseException(e);
-        }
-        return clss;
+        return classifications;
     }
 
     /**
@@ -274,58 +257,27 @@ public class EntityREST {
             throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
         }
 
-        for (AtlasClassification classification:  classifications) {
-            final ITypedStruct trait = instanceConverter.getTrait(classification);
-            try {
-                metadataService.addTrait(guid, trait);
-            } catch (IllegalArgumentException e) {
-                throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, e);
-            } catch (AtlasException e) {
-                throw toAtlasBaseException(e);
-            }
-        }
-    }
-
-    /**
-     * Update classification(s) for an entity represented by a guid.
-     * Classifications are identified by their guid or name
-     * @param guid globally unique identifier for the entity
-     */
-    @PUT
-    @Path("/guid/{guid}/classifications")
-    @Consumes({Servlets.JSON_MEDIA_TYPE, MediaType.APPLICATION_JSON})
-    @Produces(Servlets.JSON_MEDIA_TYPE)
-    public void updateClassifications(@PathParam("guid") final String guid, List<AtlasClassification> classifications) throws AtlasBaseException {
-        //Not supported in old API
-
-        if (StringUtils.isEmpty(guid)) {
-            throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
-        }
+        entitiesStore.addClassifications(guid, classifications);
     }
 
     /**
      * Deletes a given classification from an existing entity represented by a guid.
      * @param guid      globally unique identifier for the entity
-     * @param typeName name of the trait
+     * @param classificationName name of the classifcation
      */
     @DELETE
     @Path("/guid/{guid}/classification/{classificationName}")
-    @Consumes({Servlets.JSON_MEDIA_TYPE, MediaType.APPLICATION_JSON})
     @Produces(Servlets.JSON_MEDIA_TYPE)
     public void deleteClassification(@PathParam("guid") String guid,
-        @PathParam("classificationName") String typeName) throws AtlasBaseException {
+        @PathParam("classificationName") final String classificationName) throws AtlasBaseException {
 
         if (StringUtils.isEmpty(guid)) {
             throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, guid);
         }
 
-        ensureClassificationType(typeName);
+        ensureClassificationType(classificationName);
 
-        try {
-            metadataService.deleteTrait(guid, typeName);
-        } catch (AtlasException e) {
-            throw toAtlasBaseException(e);
-        }
+        entitiesStore.deleteClassifications(guid, new ArrayList<String>() {{ add(classificationName);}} );
     }
 
     /******************************************************************/
@@ -392,18 +344,10 @@ public class EntityREST {
         }
 
         if (CollectionUtils.isEmpty(entityGuids)) {
-            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "empty entity list");
+            throw new AtlasBaseException(AtlasErrorCode.INVALID_PARAMETERS, "empty guid list");
         }
 
-        final ITypedStruct trait = instanceConverter.getTrait(classification);
-
-        try {
-            metadataService.addTrait(entityGuids, trait);
-        } catch (IllegalArgumentException e) {
-            throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, e);
-        } catch (AtlasException e) {
-            throw toAtlasBaseException(e);
-        }
+        entitiesStore.addClassification(entityGuids, classification);
     }
 
     private AtlasEntityType ensureEntityType(String typeName) throws AtlasBaseException {