You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sh...@apache.org on 2016/02/04 10:48:25 UTC

incubator-atlas git commit: ATLAS-372 Expose entity deletion through REST API (dkantor via shwethags)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master d2b9b99f4 -> 20608f022


ATLAS-372 Expose entity deletion through REST API (dkantor via shwethags)


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

Branch: refs/heads/master
Commit: 20608f0221108c2495820c3b098494dd66fe0854
Parents: d2b9b99
Author: Shwetha GS <ss...@hortonworks.com>
Authored: Thu Feb 4 15:18:16 2016 +0530
Committer: Shwetha GS <ss...@hortonworks.com>
Committed: Thu Feb 4 15:18:16 2016 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/atlas/AtlasClient.java |  19 +++
 .../notification/entity/EntityNotification.java |   1 +
 .../NotificationEntityChangeListener.java       |   6 +
 release-log.txt                                 |   1 +
 .../atlas/repository/MetadataRepository.java    |   2 +-
 .../graph/GraphBackedMetadataRepository.java    |  10 +-
 .../graph/TypedInstanceToGraphMapper.java       |  32 ++++-
 .../atlas/services/DefaultMetadataService.java  |  22 ++++
 .../test/java/org/apache/atlas/TestUtils.java   |  12 +-
 ...kedMetadataRepositoryDeleteEntitiesTest.java |  12 +-
 .../service/DefaultMetadataServiceTest.java     | 130 +++++++++++++++++++
 .../atlas/listener/EntityChangeListener.java    |   8 ++
 .../apache/atlas/services/MetadataService.java  |   9 ++
 .../atlas/web/resources/EntityResource.java     |  39 +++++-
 .../web/resources/EntityJerseyResourceIT.java   |  83 ++++++++++++
 15 files changed, 360 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/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 0a730af..1fc811a 100755
--- a/client/src/main/java/org/apache/atlas/AtlasClient.java
+++ b/client/src/main/java/org/apache/atlas/AtlasClient.java
@@ -166,6 +166,8 @@ public class AtlasClient {
         UPDATE_ENTITY(BASE_URI + URI_ENTITY, HttpMethod.PUT, Response.Status.OK),
         UPDATE_ENTITY_PARTIAL(BASE_URI + URI_ENTITY, HttpMethod.POST, Response.Status.OK),
         LIST_ENTITIES(BASE_URI + URI_ENTITY, HttpMethod.GET, Response.Status.OK),
+        DELETE_ENTITIES(BASE_URI + URI_ENTITY, HttpMethod.DELETE, Response.Status.OK),
+
 
         //Trait operations
         ADD_TRAITS(BASE_URI + URI_ENTITY, HttpMethod.POST, Response.Status.CREATED),
@@ -379,6 +381,23 @@ public class AtlasClient {
     }
 
     /**
+     * Delete the specified entities from the repository
+     * 
+     * @param guids guids of entities to delete
+     * @return List of deleted entity guids
+     * @throws AtlasServiceException
+     */
+    public List<String> deleteEntities(String ... guids) throws AtlasServiceException {
+        API api = API.DELETE_ENTITIES;
+        WebResource resource = getResource(api);
+        for (String guid : guids) {
+            resource = resource.queryParam(GUID.toLowerCase(), guid);
+        }
+        JSONObject jsonResponse = callAPIWithResource(API.DELETE_ENTITIES, resource);
+        return extractResults(jsonResponse, GUID);
+    }
+    
+    /**
      * Get an entity given the entity id
      * @param guid entity id
      * @return result object

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/notification/src/main/java/org/apache/atlas/notification/entity/EntityNotification.java
----------------------------------------------------------------------
diff --git a/notification/src/main/java/org/apache/atlas/notification/entity/EntityNotification.java b/notification/src/main/java/org/apache/atlas/notification/entity/EntityNotification.java
index 5579df2..82a1100 100644
--- a/notification/src/main/java/org/apache/atlas/notification/entity/EntityNotification.java
+++ b/notification/src/main/java/org/apache/atlas/notification/entity/EntityNotification.java
@@ -33,6 +33,7 @@ public interface EntityNotification {
     enum OperationType {
         ENTITY_CREATE,
         ENTITY_UPDATE,
+        ENTITY_DELETE,
         TRAIT_ADD,
         TRAIT_DELETE
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/notification/src/main/java/org/apache/atlas/notification/entity/NotificationEntityChangeListener.java
----------------------------------------------------------------------
diff --git a/notification/src/main/java/org/apache/atlas/notification/entity/NotificationEntityChangeListener.java b/notification/src/main/java/org/apache/atlas/notification/entity/NotificationEntityChangeListener.java
index bdf9b2a..31f5c2b 100644
--- a/notification/src/main/java/org/apache/atlas/notification/entity/NotificationEntityChangeListener.java
+++ b/notification/src/main/java/org/apache/atlas/notification/entity/NotificationEntityChangeListener.java
@@ -76,9 +76,15 @@ public class NotificationEntityChangeListener implements EntityChangeListener {
         notifyOfEntityEvent(Collections.singleton(entity), EntityNotification.OperationType.TRAIT_DELETE);
     }
 
+    @Override
+    public void onEntitiesDeleted(Collection<ITypedReferenceableInstance> entities) throws AtlasException {
+        notifyOfEntityEvent(entities, EntityNotification.OperationType.ENTITY_DELETE);
+    }
+
 
     // ----- helper methods -------------------------------------------------
 
+
     // send notification of entity change
     private void notifyOfEntityEvent(Collection<ITypedReferenceableInstance> entityDefinitions,
                                      EntityNotification.OperationType operationType) throws AtlasException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 77d036b..b37c422 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -7,6 +7,7 @@ ATLAS-409 Atlas will not import avro tables with schema read from a file (dosset
 ATLAS-379 Create sqoop and falcon metadata addons (venkatnrangan,bvellanki,sowmyaramesh via shwethags)
 
 ALL CHANGES:
+ATLAS-372 Expose entity deletion through REST API (dkantor via shwethags)
 ATLAS-452 Exceptions while running HiveHookIT#testAlterTableRename (shwethags)
 ATLAS-388 UI : On creating Tag, the page to be reset for creating new Tag (Anilg via shwethags)
 ATLAS-199 webapp build fails (grunt + tests) (sanjayp via shwethags)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/main/java/org/apache/atlas/repository/MetadataRepository.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/MetadataRepository.java b/repository/src/main/java/org/apache/atlas/repository/MetadataRepository.java
index 4e86a0f..2fe42eb 100755
--- a/repository/src/main/java/org/apache/atlas/repository/MetadataRepository.java
+++ b/repository/src/main/java/org/apache/atlas/repository/MetadataRepository.java
@@ -110,7 +110,7 @@ public interface MetadataRepository {
      * @return guids of deleted entities
      * @throws RepositoryException
      */
-     List <String> deleteEntities(String... guids) throws RepositoryException;
+    TypeUtils.Pair<List<String>, List<ITypedReferenceableInstance>> deleteEntities(List<String> guids) throws RepositoryException;
     
     
     // Trait management functions

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
index e002545..2c6e58b 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
@@ -30,16 +30,13 @@ import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.MetadataRepository;
 import org.apache.atlas.repository.RepositoryException;
-import org.apache.atlas.repository.graph.TypedInstanceToGraphMapper.Operation;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.exception.EntityExistsException;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
-import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.typesystem.types.AttributeInfo;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.IDataType;
-import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.TypeUtils;
 import org.slf4j.Logger;
@@ -316,9 +313,9 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
 
     @Override
     @GraphTransaction
-    public List<String> deleteEntities(String... guids) throws RepositoryException {
+    public TypeUtils.Pair<List<String>, List<ITypedReferenceableInstance>>  deleteEntities(List<String> guids) throws RepositoryException {
 
-        if (guids == null || guids.length == 0) {
+        if (guids == null || guids.size() == 0) {
             throw new IllegalArgumentException("guids must be non-null and non-empty");
         }
         
@@ -341,6 +338,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
                 throw new RepositoryException(e);
             }
         }
-        return instanceToGraphMapper.getDeletedEntities();
+        return new TypeUtils.Pair<>(
+                instanceToGraphMapper.getDeletedEntityGuids(), instanceToGraphMapper.getDeletedEntities());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
index 8668b46..7b2890c 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/TypedInstanceToGraphMapper.java
@@ -61,8 +61,8 @@ public final class TypedInstanceToGraphMapper {
     private static final Logger LOG = LoggerFactory.getLogger(TypedInstanceToGraphMapper.class);
     private final Map<Id, Vertex> idToVertexMap = new HashMap<>();
     private final TypeSystem typeSystem = TypeSystem.getInstance();
-    private final List<String> deletedEntities = new ArrayList<>();
-
+    private final List<String> deletedEntityGuids = new ArrayList<>();
+    private final List<ITypedReferenceableInstance> deletedEntities = new ArrayList<>();
     private final GraphToTypedInstanceMapper graphToTypedInstanceMapper;
 
     private static final GraphHelper graphHelper = GraphHelper.getInstance();
@@ -688,7 +688,8 @@ public final class TypedInstanceToGraphMapper {
         
         //  Remove any underlying structs and composite entities owned by this entity.
         mapInstanceToVertex(typedInstance, instanceVertex, classType.fieldMapping().fields, false, Operation.DELETE);
-        deletedEntities.add(id._getId());
+        deletedEntityGuids.add(id._getId());
+        deletedEntities.add(typedInstance);
     }
 
     /**
@@ -728,12 +729,31 @@ public final class TypedInstanceToGraphMapper {
 
     
     /**
-     * Get the IDs of entities that have been deleted.
+     * Get the GUIDs of entities that have been deleted.
      * 
      * @return
      */
-    List<String> getDeletedEntities() {
-        return Collections.unmodifiableList(deletedEntities);
+    List<String> getDeletedEntityGuids() {
+        if (deletedEntityGuids.size() == 0) {
+            return Collections.emptyList();
+        }
+        else {
+            return Collections.unmodifiableList(deletedEntityGuids);
+        }
+    }
+    
+    /**
+     * Get the entities that have been deleted.
+     * 
+     * @return
+     */
+    List<ITypedReferenceableInstance> getDeletedEntities() {
+        if (deletedEntities.size() == 0) {
+            return Collections.emptyList();
+        }
+        else {
+            return Collections.unmodifiableList(deletedEntities);
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/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 b38face..2de8e50 100755
--- a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
@@ -21,6 +21,7 @@ package org.apache.atlas.services;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Provider;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.classification.InterfaceAudience;
@@ -54,6 +55,7 @@ import org.apache.atlas.typesystem.types.StructTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.TypeUtils;
+import org.apache.atlas.typesystem.types.TypeUtils.Pair;
 import org.apache.atlas.typesystem.types.ValueConversionException;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.apache.atlas.utils.ParamChecker;
@@ -65,6 +67,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.inject.Inject;
 import javax.inject.Singleton;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -680,4 +683,23 @@ public class DefaultMetadataService implements MetadataService {
     public void unregisterListener(EntityChangeListener listener) {
         entityChangeListeners.remove(listener);
     }
+
+    /* (non-Javadoc)
+     * @see org.apache.atlas.services.MetadataService#deleteEntities(java.lang.String)
+     */
+    @Override
+    public List<String> deleteEntities(List<String> deleteCandidateGuids) throws AtlasException {
+        ParamChecker.notEmpty(deleteCandidateGuids, "delete candidate guids cannot be empty");
+        Pair<List<String>, List<ITypedReferenceableInstance>> deleteEntitiesResult = repository.deleteEntities(deleteCandidateGuids);
+        if (deleteEntitiesResult.right.size() > 0) {
+            onEntitiesDeleted(deleteEntitiesResult.right);
+        }
+        return deleteEntitiesResult.left;
+    }
+
+    private void onEntitiesDeleted(List<ITypedReferenceableInstance> entities) throws AtlasException {
+        for (EntityChangeListener listener : entityChangeListeners) {
+            listener.onEntitiesDeleted(entities);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/test/java/org/apache/atlas/TestUtils.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/TestUtils.java b/repository/src/test/java/org/apache/atlas/TestUtils.java
index a77239c..653ec01 100755
--- a/repository/src/test/java/org/apache/atlas/TestUtils.java
+++ b/repository/src/test/java/org/apache/atlas/TestUtils.java
@@ -23,6 +23,7 @@ import com.thinkaurelius.titan.core.TitanGraph;
 import com.tinkerpop.blueprints.Edge;
 import com.tinkerpop.blueprints.Vertex;
 import com.tinkerpop.blueprints.util.io.graphson.GraphSONWriter;
+
 import org.apache.atlas.repository.graph.GraphHelper;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
@@ -187,6 +188,7 @@ public final class TestUtils {
     public static final String DATABASE_TYPE = "hive_database";
     public static final String DATABASE_NAME = "foo";
     public static final String TABLE_TYPE = "hive_table";
+    public static final String COLUMN_TYPE = "column_type";
     public static final String TABLE_NAME = "bar";
     public static final String CLASSIFICATION = "classification";
     public static final String PII = "PII";
@@ -220,7 +222,7 @@ public final class TestUtils {
         EnumTypeDefinition enumTypeDefinition = new EnumTypeDefinition("tableType", values);
 
         HierarchicalTypeDefinition<ClassType> columnsDefinition =
-                createClassTypeDef("column_type", ImmutableList.<String>of(),
+                createClassTypeDef(COLUMN_TYPE, ImmutableList.<String>of(),
                         createRequiredAttrDef("name", DataTypes.STRING_TYPE),
                         createRequiredAttrDef("type", DataTypes.STRING_TYPE));
 
@@ -228,7 +230,7 @@ public final class TestUtils {
                 new AttributeDefinition[]{createRequiredAttrDef("name", DataTypes.STRING_TYPE),});
 
         AttributeDefinition[] attributeDefinitions = new AttributeDefinition[]{
-            new AttributeDefinition("cols", String.format("array<%s>", "column_type"),
+            new AttributeDefinition("cols", String.format("array<%s>", COLUMN_TYPE),
                 Multiplicity.OPTIONAL, true, null),
             new AttributeDefinition("location", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false,
                 null),
@@ -256,7 +258,7 @@ public final class TestUtils {
                 null),
             new AttributeDefinition("sd", STORAGE_DESC_TYPE, Multiplicity.REQUIRED, true,
                 null),
-            new AttributeDefinition("columns", DataTypes.arrayTypeName("column_type"),
+            new AttributeDefinition("columns", DataTypes.arrayTypeName(COLUMN_TYPE),
                 Multiplicity.OPTIONAL, true, null),
             new AttributeDefinition("parameters", new DataTypes.MapType(DataTypes.STRING_TYPE, DataTypes.STRING_TYPE).getName(), Multiplicity.OPTIONAL, false, null),};
 
@@ -277,7 +279,7 @@ public final class TestUtils {
                                 String.format("array<%s>", DataTypes.STRING_TYPE.getName()), Multiplicity.OPTIONAL,
                                 false, null),
                         // array of classes
-                        new AttributeDefinition("columns", String.format("array<%s>", "column_type"),
+                        new AttributeDefinition("columns", String.format("array<%s>", COLUMN_TYPE),
                                 Multiplicity.OPTIONAL, true, null),
                         // array of structs
                         new AttributeDefinition("partitions", String.format("array<%s>", "partition_struct_type"),
@@ -289,7 +291,7 @@ public final class TestUtils {
                         //map of classes -
                         new AttributeDefinition("columnsMap",
                                                         DataTypes.mapTypeName(DataTypes.STRING_TYPE.getName(),
-                                                                "column_type"),
+                                                                COLUMN_TYPE),
                                                         Multiplicity.OPTIONAL, true, null),
                          //map of structs
                         new AttributeDefinition("partitionsMap",

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteEntitiesTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteEntitiesTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteEntitiesTest.java
index e199913..b024152 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteEntitiesTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryDeleteEntitiesTest.java
@@ -21,6 +21,7 @@ package org.apache.atlas.repository.graph;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.util.TitanCleanup;
 import com.tinkerpop.blueprints.Vertex;
+
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
@@ -32,6 +33,7 @@ import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.Multiplicity;
 import org.apache.atlas.typesystem.types.TypeSystem;
+import org.apache.atlas.typesystem.types.TypeUtils.Pair;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -39,7 +41,9 @@ import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
 import javax.inject.Inject;
+
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -114,8 +118,8 @@ public class GraphBackedMetadataRepositoryDeleteEntitiesTest {
         vertexCount = countVertices(Constants.ENTITY_TYPE_PROPERTY_KEY, "SecurityClearance");
         Assert.assertEquals(vertexCount, 1);
         
-        List<String> deletedEntities = repositoryService.deleteEntities(hrDeptGuid);
-        Assert.assertTrue(deletedEntities.contains(hrDeptGuid));
+        Pair<List<String>, List<ITypedReferenceableInstance>> deletedEntities = repositoryService.deleteEntities(Arrays.asList(hrDeptGuid));
+        Assert.assertTrue(deletedEntities.left.contains(hrDeptGuid));
         
         // Verify Department entity and its contained Person entities were deleted.
         verifyEntityDoesNotExist(hrDeptGuid);
@@ -145,8 +149,8 @@ public class GraphBackedMetadataRepositoryDeleteEntitiesTest {
         ITypedReferenceableInstance employee = (ITypedReferenceableInstance) listValue;
         String employeeGuid = employee.getId()._getId();
         
-        List<String> deletedEntities = repositoryService.deleteEntities(employeeGuid);
-        Assert.assertTrue(deletedEntities.contains(employeeGuid));
+        Pair<List<String>, List<ITypedReferenceableInstance>> deletedEntities = repositoryService.deleteEntities(Arrays.asList(employeeGuid));
+        Assert.assertTrue(deletedEntities.left.contains(employeeGuid));
         verifyEntityDoesNotExist(employeeGuid);
         
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
index 085888d..ea0d8e8 100644
--- a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
@@ -22,14 +22,20 @@ import com.google.common.collect.ImmutableList;
 import com.google.inject.Inject;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.util.TitanCleanup;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.typesystem.exception.TypeNotFoundException;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.utils.ParamChecker;
+import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
+import org.apache.atlas.listener.EntityChangeListener;
 import org.apache.atlas.repository.graph.GraphProvider;
 import org.apache.atlas.services.MetadataService;
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.atlas.typesystem.IStruct;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.Struct;
 import org.apache.atlas.typesystem.TypesDef;
@@ -49,6 +55,7 @@ import org.testng.annotations.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
@@ -152,6 +159,12 @@ public class DefaultMetadataServiceTest {
         return entity;
     }
 
+    private Referenceable createColumnEntity() {
+        Referenceable entity = new Referenceable(TestUtils.COLUMN_TYPE);
+        entity.set("name", RandomStringUtils.randomAlphanumeric(10));
+        entity.set("type", "VARCHAR(32)");
+        return entity;
+    }
     @Test
     public void testCreateEntityWithUniqueAttribute() throws Exception {
         //name is the unique attribute
@@ -685,4 +698,121 @@ public class DefaultMetadataServiceTest {
             //expected
         }
     }
+    
+    @Test
+    public void testDeleteEntities() throws Exception {
+        
+        
+        // Create 2 table entities, each with 3 composite column entities
+        Referenceable dbEntity = createDBEntity();
+        String dbGuid = createInstance(dbEntity);
+        Id dbId = new Id(dbGuid, 0, TestUtils.DATABASE_TYPE);
+        Referenceable table1Entity = createTableEntity(dbId);
+        Referenceable table2Entity = createTableEntity(dbId);
+        Referenceable col1 = createColumnEntity();
+        Referenceable col2 = createColumnEntity();
+        Referenceable col3 = createColumnEntity();
+        table1Entity.set("columns", ImmutableList.of(col1, col2, col3));
+        table2Entity.set("columns", ImmutableList.of(col1, col2, col3));
+        createInstance(table1Entity);
+        createInstance(table2Entity);
+        
+        // Retrieve the table entities from the repository,
+        // to get their guids and the composite column guids.
+        String entityJson = metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, 
+            "name", (String)table1Entity.get("name"));
+        Assert.assertNotNull(entityJson);
+        table1Entity = InstanceSerialization.fromJsonReferenceable(entityJson, true);
+        Object val = table1Entity.get("columns");
+        Assert.assertTrue(val instanceof List);
+        List<IReferenceableInstance> table1Columns = (List<IReferenceableInstance>) val;
+        entityJson = metadataService.getEntityDefinition(TestUtils.TABLE_TYPE, 
+            "name", (String)table2Entity.get("name"));
+        Assert.assertNotNull(entityJson);
+        table2Entity = InstanceSerialization.fromJsonReferenceable(entityJson, true);
+        val = table2Entity.get("columns");
+        Assert.assertTrue(val instanceof List);
+        List<IReferenceableInstance> table2Columns = (List<IReferenceableInstance>) val;
+
+        // Register an EntityChangeListener to verify the notification mechanism
+        // is working for deleteEntities().
+        DeleteEntitiesChangeListener listener = new DeleteEntitiesChangeListener();
+        metadataService.registerListener(listener);
+        
+        // Delete the table entities.  The deletion should cascade
+        // to their composite columns.
+        JSONArray deleteCandidateGuids = new JSONArray();
+        deleteCandidateGuids.put(table1Entity.getId()._getId());
+        deleteCandidateGuids.put(table2Entity.getId()._getId());
+        List<String> deletedGuids = metadataService.deleteEntities(
+            Arrays.asList(table1Entity.getId()._getId(), table2Entity.getId()._getId()));
+
+        // Verify that deleteEntities() response has guids for tables and their composite columns. 
+        Assert.assertTrue(deletedGuids.contains(table1Entity.getId()._getId()));
+        Assert.assertTrue(deletedGuids.contains(table2Entity.getId()._getId()));
+        for (IReferenceableInstance column : table1Columns) {
+            Assert.assertTrue(deletedGuids.contains(column.getId()._getId()));
+        }
+        for (IReferenceableInstance column : table2Columns) {
+            Assert.assertTrue(deletedGuids.contains(column.getId()._getId()));
+        }
+        
+        // Verify that tables and their composite columns have been deleted from the repository.
+        for (String guid : deletedGuids) {
+            try {
+                metadataService.getEntityDefinition(guid);
+                Assert.fail(EntityNotFoundException.class.getSimpleName() + 
+                    " expected but not thrown.  The entity with guid " + guid + 
+                    " still exists in the repository after being deleted." );
+            }
+            catch(EntityNotFoundException e) {
+                // The entity does not exist in the repository, so deletion was successful.
+            }
+        }
+        
+        // Verify that the listener was notified about the deleted entities.
+        Collection<ITypedReferenceableInstance> deletedEntitiesFromListener = listener.getDeletedEntities();
+        Assert.assertNotNull(deletedEntitiesFromListener);
+        Assert.assertEquals(deletedEntitiesFromListener.size(), deletedGuids.size());
+        List<String> deletedGuidsFromListener = new ArrayList<>(deletedGuids.size());
+        for (ITypedReferenceableInstance deletedEntity : deletedEntitiesFromListener) {
+            deletedGuidsFromListener.add(deletedEntity.getId()._getId());
+        }
+        Assert.assertEquals(deletedGuidsFromListener, deletedGuids);
+    }
+    
+    private static class DeleteEntitiesChangeListener implements EntityChangeListener {
+        
+        private Collection<ITypedReferenceableInstance> deletedEntities_;
+        
+        @Override
+        public void onEntitiesAdded(Collection<ITypedReferenceableInstance> entities)
+            throws AtlasException {
+        }
+
+        @Override
+        public void onEntitiesUpdated(Collection<ITypedReferenceableInstance> entities)
+            throws AtlasException {
+        }
+
+        @Override
+        public void onTraitAdded(ITypedReferenceableInstance entity, IStruct trait)
+            throws AtlasException {
+        }
+
+        @Override
+        public void onTraitDeleted(ITypedReferenceableInstance entity, String traitName)
+            throws AtlasException {
+        }
+
+        @Override
+        public void onEntitiesDeleted(Collection<ITypedReferenceableInstance> entities)
+            throws AtlasException {
+            deletedEntities_ = entities;
+        }
+        
+        public Collection<ITypedReferenceableInstance> getDeletedEntities() {
+            return deletedEntities_;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/server-api/src/main/java/org/apache/atlas/listener/EntityChangeListener.java
----------------------------------------------------------------------
diff --git a/server-api/src/main/java/org/apache/atlas/listener/EntityChangeListener.java b/server-api/src/main/java/org/apache/atlas/listener/EntityChangeListener.java
index 619ed85..4bf1d05 100644
--- a/server-api/src/main/java/org/apache/atlas/listener/EntityChangeListener.java
+++ b/server-api/src/main/java/org/apache/atlas/listener/EntityChangeListener.java
@@ -66,4 +66,12 @@ public interface EntityChangeListener {
      * @throws AtlasException if the listener notification fails
      */
     void onTraitDeleted(ITypedReferenceableInstance entity, String traitName) throws AtlasException;
+    
+    /**
+     * This is upon deleting entities from the repository.
+     *
+     * @param entities the deleted entities
+     * @throws AtlasException
+     */
+    void onEntitiesDeleted(Collection<ITypedReferenceableInstance> entities) throws AtlasException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/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 a91c0a0..ab402d7 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
@@ -175,6 +175,15 @@ public interface MetadataService {
     void deleteTrait(String guid, String traitNameToBeDeleted) throws AtlasException;
 
     /**
+     * Delete the specified entities from the repository
+     * 
+     * @param guids entity guids to be deleted
+     * @return List of guids for deleted entities 
+     * @throws AtlasException
+     */
+    List<String> deleteEntities(List<String> guids) throws AtlasException;
+    
+    /**
      * Register a listener for entity change.
      *
      * @param listener  the listener to register

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/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 f5ab4d8..e11b036 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
@@ -21,14 +21,14 @@ package org.apache.atlas.web.resources;
 import com.google.common.base.Preconditions;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
+import org.apache.atlas.services.MetadataService;
+import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.exception.EntityExistsException;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.exception.TypeNotFoundException;
-import org.apache.atlas.utils.ParamChecker;
-import org.apache.atlas.services.MetadataService;
-import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.json.InstanceSerialization;
 import org.apache.atlas.typesystem.types.ValueConversionException;
+import org.apache.atlas.utils.ParamChecker;
 import org.apache.atlas.web.util.Servlets;
 import org.apache.commons.lang.StringUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -250,7 +250,7 @@ public class EntityResource {
             return updateEntityAttributeByGuid(guid, attribute, request);
         }
     }
-
+    
     private Response updateEntityPartialByGuid(String guid, HttpServletRequest request) {
         try {
             ParamChecker.notEmpty(guid, "Guid property cannot be null");
@@ -312,6 +312,37 @@ public class EntityResource {
     }
 
     /**
+     * Delete entities from the repository
+     * 
+     * @param guids deletion candidate guids
+     * @param request
+     * @return response payload as json
+     */
+    @DELETE
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public Response deleteEntities(@QueryParam("guid") List<String> guids, @Context HttpServletRequest request) {
+        
+        try {
+            List<String> deletedGuids = metadataService.deleteEntities(guids);
+            JSONObject response = new JSONObject();
+            response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
+            JSONArray guidArray = new JSONArray(deletedGuids.size());
+            for (String guid : deletedGuids) {
+                guidArray.put(guid);
+            }
+            response.put(AtlasClient.GUID, guidArray);
+            return Response.ok(response).build();
+        }  catch (AtlasException | IllegalArgumentException e) {
+            LOG.error("Unable to delete entities {}", guids, e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.BAD_REQUEST));
+        } catch (Throwable e) {
+            LOG.error("Unable to delete entities {}", guids, e);
+            throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
+        }
+    }
+
+    /**
      * Fetch the complete definition of an entity given its GUID.
      *
      * @param guid GUID for the entity

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/20608f02/webapp/src/test/java/org/apache/atlas/web/resources/EntityJerseyResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/EntityJerseyResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/EntityJerseyResourceIT.java
index f19463f..893099c 100755
--- a/webapp/src/test/java/org/apache/atlas/web/resources/EntityJerseyResourceIT.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/EntityJerseyResourceIT.java
@@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.inject.Inject;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasServiceException;
 import org.apache.atlas.notification.NotificationConsumer;
@@ -58,6 +59,7 @@ import org.testng.annotations.Test;
 
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.core.Response;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -698,4 +700,85 @@ public class EntityJerseyResourceIT extends BaseResourceIT {
         Assert.assertTrue(refs.get(0).equalsContents(columns.get(0)));
         Assert.assertTrue(refs.get(1).equalsContents(columns.get(1)));
     }
+    
+    @Test
+    public void testDeleteEntitiesViaRestApi() throws Exception {
+        // Create 2 database entities
+        Referenceable db1 = new Referenceable(DATABASE_TYPE);
+        db1.set("name", randomString());
+        db1.set("description", randomString());
+        Id db1Id = createInstance(db1);
+        Referenceable db2 = new Referenceable(DATABASE_TYPE);
+        db2.set("name", randomString());
+        db2.set("description", randomString());
+        Id db2Id = createInstance(db2);
+        
+        // Delete the database entities
+        ClientResponse clientResponse = service.path(ENTITIES).
+            queryParam(AtlasClient.GUID.toLowerCase(), db1Id._getId()).
+            queryParam(AtlasClient.GUID.toLowerCase(), db2Id._getId()).
+            accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE).method(HttpMethod.DELETE, ClientResponse.class);
+        Assert.assertEquals(clientResponse.getStatus(), Response.Status.OK.getStatusCode());
+
+        // Verify that response has guids for both database entities
+        JSONObject response = new JSONObject(clientResponse.getEntity(String.class));
+        final String deletedGuidsJson = response.getString(AtlasClient.GUID);
+        Assert.assertNotNull(deletedGuidsJson);
+        JSONArray guidsArray = new JSONArray(deletedGuidsJson);
+        Assert.assertEquals(guidsArray.length(), 2);
+        List<String> deletedGuidsList = new ArrayList<>(2);
+        for (int index = 0; index < guidsArray.length(); index++) {
+            deletedGuidsList.add(guidsArray.getString(index));
+        }
+        Assert.assertTrue(deletedGuidsList.contains(db1Id._getId()));
+        Assert.assertTrue(deletedGuidsList.contains(db2Id._getId()));
+
+        // Verify entities were deleted from the repository.
+        for (String guid : deletedGuidsList) {
+            try {
+                serviceClient.getEntity(guid);
+                Assert.fail(AtlasServiceException.class.getSimpleName() + 
+                    " was expected but not thrown.  The entity with guid " + guid + 
+                    " still exists in the repository after being deleted.");
+            }
+            catch (AtlasServiceException e) {
+                Assert.assertTrue(e.getMessage().contains(Integer.toString(Response.Status.NOT_FOUND.getStatusCode())));
+            }
+        }
+    }
+    
+    @Test
+    public void testDeleteEntitiesViaClientApi() throws Exception {
+        // Create 2 database entities
+        Referenceable db1 = new Referenceable(DATABASE_TYPE);
+        db1.set("name", randomString());
+        db1.set("description", randomString());
+        Id db1Id = createInstance(db1);
+        Referenceable db2 = new Referenceable(DATABASE_TYPE);
+        db2.set("name", randomString());
+        db2.set("description", randomString());
+        Id db2Id = createInstance(db2);
+        
+        // Delete the database entities
+        List<String> deletedGuidsList = serviceClient.deleteEntities(db1Id._getId(), db2Id._getId());
+        
+        // Verify that deleteEntities() response has database entity guids 
+        Assert.assertEquals(deletedGuidsList.size(), 2);
+        Assert.assertTrue(deletedGuidsList.contains(db1Id._getId()));   
+        Assert.assertTrue(deletedGuidsList.contains(db2Id._getId()));
+        
+        // Verify entities were deleted from the repository.
+        for (String guid : deletedGuidsList) {
+            try {
+                serviceClient.getEntity(guid);
+                Assert.fail(AtlasServiceException.class.getSimpleName() + 
+                    " was expected but not thrown.  The entity with guid " + guid + 
+                    " still exists in the repository after being deleted.");
+            }
+            catch (AtlasServiceException e) {
+                Assert.assertTrue(e.getMessage().contains(Integer.toString(Response.Status.NOT_FOUND.getStatusCode())));
+            }
+        }
+    }
+
 }