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 2016/07/25 21:19:32 UTC

incubator-atlas git commit: ATLAS-1049 List types by supertype (shwethags via sumasai)

Repository: incubator-atlas
Updated Branches:
  refs/heads/master fa47dd2d5 -> d934645b2


ATLAS-1049 List types by supertype (shwethags via 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/d934645b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/d934645b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/d934645b

Branch: refs/heads/master
Commit: d934645b294e390ad4690791e55932d71ea41391
Parents: fa47dd2
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Mon Jul 25 12:45:29 2016 -0700
Committer: Suma Shivaprasad <su...@gmail.com>
Committed: Mon Jul 25 12:45:29 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/atlas/AtlasClient.java |  54 +++++++
 release-log.txt                                 |   1 +
 .../atlas/services/DefaultMetadataService.java  |  24 +--
 .../apache/atlas/services/MetadataService.java  |  19 +--
 .../atlas/typesystem/types/TypeSystem.java      |  11 +-
 .../types/cache/DefaultTypeCache.java           | 125 +++++++++++----
 .../atlas/typesystem/types/cache/TypeCache.java |  21 +--
 .../types/cache/DefaultTypeCacheTest.java       | 157 ++++++++++++++-----
 .../atlas/web/resources/TypesResource.java      |  52 +++---
 .../atlas/web/resources/BaseResourceIT.java     |  20 +--
 .../web/resources/TypesJerseyResourceIT.java    |  28 +++-
 11 files changed, 352 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/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 32e1bb0..d92cabc 100755
--- a/client/src/main/java/org/apache/atlas/AtlasClient.java
+++ b/client/src/main/java/org/apache/atlas/AtlasClient.java
@@ -36,6 +36,7 @@ import org.apache.atlas.typesystem.TypesDef;
 import org.apache.atlas.typesystem.json.InstanceSerialization;
 import org.apache.atlas.typesystem.json.TypesSerialization;
 import org.apache.atlas.typesystem.types.AttributeDefinition;
+import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
@@ -107,6 +108,9 @@ public class AtlasClient {
     public static final String ATTRIBUTE_NAME = "property";
     public static final String ATTRIBUTE_VALUE = "value";
 
+    public static final String SUPERTYPE = "supertype";
+    public static final String NOT_SUPERTYPE = "notsupertype";
+
     public static final String ASSET_TYPE = "Asset";
     public static final String NAME = "name";
     public static final String DESCRIPTION = "description";
@@ -606,11 +610,61 @@ public class AtlasClient {
         return updateType(TypesSerialization.toJson(typeDef));
     }
 
+    /**
+     * Returns all type names in the system
+     * @return list of type names
+     * @throws AtlasServiceException
+     */
     public List<String> listTypes() throws AtlasServiceException {
         final JSONObject jsonObject = callAPI(API.LIST_TYPES, null);
         return extractResults(jsonObject, AtlasClient.RESULTS, new ExtractOperation<String, String>());
     }
 
+    /**
+     * Returns all type names with the given category
+     * @param category
+     * @return list of type names
+     * @throws AtlasServiceException
+     */
+    public List<String> listTypes(final DataTypes.TypeCategory category) throws AtlasServiceException {
+        JSONObject response = callAPIWithRetries(API.LIST_TYPES, null, new ResourceCreator() {
+            @Override
+            public WebResource createResource() {
+                WebResource resource = getResource(API.LIST_TYPES);
+                resource = resource.queryParam(TYPE, category.name());
+                return resource;
+            }
+        });
+        return extractResults(response, AtlasClient.RESULTS, new ExtractOperation<String, String>());
+    }
+
+    /**
+     * Return the list of type names in the type system which match the specified filter.
+     *
+     * @param category returns types whose category is the given typeCategory
+     * @param superType returns types which contain the given supertype
+     * @param notSupertype returns types which do not contain the given supertype
+     *
+     * Its possible to specify combination of these filters in one request and the conditions are combined with AND
+     * For example, typeCategory = TRAIT && supertype contains 'X' && supertype !contains 'Y'
+     * If there is no filter, all the types are returned
+     * @return list of type names
+     */
+    public List<String> listTypes(final DataTypes.TypeCategory category, final String superType,
+                                  final String notSupertype) throws AtlasServiceException {
+        JSONObject response = callAPIWithRetries(API.LIST_TYPES, null, new ResourceCreator() {
+            @Override
+            public WebResource createResource() {
+                WebResource resource = getResource(API.LIST_TYPES);
+                resource = resource.queryParam(TYPE, category.name());
+                resource = resource.queryParam(SUPERTYPE, superType);
+                resource = resource.queryParam(NOT_SUPERTYPE, notSupertype);
+                return resource;
+            }
+        });
+        return extractResults(response, AtlasClient.RESULTS, new ExtractOperation<String, String>());
+    }
+
     public TypesDef getType(String typeName) throws AtlasServiceException {
         try {
             JSONObject response = callAPI(API.GET_TYPE, null, typeName);;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index f0fff9a..f08b90a 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -6,6 +6,7 @@ INCOMPATIBLE CHANGES:
 
 
 ALL CHANGES:
+ATLAS-1049 List types by supertype (shwethags via sumasai)
 ATLAS-1032 Atlas hook package should not include libraries already present in host component - like log4j(mneethiraj via sumasai)
 ATLAS-1001 UI Paginate search APIs (kevalbhatt18 via sumasai)
 ATLAS-1042 Performance improvement changes for propertykey+typeName based queries (sumasai via shwethags)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/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 b870c62..e6b75f4 100755
--- a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
@@ -23,11 +23,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.inject.Provider;
 
-import com.thinkaurelius.titan.core.schema.TitanManagement;
-import com.tinkerpop.blueprints.Vertex;
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
-import org.apache.atlas.AtlasConstants;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.EntityAuditEvent;
 import org.apache.atlas.RequestContext;
@@ -302,23 +299,16 @@ public class DefaultMetadataService implements MetadataService, ActiveStateChang
     }
 
     /**
-     * Return the list of types in the repository.
+     * Return the list of type names in the type system which match the specified filter.
      *
-     * @return list of type names in the repository
+     * @return list of type names
+     * @param filterMap - Map of filter for type names. Valid keys are CATEGORY, SUPERTYPE, NOT_SUPERTYPE
+     * For example, CATEGORY = TRAIT && SUPERTYPE contains 'X' && SUPERTYPE !contains 'Y'
+     * If there is no filter, all the types are returned
      */
     @Override
-    public List<String> getTypeNamesList() throws AtlasException {
-        return typeSystem.getTypeNames();
-    }
-
-    /**
-     * Return the list of trait type names in the type system.
-     *
-     * @return list of trait type names in the type system
-     */
-    @Override
-    public List<String> getTypeNamesByCategory(DataTypes.TypeCategory typeCategory) throws AtlasException {
-        return typeSystem.getTypeNamesByCategory(typeCategory);
+    public List<String> getTypeNames(Map<TypeCache.TYPE_FILTER, String> filterMap) throws AtlasException {
+        return typeSystem.getTypeNames(filterMap);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/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 ed0f7fd..60d8790 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
@@ -26,10 +26,11 @@ import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.Struct;
-import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.cache.TypeCache;
 import org.codehaus.jettison.json.JSONObject;
 
 import java.util.List;
+import java.util.Map;
 
 /**
  * Metadata service.
@@ -62,18 +63,14 @@ public interface MetadataService {
     String getTypeDefinition(String typeName) throws AtlasException;
 
     /**
-     * Return the list of types in the type system.
+     * Return the list of type names in the type system which match the specified filter.
      *
-     * @return list of type names in the type system
+     * @return list of type names
+     * @param filterMap - Map of filter for type names. Valid keys are CATEGORY, SUPERTYPE, NOT_SUPERTYPE
+     * For example, CATEGORY = TRAIT && SUPERTYPE contains 'X' && SUPERTYPE !contains 'Y'
+     * If there is no filter, all the types are returned
      */
-    List<String> getTypeNamesList() throws AtlasException;
-
-    /**
-     * Return the list of trait type names in the type system.
-     *
-     * @return list of trait type names in the type system
-     */
-    List<String> getTypeNamesByCategory(DataTypes.TypeCategory typeCategory) throws AtlasException;
+    List<String> getTypeNames(Map<TypeCache.TYPE_FILTER, String> filterMap) throws AtlasException;
 
     /**
      * Creates an entity, instance of the type.

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
index 1fc4da7..70f9ea5 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeSystem.java
@@ -101,12 +101,17 @@ public class TypeSystem {
         return ImmutableList.copyOf(typeNames);
     }
 
-    public ImmutableList<String> getTypeNamesByCategory(DataTypes.TypeCategory typeCategory) throws AtlasException {
-        return ImmutableList.copyOf(typeCache.getTypeNames(typeCategory));
+    public ImmutableList<String> getTypeNamesByCategory(final DataTypes.TypeCategory typeCategory) throws AtlasException {
+        return getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+            put(TypeCache.TYPE_FILTER.CATEGORY, typeCategory.name());
+        }});
     }
 
-    private void registerPrimitiveTypes() {
+    public ImmutableList<String> getTypeNames(Map<TypeCache.TYPE_FILTER, String> filterMap) throws AtlasException {
+        return ImmutableList.copyOf(typeCache.getTypeNames(filterMap));
+    }
 
+    private void registerPrimitiveTypes() {
         coreTypes.put(DataTypes.BOOLEAN_TYPE.getName(), DataTypes.BOOLEAN_TYPE);
         coreTypes.put(DataTypes.BYTE_TYPE.getName(), DataTypes.BYTE_TYPE);
         coreTypes.put(DataTypes.SHORT_TYPE.getName(), DataTypes.SHORT_TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCache.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCache.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCache.java
index f683ed0..150418e 100644
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCache.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCache.java
@@ -17,22 +17,24 @@
  */
 package org.apache.atlas.typesystem.types.cache;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-
+import com.google.inject.Singleton;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
 import org.apache.atlas.typesystem.types.EnumType;
+import org.apache.atlas.typesystem.types.HierarchicalType;
 import org.apache.atlas.typesystem.types.IDataType;
 import org.apache.atlas.typesystem.types.StructType;
 import org.apache.atlas.typesystem.types.TraitType;
+import org.apache.commons.lang.StringUtils;
 
-import com.google.inject.Singleton;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Caches the types in-memory within the same process space.
@@ -42,6 +44,10 @@ import com.google.inject.Singleton;
 public class DefaultTypeCache implements TypeCache {
 
     private Map<String, IDataType> types_ = new ConcurrentHashMap<>();
+    private static final List<TypeCategory> validTypeFilterCategories =
+            Arrays.asList(TypeCategory.CLASS, TypeCategory.TRAIT, TypeCategory.ENUM, TypeCategory.STRUCT);
+    private static final List<TypeCategory> validSupertypeFilterCategories =
+            Arrays.asList(TypeCategory.CLASS, TypeCategory.TRAIT);
 
     /*
      * (non-Javadoc)
@@ -61,32 +67,29 @@ public class DefaultTypeCache implements TypeCache {
      */
     @Override
     public boolean has(TypeCategory typeCategory, String typeName)
-        throws AtlasException {
+            throws AtlasException {
 
         assertValidTypeCategory(typeCategory);
         return has(typeName);
     }
 
-    private void assertValidTypeCategory(TypeCategory typeCategory) throws
-        AtlasException {
+    private void assertValidTypeCategory(String typeCategory) {
+        assertValidTypeCategory(TypeCategory.valueOf(typeCategory));
+    }
 
+    private void assertValidTypeCategory(TypeCategory typeCategory) {
         // there might no need of 'typeCategory' in this implementation for
         // certain API, but for a distributed cache, it might help for the
         // implementers to partition the types per their category
         // while persisting so that look can be efficient
 
         if (typeCategory == null) {
-            throw new AtlasException("Category of the types to be filtered is null.");
+            throw new IllegalArgumentException("Category of the types to be filtered is null.");
         }
 
-        boolean validTypeCategory = typeCategory.equals(TypeCategory.CLASS) ||
-            typeCategory.equals(TypeCategory.TRAIT) ||
-            typeCategory.equals(TypeCategory.ENUM) ||
-            typeCategory.equals(TypeCategory.STRUCT);
-
-        if (!validTypeCategory) {
-            throw new AtlasException("Category of the types should be one of CLASS "
-                + "| TRAIT | ENUM | STRUCT.");
+        if (!validTypeFilterCategories.contains(typeCategory)) {
+            throw new IllegalArgumentException("Category of the types should be one of " +
+                    StringUtils.join(validTypeFilterCategories, ", "));
         }
     }
 
@@ -113,29 +116,83 @@ public class DefaultTypeCache implements TypeCache {
         return get(typeName);
     }
 
-    /*
-     * (non-Javadoc)
-     * @see
-     * org.apache.atlas.typesystem.types.cache.TypeCache#getNames(org
-     * .apache.atlas.typesystem.types.DataTypes.TypeCategory)
+    /**
+     * Return the list of type names in the type system which match the specified filter.
+     *
+     * @return list of type names
+     * @param filterMap - Map of filter for type names. Valid keys are CATEGORY, SUPERTYPE, NOT_SUPERTYPE
+     * For example, CATEGORY = TRAIT && SUPERTYPE contains 'X' && SUPERTYPE !contains 'Y'
      */
     @Override
-    public Collection<String> getTypeNames(TypeCategory typeCategory) throws AtlasException {
-
-        assertValidTypeCategory(typeCategory);
+    public Collection<String> getTypeNames(Map<TYPE_FILTER, String> filterMap) throws AtlasException {
+        assertFilter(filterMap);
 
         List<String> typeNames = new ArrayList<>();
-        for (Entry<String, IDataType> typeEntry : types_.entrySet()) {
-            String name = typeEntry.getKey();
-            IDataType type = typeEntry.getValue();
-
-            if (type.getTypeCategory().equals(typeCategory)) {
-                typeNames.add(name);
+        for (IDataType type : types_.values()) {
+            if (shouldIncludeType(type, filterMap)) {
+                typeNames.add(type.getName());
             }
         }
         return typeNames;
     }
 
+    private boolean shouldIncludeType(IDataType type, Map<TYPE_FILTER, String> filterMap) {
+        if (filterMap == null) {
+            return true;
+        }
+
+        for (Entry<TYPE_FILTER, String> filterEntry : filterMap.entrySet()) {
+            switch (filterEntry.getKey()) {
+            case CATEGORY:
+                if (!filterEntry.getValue().equals(type.getTypeCategory().name())) {
+                    return false;
+                }
+                break;
+
+            case SUPERTYPE:
+                if (!validSupertypeFilterCategories.contains(type.getTypeCategory()) ||
+                        !((HierarchicalType) type).getAllSuperTypeNames().contains(filterEntry.getValue())) {
+                    return false;
+                }
+                break;
+
+            case NOT_SUPERTYPE:
+                if (!validSupertypeFilterCategories.contains(type.getTypeCategory()) ||
+                        type.getName().equals(filterEntry.getValue()) ||
+                        ((HierarchicalType) type).getAllSuperTypeNames().contains(filterEntry.getValue())) {
+                    return false;
+                }
+                break;
+            }
+        }
+        return true;
+    }
+
+
+    private void assertFilter(Map<TYPE_FILTER, String> filterMap) throws AtlasException {
+        if (filterMap == null) {
+            return;
+        }
+
+        for (Entry<TYPE_FILTER, String> filterEntry : filterMap.entrySet()) {
+            switch (filterEntry.getKey()) {
+            case CATEGORY:
+                assertValidTypeCategory(filterEntry.getValue());
+                break;
+
+            case SUPERTYPE:
+            case NOT_SUPERTYPE:
+                if (!has(filterEntry.getValue())) {
+                    throw new IllegalArgumentException("Invalid supertype " + filterEntry.getValue());
+                }
+                break;
+
+            default:
+                throw new IllegalStateException("Unhandled filter " + filterEntry.getKey());
+            }
+        }
+    }
+
     /*
      * (non-Javadoc)
      * @see

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/TypeCache.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/TypeCache.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/TypeCache.java
index 27622c2..2bcbdd7 100644
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/TypeCache.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/cache/TypeCache.java
@@ -18,12 +18,13 @@
 
 package org.apache.atlas.typesystem.types.cache;
 
-import java.util.Collection;
-
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.IDataType;
 
+import java.util.Collection;
+import java.util.Map;
+
 /**
  * The types are cached to allow faster lookup when type info is needed during
  * creation/updation of entities, DSL query translation/execution.
@@ -39,6 +40,10 @@ import org.apache.atlas.typesystem.types.IDataType;
 @SuppressWarnings("rawtypes")
 public interface TypeCache {
 
+    enum TYPE_FILTER {
+        CATEGORY, SUPERTYPE, NOT_SUPERTYPE
+    }
+
     /**
      * @param typeName
      * @return true if the type exists in cache, false otherwise.
@@ -56,7 +61,7 @@ public interface TypeCache {
     boolean has(DataTypes.TypeCategory typeCategory, String typeName) throws AtlasException;
 
     /**
-     * @param name The name of the type.
+     * @param typeName The name of the type.
      * @return returns non-null type if cached, otherwise null
      * @throws AtlasException
      */
@@ -72,18 +77,16 @@ public interface TypeCache {
     public IDataType get(DataTypes.TypeCategory typeCategory, String typeName) throws AtlasException;
 
     /**
-     * @param typeCategory The category of types to filter the returned types. Cannot be null.
-     * The category can be one of TypeCategory.CLASS | TypeCategory.TRAIT |
-     * TypeCategory.STRUCT | TypeCategory.ENUM.
-     * @return
+     *
+     * @param filter @return
      * @throws AtlasException
      */
-    Collection<String> getTypeNames(DataTypes.TypeCategory typeCategory) throws AtlasException;
+    Collection<String> getTypeNames(Map<TYPE_FILTER, String> filter) throws AtlasException;
 
     /**
      * This is a convenience API to get the names of all types.
      *
-     * @see TypeCache#getTypeNames(org.apache.atlas.typesystem.types.DataTypes.TypeCategory)
+     * @see TypeCache#getTypeNames(Map)
      * @return
      * @throws AtlasException
      */

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/typesystem/src/test/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCacheTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCacheTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCacheTest.java
index 5ed0d5e..335023f 100644
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCacheTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/cache/DefaultTypeCacheTest.java
@@ -17,18 +17,8 @@
  */
 package org.apache.atlas.typesystem.types.cache;
 
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertNotNull;
-import static org.testng.Assert.assertNull;
-import static org.testng.Assert.assertTrue;
-import static org.testng.Assert.fail;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.atlas.AtlasException;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
@@ -44,7 +34,19 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import com.google.common.collect.ImmutableSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
 
 /**
  * Tests functional behavior of {@link DefaultTypeCache}
@@ -186,25 +188,31 @@ public class DefaultTypeCacheTest {
         assertEquals(ENUMTYPE_SHIPPING, allTypeNames.get(3));
     }
 
+    private Collection<String> getTypeNamesByCategory(final TypeCategory category)
+            throws AtlasException {
+        return cache.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+            put(TypeCache.TYPE_FILTER.CATEGORY, category.name());
+        }});
+    }
+
     @Test
     public void testCacheGetTypeNamesByCategory() throws Exception {
-
-        List<String> classTypes = new ArrayList<String>(cache.getTypeNames(TypeCategory.CLASS));
+        List<String> classTypes = new ArrayList(getTypeNamesByCategory(TypeCategory.CLASS));
         final int EXPECTED_CLASSTYPE_COUNT = 1;
         assertEquals(classTypes.size(), EXPECTED_CLASSTYPE_COUNT);
         assertEquals(CLASSTYPE_CUSTOMER, classTypes.get(0));
 
-        List<String> structTypes = new ArrayList<String>(cache.getTypeNames(TypeCategory.STRUCT));
+        List<String> structTypes = new ArrayList(getTypeNamesByCategory(TypeCategory.STRUCT));
         final int EXPECTED_STRUCTTYPE_COUNT = 1;
         assertEquals(structTypes.size(), EXPECTED_STRUCTTYPE_COUNT);
         assertEquals(STRUCTTYPE_ADDRESS, structTypes.get(0));
 
-        List<String> traitTypes = new ArrayList<String>(cache.getTypeNames(TypeCategory.TRAIT));
+        List<String> traitTypes = new ArrayList(getTypeNamesByCategory(TypeCategory.TRAIT));
         final int EXPECTED_TRAITTYPE_COUNT = 1;
         assertEquals(traitTypes.size(), EXPECTED_TRAITTYPE_COUNT);
         assertEquals(TRAITTYPE_PRIVILEGED, traitTypes.get(0));
 
-        List<String> enumTypes = new ArrayList<String>(cache.getTypeNames(TypeCategory.ENUM));
+        List<String> enumTypes = new ArrayList(getTypeNamesByCategory(TypeCategory.ENUM));
         final int EXPECTED_ENUMTYPE_COUNT = 1;
         assertEquals(enumTypes.size(), EXPECTED_ENUMTYPE_COUNT);
         assertEquals(ENUMTYPE_SHIPPING, enumTypes.get(0));
@@ -238,11 +246,10 @@ public class DefaultTypeCacheTest {
 
     @Test
     public void testCacheRemove() throws Exception {
-
         cache.remove(CLASSTYPE_CUSTOMER);
         assertNull(cache.get(CLASSTYPE_CUSTOMER));
         assertFalse(cache.has(CLASSTYPE_CUSTOMER));
-        assertTrue(cache.getTypeNames(TypeCategory.CLASS).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.CLASS).isEmpty());
 
         final int EXPECTED_TYPE_COUNT = 3;
         assertEquals(cache.getAllTypeNames().size(), EXPECTED_TYPE_COUNT);
@@ -254,7 +261,7 @@ public class DefaultTypeCacheTest {
         cache.remove(TypeCategory.CLASS, CLASSTYPE_CUSTOMER);
         assertNull(cache.get(CLASSTYPE_CUSTOMER));
         assertFalse(cache.has(CLASSTYPE_CUSTOMER));
-        assertTrue(cache.getTypeNames(TypeCategory.CLASS).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.CLASS).isEmpty());
 
         final int EXPECTED_TYPE_COUNT = 3;
         assertEquals(cache.getAllTypeNames().size(), EXPECTED_TYPE_COUNT);
@@ -277,10 +284,10 @@ public class DefaultTypeCacheTest {
         assertNull(cache.get(ENUMTYPE_SHIPPING));
         assertFalse(cache.has(ENUMTYPE_SHIPPING));
 
-        assertTrue(cache.getTypeNames(TypeCategory.CLASS).isEmpty());
-        assertTrue(cache.getTypeNames(TypeCategory.STRUCT).isEmpty());
-        assertTrue(cache.getTypeNames(TypeCategory.TRAIT).isEmpty());
-        assertTrue(cache.getTypeNames(TypeCategory.ENUM).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.CLASS).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.STRUCT).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.TRAIT).isEmpty());
+        assertTrue(getTypeNamesByCategory(TypeCategory.ENUM).isEmpty());
 
         assertTrue(cache.getAllTypeNames().isEmpty());
     }
@@ -299,45 +306,37 @@ public class DefaultTypeCacheTest {
         fail("type should only be an instance of ClassType | EnumType | StructType | TraitType in 'put'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testGetTypeWithNullCategory() throws Exception {
 
         cache.get(null, CLASSTYPE_CUSTOMER);
         fail("Null TypeCategory should be not allowed in 'get'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testGetTypeWithInvalidCategory() throws Exception {
 
         cache.get(TypeCategory.PRIMITIVE, DataTypes.BOOLEAN_TYPE.getName());
         fail("TypeCategory should only be one of TypeCategory.CLASS | ENUM | STRUCT | TRAIT in 'get'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testCacheHasTypeWithNullCategory() throws Exception {
 
         cache.has(null, CLASSTYPE_CUSTOMER);
         fail("Null TypeCategory should be not allowed in 'has'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testCacheHasTypeWithInvalidCategory() throws Exception {
 
         cache.has(TypeCategory.PRIMITIVE, DataTypes.BOOLEAN_TYPE.getName());
         fail("TypeCategory should only be one of TypeCategory.CLASS | ENUM | STRUCT | TRAIT in 'has'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
-    public void testCacheGetTypeNamesByNullCategory() throws Exception {
-
-        cache.getTypeNames(null);
-        fail("Null TypeCategory should be not allowed in 'getNames'");
-    }
-
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testCacheGetTypeNamesByInvalidCategory() throws Exception {
-
-        cache.getTypeNames(TypeCategory.PRIMITIVE);
+        getTypeNamesByCategory(TypeCategory.PRIMITIVE);
         fail("TypeCategory should only be one of TypeCategory.CLASS | ENUM | STRUCT | TRAIT in 'getNames'");
     }
 
@@ -367,17 +366,93 @@ public class DefaultTypeCacheTest {
         fail("type should only one of ClassType | EnumType | StructType | TraitType in 'putAll'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testCacheRemoveByNullCategory() throws Exception {
 
         cache.remove(null, CLASSTYPE_CUSTOMER);
         fail("Null type should be not allowed in 'remove'");
     }
 
-    @Test(expectedExceptions = AtlasException.class)
+    @Test(expectedExceptions = IllegalArgumentException.class)
     public void testCacheRemoveByInvalidCategory() throws Exception {
 
         cache.remove(TypeCategory.PRIMITIVE, DataTypes.BOOLEAN_TYPE.getName());
         fail("TypeCategory should only be one of TypeCategory.CLASS | ENUM | STRUCT | TRAIT in 'remove'");
     }
+
+    @Test
+    public void testGetTypesByFilter() throws Exception {
+        // init TypeSystem
+        TypeSystem ts = TypeSystem.getInstance().reset();
+
+        ts.defineClassType(TypesUtil.createClassTypeDef("A", ImmutableSet.<String>of()));
+        ts.defineClassType(TypesUtil.createClassTypeDef("A1", ImmutableSet.of("A")));
+
+        ts.defineClassType(TypesUtil.createClassTypeDef("B", ImmutableSet.<String>of()));
+
+        ts.defineClassType(TypesUtil.createClassTypeDef("C", ImmutableSet.of("B", "A")));
+
+        //supertype ~ A
+        ImmutableList<String> results = ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+                    put(TypeCache.TYPE_FILTER.SUPERTYPE, "A");
+                }});
+        assertTrue(results.containsAll(Arrays.asList("A1", "C")), "Results: " + results);
+
+        //!supertype doesn't return the type itself
+        results = ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+            put(TypeCache.TYPE_FILTER.NOT_SUPERTYPE, "A");
+        }});
+        assertTrue(results.containsAll(Arrays.asList("B")), "Results: " + results);
+
+        //supertype ~ A && supertype !~ B
+        results = ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+            put(TypeCache.TYPE_FILTER.SUPERTYPE, "A");
+            put(TypeCache.TYPE_FILTER.NOT_SUPERTYPE, "B");
+        }});
+        assertTrue(results.containsAll(Arrays.asList("A1")), "Results: " + results);
+
+        //none of category trait
+        results = ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+            put(TypeCache.TYPE_FILTER.CATEGORY, TypeCategory.TRAIT.name());
+            put(TypeCache.TYPE_FILTER.SUPERTYPE, "A");
+        }});
+        assertTrue(results.isEmpty(), "Results: " + results);
+
+        //no filter returns all types
+        results = ts.getTypeNames(null);
+        assertTrue(results.containsAll(Arrays.asList("A", "A1", "B", "C")), "Results: " + results);
+
+        results = ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>());
+        assertTrue(results.containsAll(Arrays.asList("A", "A1", "B", "C")), "Results: " + results);
+
+        //invalid category
+        try {
+            ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+                put(TypeCache.TYPE_FILTER.CATEGORY, "A");
+            }});
+            fail("Expected IllegalArgumentException");
+        } catch (IllegalArgumentException e) {
+            //expected
+        }
+
+        //invalid supertype
+        try {
+            ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+                put(TypeCache.TYPE_FILTER.SUPERTYPE, "X");
+            }});
+            fail("Expected IllegalArgumentException");
+        } catch (IllegalArgumentException e) {
+            //expected
+        }
+
+        //invalid supertype
+        try {
+            ts.getTypeNames(new HashMap<TypeCache.TYPE_FILTER, String>() {{
+                put(TypeCache.TYPE_FILTER.NOT_SUPERTYPE, "X");
+            }});
+            fail("Expected IllegalArgumentException");
+        } catch (IllegalArgumentException e) {
+            //expected
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
index b1e51fa..ace0d14 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/TypesResource.java
@@ -23,9 +23,10 @@ import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.typesystem.exception.TypeExistsException;
-import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.cache.TypeCache;
 import org.apache.atlas.utils.AtlasPerfTracer;
 import org.apache.atlas.web.util.Servlets;
+import org.apache.commons.lang.StringUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
@@ -36,7 +37,6 @@ import javax.inject.Inject;
 import javax.inject.Singleton;
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
-import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
@@ -48,7 +48,9 @@ import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * This class provides RESTful API for Types.
@@ -67,8 +69,6 @@ public class TypesResource {
 
     private final MetadataService metadataService;
 
-    static final String TYPE_ALL = "all";
-
     @Inject
     public TypesResource(MetadataService metadataService) {
         this.metadataService = metadataService;
@@ -209,30 +209,33 @@ public class TypesResource {
     }
 
     /**
-     * Gets the list of trait type names registered in the type system.
+     * Return the list of type names in the type system which match the specified filter.
+     *
+     * @return list of type names
+     * @param typeCategory returns types whose category is the given typeCategory
+     * @param supertype returns types which contain the given supertype
+     * @param notsupertype returns types which do not contain the given supertype
      *
-     * @param type type should be the name of enum
-     *             org.apache.atlas.typesystem.types.DataTypes.TypeCategory
-     *             Typically, would be one of all, TRAIT, CLASS, ENUM, STRUCT
-     * @return entity names response payload as json
+     * Its possible to specify combination of these filters in one request and the conditions are combined with AND
+     * For example, typeCategory = TRAIT && supertype contains 'X' && supertype !contains 'Y'
+     * If there is no filter, all the types are returned
      */
     @GET
     @Produces(Servlets.JSON_MEDIA_TYPE)
-    public Response getTypesByFilter(@Context HttpServletRequest request,
-            @DefaultValue(TYPE_ALL) @QueryParam("type") String type) {
+    public Response getTypesByFilter(@Context HttpServletRequest request, @QueryParam("type") String typeCategory,
+                                     @QueryParam("supertype") String supertype,
+                                     @QueryParam("notsupertype") String notsupertype) {
         AtlasPerfTracer perf = null;
         try {
             if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) {
-                perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "TypesResource.getTypesByFilter(" + type + ")");
+                perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "TypesResource.getTypesByFilter(" + typeCategory + ")");
             }
 
-            List<String> result;
-            if (TYPE_ALL.equals(type)) {
-                result = metadataService.getTypeNamesList();
-            } else {
-                DataTypes.TypeCategory typeCategory = DataTypes.TypeCategory.valueOf(type);
-                result = metadataService.getTypeNamesByCategory(typeCategory);
-            }
+            Map<TypeCache.TYPE_FILTER, String> filterMap = new HashMap<>();
+            addToFilterIfNotEmpty(filterMap, TypeCache.TYPE_FILTER.CATEGORY, typeCategory);
+            addToFilterIfNotEmpty(filterMap, TypeCache.TYPE_FILTER.SUPERTYPE, supertype);
+            addToFilterIfNotEmpty(filterMap, TypeCache.TYPE_FILTER.NOT_SUPERTYPE, notsupertype);
+            List<String> result = metadataService.getTypeNames(filterMap);
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.RESULTS, new JSONArray(result));
@@ -241,9 +244,9 @@ public class TypesResource {
 
             return Response.ok(response).build();
         } catch (IllegalArgumentException | AtlasException ie) {
-            LOG.error("Unsupported typeName while retrieving type list {}", type);
+            LOG.error("Unsupported typeName while retrieving type list {}", typeCategory);
             throw new WebApplicationException(
-                    Servlets.getErrorResponse(new Exception("Unsupported type " + type, ie), Response.Status.BAD_REQUEST));
+                    Servlets.getErrorResponse(new Exception("Unsupported type " + typeCategory, ie), Response.Status.BAD_REQUEST));
         } catch (Throwable e) {
             LOG.error("Unable to get types list", e);
             throw new WebApplicationException(Servlets.getErrorResponse(e, Response.Status.INTERNAL_SERVER_ERROR));
@@ -251,4 +254,11 @@ public class TypesResource {
             AtlasPerfTracer.log(perf);
         }
     }
+
+    private void addToFilterIfNotEmpty(Map<TypeCache.TYPE_FILTER, String> filterMap, TypeCache.TYPE_FILTER filterType,
+                                       String filterValue) {
+        if (StringUtils.isNotEmpty(filterValue)) {
+            filterMap.put(filterType, filterValue);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
index ffc5249..6b54fcd 100755
--- a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
@@ -22,7 +22,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import kafka.consumer.ConsumerTimeoutException;
@@ -51,18 +50,14 @@ import org.apache.atlas.typesystem.types.TypeUtils;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.apache.atlas.utils.AuthenticationUtil;
 import org.apache.atlas.utils.ParamChecker;
-import org.apache.atlas.web.util.Servlets;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang.RandomStringUtils;
 import org.codehaus.jettison.json.JSONArray;
-import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriBuilder;
 import java.util.List;
 
@@ -109,19 +104,8 @@ public abstract class BaseResourceIT {
         }
     }
 
-    protected void createType(String typesAsJSON) throws Exception {
-        WebResource resource = service.path("api/atlas/types");
-
-        ClientResponse clientResponse = resource.accept(Servlets.JSON_MEDIA_TYPE).type(Servlets.JSON_MEDIA_TYPE)
-                .method(HttpMethod.POST, ClientResponse.class, typesAsJSON);
-        Assert.assertEquals(clientResponse.getStatus(), Response.Status.CREATED.getStatusCode());
-
-        String responseAsString = clientResponse.getEntity(String.class);
-        Assert.assertNotNull(responseAsString);
-
-        JSONObject response = new JSONObject(responseAsString);
-        Assert.assertNotNull(response.get("types"));
-        Assert.assertNotNull(response.get(AtlasClient.REQUEST_ID));
+    protected List<String> createType(String typesAsJSON) throws Exception {
+        return serviceClient.createType(TypesSerialization.fromJson(typesAsJSON));
     }
 
     protected Id createInstance(Referenceable referenceable) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d934645b/webapp/src/test/java/org/apache/atlas/web/resources/TypesJerseyResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/TypesJerseyResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/TypesJerseyResourceIT.java
index 947e138..43584f9 100755
--- a/webapp/src/test/java/org/apache/atlas/web/resources/TypesJerseyResourceIT.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/TypesJerseyResourceIT.java
@@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 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.typesystem.TypesDef;
@@ -47,10 +46,11 @@ import org.testng.annotations.Test;
 
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.core.Response;
-
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.fail;
 
@@ -126,7 +126,7 @@ public class TypesJerseyResourceIT extends BaseResourceIT {
         typeDefinition = TypesUtil.createClassTypeDef(typeDefinition.typeName,
             ImmutableSet.<String>of(),
                 TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
-                TypesUtil.createOptionalAttrDef("description", DataTypes.STRING_TYPE));
+                createOptionalAttrDef("description", DataTypes.STRING_TYPE));
         TypesDef typeDef = TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(),
                 ImmutableList.<StructTypeDefinition>of(), ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
                 ImmutableList.of(typeDefinition));
@@ -224,6 +224,22 @@ public class TypesJerseyResourceIT extends BaseResourceIT {
         Assert.assertTrue(list.length() >= traitsAdded.length);
     }
 
+    @Test
+    public void testListTypesByFilter() throws Exception {
+        AttributeDefinition attr = TypesUtil.createOptionalAttrDef("attr", DataTypes.STRING_TYPE);
+        String a = createType(TypesSerialization.toJson(
+                TypesUtil.createClassTypeDef("A" + randomString(), ImmutableSet.<String>of(), attr), false)).get(0);
+        String a1 = createType(TypesSerialization.toJson(
+                TypesUtil.createClassTypeDef("A1" + randomString(), ImmutableSet.of(a), attr), false)).get(0);
+        String b = createType(TypesSerialization.toJson(
+                TypesUtil.createClassTypeDef("B" + randomString(), ImmutableSet.<String>of(), attr), false)).get(0);
+        String c = createType(TypesSerialization.toJson(
+                TypesUtil.createClassTypeDef("C" + randomString(), ImmutableSet.of(a, b), attr), false)).get(0);
+
+        List<String> results = serviceClient.listTypes(DataTypes.TypeCategory.CLASS, a, b);
+        assertEquals(results, Arrays.asList(a1), "Results: " + results);
+    }
+
     private String[] addTraits() throws Exception {
         String[] traitNames = {"class_trait", "secure_trait", "pii_trait", "ssn_trait", "salary_trait", "sox_trait",};
 
@@ -250,9 +266,9 @@ public class TypesJerseyResourceIT extends BaseResourceIT {
                 .createClassTypeDef("table", ImmutableSet.<String>of(),
                         TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
                         TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE),
-                        TypesUtil.createOptionalAttrDef("columnNames", DataTypes.arrayTypeName(DataTypes.STRING_TYPE)),
-                        TypesUtil.createOptionalAttrDef("created", DataTypes.DATE_TYPE), TypesUtil
-                                .createOptionalAttrDef("parameters",
+                        createOptionalAttrDef("columnNames", DataTypes.arrayTypeName(DataTypes.STRING_TYPE)),
+                        createOptionalAttrDef("created", DataTypes.DATE_TYPE),
+                        createOptionalAttrDef("parameters",
                                         DataTypes.mapTypeName(DataTypes.STRING_TYPE, DataTypes.STRING_TYPE)),
                         TypesUtil.createRequiredAttrDef("type", DataTypes.STRING_TYPE),
                         new AttributeDefinition("database", "database", Multiplicity.REQUIRED, false, "database"));