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 2015/12/03 06:50:52 UTC

[2/2] incubator-atlas git commit: ATLAS-171 Ability to update type definition(shwethags via sumasai)

ATLAS-171 Ability to update type definition(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/bf5672c5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/bf5672c5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/bf5672c5

Branch: refs/heads/master
Commit: bf5672c545ff2e076c0d1e0eac20526626478155
Parents: 919120f
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Thu Dec 3 11:19:45 2015 +0530
Committer: Suma Shivaprasad <su...@gmail.com>
Committed: Thu Dec 3 11:19:45 2015 +0530

----------------------------------------------------------------------
 .../hive/model/HiveDataModelGenerator.java      |   4 +-
 .../main/java/org/apache/atlas/AtlasClient.java |  62 +++++-
 .../atlas/listener/TypesChangeListener.java     |   4 +-
 pom.xml                                         |   1 +
 release-log.txt                                 |   1 +
 .../graph/GraphBackedSearchIndexer.java         |   5 +
 .../typestore/GraphBackedTypeStore.java         |   3 +-
 .../atlas/services/DefaultMetadataService.java  |  53 +++++-
 .../apache/atlas/services/MetadataService.java  |   8 +
 .../apache/atlas/BaseHiveRepositoryTest.java    |   6 +-
 .../test/java/org/apache/atlas/TestUtils.java   |   3 +-
 .../typestore/GraphBackedTypeStoreTest.java     |  64 ++++++-
 .../typesystem/types/AbstractDataType.java      |  10 +
 .../atlas/typesystem/types/AttributeInfo.java   |  37 ++++
 .../apache/atlas/typesystem/types/EnumType.java |  20 ++
 .../typesystem/types/HierarchicalType.java      |  30 ++-
 .../atlas/typesystem/types/IDataType.java       |   3 +
 .../atlas/typesystem/types/StructType.java      |  24 ++-
 .../atlas/typesystem/types/TypeSystem.java      | 189 ++++++++++++-------
 .../typesystem/types/TypeUpdateException.java   |  39 ++++
 .../atlas/typesystem/types/TypeUtils.java       |  57 ++++--
 .../atlas/typesystem/types/utils/TypesUtil.java |   9 +
 .../apache/atlas/typesystem/types/BaseTest.java |   9 +-
 .../atlas/typesystem/types/ClassTest.java       |  30 ++-
 .../apache/atlas/typesystem/types/EnumTest.java |  44 ++++-
 .../typesystem/types/HierarchicalTypeTest.java  |  84 +++++++++
 .../atlas/typesystem/types/StructTest.java      |  28 ++-
 .../atlas/typesystem/types/TraitTest.java       |  27 ++-
 .../typesystem/types/TypeUpdateBaseTest.java    |  98 ++++++++++
 .../org/apache/atlas/examples/QuickStart.java   |   3 +-
 .../atlas/web/resources/TypesResource.java      |  45 +++++
 .../atlas/web/resources/BaseResourceIT.java     |   3 +-
 .../web/resources/EntityJerseyResourceIT.java   |  36 +++-
 .../MetadataDiscoveryJerseyResourceIT.java      |   4 +-
 .../web/resources/TypesJerseyResourceIT.java    |  29 +++
 35 files changed, 936 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
index 2571295..994c813 100755
--- a/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
+++ b/addons/hive-bridge/src/main/java/org/apache/atlas/hive/model/HiveDataModelGenerator.java
@@ -34,7 +34,7 @@ import org.apache.atlas.typesystem.types.Multiplicity;
 import org.apache.atlas.typesystem.types.StructType;
 import org.apache.atlas.typesystem.types.StructTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
-import org.apache.atlas.typesystem.types.TypeUtils;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -102,7 +102,7 @@ public class HiveDataModelGenerator {
     }
 
     public TypesDef getTypesDef() {
-        return TypeUtils.getTypesDef(getEnumTypeDefinitions(), getStructTypeDefinitions(), getTraitTypeDefinitions(),
+        return TypesUtil.getTypesDef(getEnumTypeDefinitions(), getStructTypeDefinitions(), getTraitTypeDefinitions(),
                 getClassTypeDefinitions());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/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 7b6bcb7..2e8a268 100755
--- a/client/src/main/java/org/apache/atlas/AtlasClient.java
+++ b/client/src/main/java/org/apache/atlas/AtlasClient.java
@@ -25,7 +25,9 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 import org.apache.atlas.security.SecureClientUtils;
 import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.TypesDef;
 import org.apache.atlas.typesystem.json.InstanceSerialization;
+import org.apache.atlas.typesystem.json.TypesSerialization;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -132,6 +134,7 @@ public class AtlasClient {
 
         //Type operations
         CREATE_TYPE(BASE_URI + TYPES, HttpMethod.POST),
+        UPDATE_TYPE(BASE_URI + TYPES, HttpMethod.PUT),
         GET_TYPE(BASE_URI + TYPES, HttpMethod.GET),
         LIST_TYPES(BASE_URI + TYPES, HttpMethod.GET),
         LIST_TRAIT_TYPES(BASE_URI + TYPES + "?type=trait", HttpMethod.GET),
@@ -181,13 +184,45 @@ public class AtlasClient {
      * @return result json object
      * @throws AtlasServiceException
      */
-    public JSONObject createType(String typeAsJson) throws AtlasServiceException {
-        return callAPI(API.CREATE_TYPE, typeAsJson);
+    public List<String> createType(String typeAsJson) throws AtlasServiceException {
+        JSONObject response = callAPI(API.CREATE_TYPE, typeAsJson);
+        return extractResults(response, AtlasClient.TYPES);
+    }
+
+    /**
+     * Register the given type(meta model)
+     * @param typeDef type definition
+     * @return result json object
+     * @throws AtlasServiceException
+     */
+    public List<String> createType(TypesDef typeDef) throws AtlasServiceException {
+        return createType(TypesSerialization.toJson(typeDef));
+    }
+
+    /**
+     * Register the given type(meta model)
+     * @param typeAsJson type definition a jaon
+     * @return result json object
+     * @throws AtlasServiceException
+     */
+    public List<String> updateType(String typeAsJson) throws AtlasServiceException {
+        JSONObject response = callAPI(API.UPDATE_TYPE, typeAsJson);
+        return extractResults(response, AtlasClient.TYPES);
+    }
+
+    /**
+     * Register the given type(meta model)
+     * @param typeDef type definition
+     * @return result json object
+     * @throws AtlasServiceException
+     */
+    public List<String> updateType(TypesDef typeDef) throws AtlasServiceException {
+        return updateType(TypesSerialization.toJson(typeDef));
     }
 
     public List<String> listTypes() throws AtlasServiceException {
         final JSONObject jsonObject = callAPI(API.LIST_TYPES, null);
-        return extractResults(jsonObject);
+        return extractResults(jsonObject, AtlasClient.RESULTS);
     }
 
     public String getType(String typeName) throws AtlasServiceException {
@@ -230,6 +265,14 @@ public class AtlasClient {
         return createEntity(new JSONArray(Arrays.asList(entitiesAsJson)));
     }
 
+    public JSONArray createEntity(Referenceable... entities) throws AtlasServiceException {
+        JSONArray entityArray = new JSONArray(entities.length);
+        for (Referenceable entity : entities) {
+            entityArray.put(InstanceSerialization.toJson(entity, true));
+        }
+        return createEntity(entityArray);
+    }
+
     /**
      * Get an entity given the entity id
      * @param guid entity id
@@ -286,15 +329,20 @@ public class AtlasClient {
         WebResource resource = getResource(API.LIST_ENTITIES);
         resource = resource.queryParam(TYPE, entityType);
         JSONObject jsonResponse = callAPIWithResource(API.LIST_ENTITIES, resource);
-        return extractResults(jsonResponse);
+        return extractResults(jsonResponse, AtlasClient.RESULTS);
     }
 
-    private List<String> extractResults(JSONObject jsonResponse) throws AtlasServiceException {
+    private List<String> extractResults(JSONObject jsonResponse, String key) throws AtlasServiceException {
         try {
-            JSONArray results = jsonResponse.getJSONArray(AtlasClient.RESULTS);
+            JSONArray results = jsonResponse.getJSONArray(key);
             ArrayList<String> resultsList = new ArrayList<>();
             for (int index = 0; index < results.length(); index++) {
-                resultsList.add(results.getString(index));
+                Object element = results.get(index);
+                if (element instanceof String) {
+                    resultsList.add((String) element);
+                } else if (element instanceof JSONObject) {
+                    resultsList.add(((JSONObject) element).getString(AtlasClient.NAME));
+                }
             }
             return resultsList;
         } catch (JSONException e) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/common/src/main/java/org/apache/atlas/listener/TypesChangeListener.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/listener/TypesChangeListener.java b/common/src/main/java/org/apache/atlas/listener/TypesChangeListener.java
index dee396a..5ff6d4a 100644
--- a/common/src/main/java/org/apache/atlas/listener/TypesChangeListener.java
+++ b/common/src/main/java/org/apache/atlas/listener/TypesChangeListener.java
@@ -44,6 +44,6 @@ public interface TypesChangeListener {
      */
     // void onRemove(String typeName) throws MetadataException;
 
-    // This is upon updating an existing type to the store
-    // void onChange() throws MetadataException;
+     //This is upon updating an existing type to the store
+     void onChange(Collection<? extends IDataType> dataTypes) throws AtlasException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ede8c53..2f7ee82 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1426,6 +1426,7 @@
                     <reuseForks>false</reuseForks>
                     <forkCount>1</forkCount>
                     <threadCount>5</threadCount>
+                    <redirectTestOutputToFile>true</redirectTestOutputToFile>
                 </configuration>
                 <executions>
                     <execution>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 1c71dd1..9e3919d 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -9,6 +9,7 @@ ATLAS-54 Rename configs in hive hook (shwethags)
 ATLAS-3 Mixed Index creation fails with Date types (sumasai via shwethags)
 
 ALL CHANGES:
+ATLAS-171 Ability to update type definition(shwethags via sumasai)
 ATLAS-352 Improve write performance on type and entity creation with Hbase (sumasai)
 ATLAS-350 Document jaas config details for atlas (tbeerbower via shwethags)
 ATLAS-344 Document HBase permissions for secure cluster (tbeerbower via shwethags)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
index 10babed..3ea5fde 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
@@ -161,6 +161,11 @@ public class GraphBackedSearchIndexer implements SearchIndexer {
         commit();
     }
 
+    @Override
+    public void onChange(Collection<? extends IDataType> dataTypes) throws AtlasException {
+        onAdd(dataTypes);
+    }
+
     private void addIndexForType(IDataType dataType) {
         switch (dataType.getTypeCategory()) {
         case PRIMITIVE:

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java b/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
index eed297f..9789be7 100755
--- a/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
+++ b/repository/src/main/java/org/apache/atlas/repository/typestore/GraphBackedTypeStore.java
@@ -45,6 +45,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.utils.TypesUtil;
 import org.codehaus.jettison.json.JSONException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -253,7 +254,7 @@ public class GraphBackedTypeStore implements ITypeStore {
                 throw new IllegalArgumentException("Unhandled type category " + typeCategory);
             }
         }
-        return TypeUtils.getTypesDef(enums.build(), structs.build(), traits.build(), classTypes.build());
+        return TypesUtil.getTypesDef(enums.build(), structs.build(), traits.build(), classTypes.build());
     }
 
     private EnumTypeDefinition getEnumType(Vertex vertex) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/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 7dfe165..db61de9 100755
--- a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
@@ -50,7 +50,6 @@ import org.apache.atlas.typesystem.types.Multiplicity;
 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.ValueConversionException;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.codehaus.jettison.json.JSONArray;
@@ -148,8 +147,7 @@ public class DefaultMetadataService implements MetadataService {
 
     private void createType(HierarchicalTypeDefinition<ClassType> type) throws AtlasException {
         if (!typeSystem.isRegistered(type.typeName)) {
-            TypesDef typesDef = TypeUtils
-                    .getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+            TypesDef typesDef = TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
                             ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
                             ImmutableList.of(type));
             createType(TypesSerialization.toJson(typesDef));
@@ -191,6 +189,34 @@ public class DefaultMetadataService implements MetadataService {
         }
     }
 
+    @Override
+    public JSONObject updateType(String typeDefinition) throws AtlasException {
+        ParamChecker.notEmpty(typeDefinition, "type definition cannot be empty");
+        TypesDef typesDef = validateTypeDefinition(typeDefinition);
+
+        try {
+            final Map<String, IDataType> typesAdded = typeSystem.updateTypes(typesDef);
+
+            try {
+                /* Create indexes first so that if index creation fails then we rollback
+                   the typesystem and also do not persist the graph
+                 */
+                onTypesUpdated(typesAdded);
+                typeStore.store(typeSystem, ImmutableList.copyOf(typesAdded.keySet()));
+            } catch (Throwable t) {
+                typeSystem.removeTypes(typesAdded.keySet());
+                throw new AtlasException("Unable to persist types ", t);
+            }
+
+            return new JSONObject() {{
+                put(AtlasClient.TYPES, typesAdded.keySet());
+            }};
+        } catch (JSONException e) {
+            LOG.error("Unable to create response for types={}", typeDefinition, e);
+            throw new AtlasException("Unable to create response ", e);
+        }
+    }
+
     private TypesDef validateTypeDefinition(String typeDefinition) {
         try {
             TypesDef typesDef = TypesSerialization.fromJson(typeDefinition);
@@ -343,7 +369,7 @@ public class DefaultMetadataService implements MetadataService {
 
         repository.updateEntity(guid, property, value);
 
-        onEntityUpdated(repository.getEntityDefinition(guid), property, value);
+        onEntityUpdated(repository.getEntityDefinition(guid));
     }
 
     private void validateTypeExists(String entityType) throws AtlasException {
@@ -466,7 +492,24 @@ public class DefaultMetadataService implements MetadataService {
         }
     }
 
-    private void onEntityUpdated(ITypedReferenceableInstance entity, String property, String value)
+    private void onTypesUpdated(Map<String, IDataType> typesUpdated) throws AtlasException {
+        Map<TypesChangeListener, Throwable> caughtExceptions = new HashMap<>();
+        for (Provider<TypesChangeListener> indexerProvider : typeChangeListeners) {
+            final TypesChangeListener listener = indexerProvider.get();
+            try {
+                listener.onChange(typesUpdated.values());
+            } catch (IndexCreationException ice) {
+                LOG.error("Index creation for listener {} failed ", indexerProvider, ice);
+                caughtExceptions.put(listener, ice);
+            }
+        }
+
+        if (caughtExceptions.size() > 0) {
+            throw new IndexCreationException("Index creation failed for types " + typesUpdated.keySet() + ". Aborting");
+        }
+    }
+
+    private void onEntityUpdated(ITypedReferenceableInstance entity)
         throws AtlasException {
         for (EntityChangeListener listener : entityChangeListeners) {
             listener.onEntityUpdated(entity);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/repository/src/main/java/org/apache/atlas/services/MetadataService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/services/MetadataService.java b/repository/src/main/java/org/apache/atlas/services/MetadataService.java
index d048cc6..f027b79 100755
--- a/repository/src/main/java/org/apache/atlas/services/MetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/MetadataService.java
@@ -40,6 +40,14 @@ public interface MetadataService {
     JSONObject createType(String typeDefinition) throws AtlasException;
 
     /**
+     * Updates the given types in the type definition
+     * @param typeDefinition
+     * @return
+     * @throws AtlasException
+     */
+    JSONObject updateType(String typeDefinition) throws AtlasException;
+
+    /**
      * Return the definition for the given type.
      *
      * @param typeName name for this type, must be unique

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/repository/src/test/java/org/apache/atlas/BaseHiveRepositoryTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/BaseHiveRepositoryTest.java b/repository/src/test/java/org/apache/atlas/BaseHiveRepositoryTest.java
index 6b715ac..f9378e4 100644
--- a/repository/src/test/java/org/apache/atlas/BaseHiveRepositoryTest.java
+++ b/repository/src/test/java/org/apache/atlas/BaseHiveRepositoryTest.java
@@ -28,7 +28,6 @@ import org.apache.atlas.services.DefaultMetadataService;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
 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.persistence.Id;
 import org.apache.atlas.typesystem.types.AttributeDefinition;
@@ -41,10 +40,7 @@ import org.apache.atlas.typesystem.types.Multiplicity;
 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.utils.TypesUtil;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Guice;
 
 import javax.inject.Inject;
@@ -170,7 +166,7 @@ public class BaseHiveRepositoryTest {
 
         HierarchicalTypeDefinition<TraitType> jdbcTraitDef = TypesUtil.createTraitTypeDef("JdbcAccess", null);
 
-        return TypeUtils.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
             ImmutableList.of(dimTraitDef, factTraitDef, piiTraitDef, metricTraitDef, etlTraitDef, jdbcTraitDef),
             ImmutableList.of(dbClsDef, storageDescClsDef, columnClsDef, tblClsDef, loadProcessClsDef, viewClsDef, partClsDef));
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/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 564dbfe..01a8158 100755
--- a/repository/src/test/java/org/apache/atlas/TestUtils.java
+++ b/repository/src/test/java/org/apache/atlas/TestUtils.java
@@ -38,7 +38,6 @@ import org.apache.atlas.typesystem.types.Multiplicity;
 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.utils.TypesUtil;
 import org.apache.commons.lang.RandomStringUtils;
 import org.testng.Assert;
@@ -272,7 +271,7 @@ public final class TestUtils {
                 createTraitTypeDef("fetl" + CLASSIFICATION, ImmutableList.of(CLASSIFICATION),
                         createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
 
-        return TypeUtils.getTypesDef(ImmutableList.of(enumTypeDefinition),
+        return TypesUtil.getTypesDef(ImmutableList.of(enumTypeDefinition),
                 ImmutableList.of(structTypeDefinition, partitionDefinition),
                 ImmutableList.of(classificationTypeDefinition, fetlClassificationTypeDefinition, piiTypeDefinition),
                 ImmutableList.of(superTypeDefinition, databaseTypeDefinition, columnsDefinition, tableTypeDefinition));

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java b/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
index 720c7dd..8402eed 100755
--- a/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
@@ -18,13 +18,13 @@
 
 package org.apache.atlas.repository.typestore;
 
+import com.google.common.collect.ImmutableList;
 import com.thinkaurelius.titan.core.TitanGraph;
 import com.thinkaurelius.titan.core.util.TitanCleanup;
 import com.tinkerpop.blueprints.Direction;
 import com.tinkerpop.blueprints.Edge;
 import com.tinkerpop.blueprints.Vertex;
 import org.apache.atlas.AtlasException;
-import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.repository.graph.GraphHelper;
@@ -33,21 +33,30 @@ import org.apache.atlas.typesystem.TypesDef;
 import org.apache.atlas.typesystem.types.AttributeDefinition;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.EnumType;
 import org.apache.atlas.typesystem.types.EnumTypeDefinition;
 import org.apache.atlas.typesystem.types.EnumValue;
 import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
+import org.apache.atlas.typesystem.types.Multiplicity;
+import org.apache.atlas.typesystem.types.StructType;
 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.utils.TypesUtil;
+import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
-import org.testng.Assert;
 
 import javax.inject.Inject;
 import java.util.List;
 
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createStructTypeDef;
+
 @Guice(modules = RepositoryMetadataModule.class)
 public class GraphBackedTypeStoreTest {
     @Inject
@@ -77,7 +86,6 @@ public class GraphBackedTypeStoreTest {
     }
 
     @Test
-    @GraphTransaction
     public void testStore() throws AtlasException {
         typeStore.store(ts);
         dumpGraph();
@@ -137,4 +145,54 @@ public class GraphBackedTypeStoreTest {
         ts.reset();
         ts.defineTypes(types);
     }
+
+    @Test(dependsOnMethods = "testStore")
+    public void testTypeUpdate() throws Exception {
+        //Add enum value
+        EnumTypeDefinition orgLevelEnum = new EnumTypeDefinition("OrgLevel", new EnumValue("L1", 1),
+                new EnumValue("L2", 2), new EnumValue("L3", 3));
+
+        //Add attribute
+        StructTypeDefinition addressDetails =
+                createStructTypeDef("Address", createRequiredAttrDef("street", DataTypes.STRING_TYPE),
+                        createRequiredAttrDef("city", DataTypes.STRING_TYPE),
+                        createOptionalAttrDef("state", DataTypes.STRING_TYPE));
+
+        //Add supertype
+        HierarchicalTypeDefinition<ClassType> superTypeDef = createClassTypeDef("Division", ImmutableList.<String>of(),
+                createOptionalAttrDef("dname", DataTypes.STRING_TYPE));
+
+        HierarchicalTypeDefinition<ClassType> deptTypeDef = createClassTypeDef("Department",
+                ImmutableList.of(superTypeDef.typeName), createRequiredAttrDef("name", DataTypes.STRING_TYPE),
+                new AttributeDefinition("employees", String.format("array<%s>", "Person"), Multiplicity.COLLECTION,
+                        true, "department"));
+        TypesDef typesDef = TypesUtil.getTypesDef(ImmutableList.of(orgLevelEnum), ImmutableList.of(addressDetails),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+                ImmutableList.of(deptTypeDef, superTypeDef));
+
+        ts.updateTypes(typesDef);
+        typeStore.store(ts, ImmutableList.of(orgLevelEnum.name, addressDetails.typeName, superTypeDef.typeName,
+                deptTypeDef.typeName));
+
+        //Validate the updated types
+        TypesDef types = typeStore.restore();
+        ts.reset();
+        ts.defineTypes(types);
+
+        //Assert new enum value
+        EnumType orgLevel = ts.getDataType(EnumType.class, orgLevelEnum.name);
+        Assert.assertEquals(orgLevel.name, orgLevelEnum.name);
+        Assert.assertEquals(orgLevel.values().size(), orgLevelEnum.enumValues.length);
+        Assert.assertEquals(orgLevel.fromValue("L3").ordinal, 3);
+
+        //Assert new attribute
+        StructType addressType = ts.getDataType(StructType.class, addressDetails.typeName);
+        Assert.assertEquals(addressType.numFields, 3);
+        Assert.assertEquals(addressType.fieldMapping.fields.get("state").dataType(), DataTypes.STRING_TYPE);
+
+        //Assert new super type
+        ClassType deptType = ts.getDataType(ClassType.class, deptTypeDef.typeName);
+        Assert.assertTrue(deptType.superTypes.contains(superTypeDef.typeName));
+        Assert.assertNotNull(ts.getDataType(ClassType.class, superTypeDef.typeName));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/AbstractDataType.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AbstractDataType.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AbstractDataType.java
index f2d15eb..ed0400a 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AbstractDataType.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AbstractDataType.java
@@ -40,7 +40,17 @@ abstract class AbstractDataType<T> implements IDataType<T> {
         } else {
             TypeUtils.outputVal(val == null ? "<null>" : val.toString(), buf, prefix);
         }
+    }
 
+    /**
+     * Validate that current definition can be updated with the new definition
+     * @param newType
+     */
+    @Override
+    public void validateUpdate(IDataType newType) throws TypeUpdateException {
+        if (!getName().equals(newType.getName()) || !getClass().getName().equals(newType.getClass().getName())) {
+            throw new TypeUpdateException(newType);
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
index 3e1ef18..330d1cb 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/AttributeInfo.java
@@ -71,6 +71,43 @@ public class AttributeInfo {
                 '}';
     }
 
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        AttributeInfo that = (AttributeInfo) o;
+
+        if (isComposite != that.isComposite) {
+            return false;
+        }
+        if (isUnique != that.isUnique) {
+            return false;
+        }
+        if (isIndexable != that.isIndexable) {
+            return false;
+        }
+        if (!dataType.getName().equals(that.dataType.getName())) {
+            return false;
+        }
+        if (!multiplicity.equals(that.multiplicity)) {
+            return false;
+        }
+        if (!name.equals(that.name)) {
+            return false;
+        }
+        if (reverseAttributeName != null ? !reverseAttributeName.equals(that.reverseAttributeName) :
+                that.reverseAttributeName != null) {
+            return false;
+        }
+
+        return true;
+    }
+
     public String toJson() throws JSONException {
         JSONObject json = new JSONObject();
         json.put("name", name);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/EnumType.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/EnumType.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/EnumType.java
index f6927da..1439303 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/EnumType.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/EnumType.java
@@ -76,6 +76,26 @@ public class EnumType extends AbstractDataType<EnumValue> {
         return DataTypes.TypeCategory.ENUM;
     }
 
+    @Override
+    public void validateUpdate(IDataType newType) throws TypeUpdateException {
+        super.validateUpdate(newType);
+
+        EnumType enumType = (EnumType)newType;
+        for (EnumValue enumValue : values()) {
+            //The old enum value should be part of new enum definition as well
+            if (!enumType.valueMap.containsKey(enumValue.value)) {
+                throw new TypeUpdateException("Value " + enumValue.value + " is missing in new type");
+            }
+
+            //The ordinal for old enum value can't change
+            EnumValue newEnumValue = enumType.valueMap.get(enumValue.value);
+            if (enumValue.ordinal != newEnumValue.ordinal) {
+                throw new TypeUpdateException(String.format("Ordinal mismatch %s(%s) != %s(%s)", enumValue.value,
+                        enumValue.ordinal, newEnumValue.value, newEnumValue.ordinal));
+            }
+        }
+    }
+
     public EnumValue fromOrdinal(int o) {
         return ordinalMap.get(o);
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/HierarchicalType.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/HierarchicalType.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/HierarchicalType.java
index 5bb04c6..5a331a6 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/HierarchicalType.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/HierarchicalType.java
@@ -105,6 +105,30 @@ public abstract class HierarchicalType<ST extends HierarchicalType, T> extends A
         return (cType == this || cType.superTypePaths.containsKey(getName()));
     }
 
+    /**
+     * Validate that current definition can be updated with the new definition
+     * @param newType
+     * @return true if the current definition can be updated with the new definition, else false
+     */
+    @Override
+    public void validateUpdate(IDataType newType) throws TypeUpdateException {
+        super.validateUpdate(newType);
+
+        HierarchicalType newHierarchicalType = (HierarchicalType) newType;
+
+        //validate on supertypes
+        if (!newHierarchicalType.superTypes.containsAll(superTypes)) {
+            throw new TypeUpdateException(newType, "New type doesn't contain all super types of old type");
+        }
+
+        //validate on fields
+        try {
+            TypeUtils.validateUpdate(fieldMapping, newHierarchicalType.fieldMapping);
+        } catch (TypeUpdateException e) {
+            throw new TypeUpdateException(newType, e);
+        }
+    }
+
     protected void setupSuperTypesGraph() throws AtlasException {
         setupSuperTypesGraph(superTypes);
     }
@@ -147,9 +171,9 @@ public abstract class HierarchicalType<ST extends HierarchicalType, T> extends A
     protected Pair<FieldMapping, ImmutableMap<String, String>> constructFieldMapping(ImmutableList<String> superTypes,
             AttributeInfo... fields) throws AtlasException {
 
-        Map<String, AttributeInfo> fieldsMap = new LinkedHashMap<String, AttributeInfo>();
-        Map<String, Integer> fieldPos = new HashMap<String, Integer>();
-        Map<String, Integer> fieldNullPos = new HashMap<String, Integer>();
+        Map<String, AttributeInfo> fieldsMap = new LinkedHashMap();
+        Map<String, Integer> fieldPos = new HashMap();
+        Map<String, Integer> fieldNullPos = new HashMap();
         Map<String, String> attributeNameToType = new HashMap<>();
 
         int numBools = 0;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/IDataType.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/IDataType.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/IDataType.java
index 7cb2620..d9b1b34 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/IDataType.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/IDataType.java
@@ -28,4 +28,7 @@ public interface IDataType<T> {
     DataTypes.TypeCategory getTypeCategory();
 
     void output(T val, Appendable buf, String prefix) throws AtlasException;
+
+    void validateUpdate(IDataType newType) throws TypeUpdateException;
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/StructType.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/StructType.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/StructType.java
index 448827e..1a40484 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/StructType.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/StructType.java
@@ -18,7 +18,6 @@
 
 package org.apache.atlas.typesystem.types;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.IStruct;
 import org.apache.atlas.typesystem.ITypedStruct;
@@ -49,11 +48,11 @@ public class StructType extends AbstractDataType<IStruct> implements IConstructa
         this.handler = null;
     }
 
-    protected StructType(TypeSystem typeSystem, String name, ImmutableList<String> superTypes, AttributeInfo... fields)
+    protected StructType(TypeSystem typeSystem, String name, AttributeInfo... fields)
     throws AtlasException {
         this.typeSystem = typeSystem;
         this.name = name;
-        this.fieldMapping = constructFieldMapping(superTypes, fields);
+        this.fieldMapping = constructFieldMapping(fields);
         infoToNameMap = TypeUtils.buildAttrInfoToNameMap(this.fieldMapping);
         this.numFields = this.fieldMapping.fields.size();
         this.handler = new TypedStructHandler(this);
@@ -68,7 +67,24 @@ public class StructType extends AbstractDataType<IStruct> implements IConstructa
         return name;
     }
 
-    protected FieldMapping constructFieldMapping(ImmutableList<String> superTypes, AttributeInfo... fields)
+    /**
+     * Validate that current definition can be updated with the new definition
+     * @param newType
+     * @return true if the current definition can be updated with the new definition, else false
+     */
+    @Override
+    public void validateUpdate(IDataType newType) throws TypeUpdateException {
+        super.validateUpdate(newType);
+
+        StructType newStructType = (StructType) newType;
+        try {
+            TypeUtils.validateUpdate(fieldMapping, newStructType.fieldMapping);
+        } catch (TypeUpdateException e) {
+            throw new TypeUpdateException(newType, e);
+        }
+    }
+
+    protected FieldMapping constructFieldMapping(AttributeInfo... fields)
     throws AtlasException {
 
         Map<String, AttributeInfo> fieldsMap = new LinkedHashMap<String, AttributeInfo>();

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/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 a6367b1..d9b1edb 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
@@ -35,7 +35,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -193,7 +192,7 @@ public class TypeSystem {
             infos[i] = new AttributeInfo(this, attrDefs[i], tempTypes);
         }
 
-        return new StructType(this, name, null, infos);
+        return new StructType(this, name, infos);
     }
 
     public TraitType defineTraitType(HierarchicalTypeDefinition<TraitType> traitDef) throws AtlasException {
@@ -225,6 +224,18 @@ public class TypeSystem {
         return transientTypes.defineTypes();
     }
 
+    public Map<String, IDataType> updateTypes(TypesDef typesDef) throws AtlasException {
+        ImmutableList<EnumTypeDefinition> enumDefs = ImmutableList.copyOf(typesDef.enumTypesAsJavaList());
+        ImmutableList<StructTypeDefinition> structDefs = ImmutableList.copyOf(typesDef.structTypesAsJavaList());
+        ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs =
+                ImmutableList.copyOf(typesDef.traitTypesAsJavaList());
+        ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs =
+                ImmutableList.copyOf(typesDef.classTypesAsJavaList());
+
+        TransientTypeSystem transientTypes = new TransientTypeSystem(enumDefs, structDefs, traitDefs, classDefs);
+        return transientTypes.defineTypes(true);
+    }
+
     public Map<String, IDataType> defineTypes(TypesDef typesDef) throws AtlasException {
         ImmutableList<EnumTypeDefinition> enumDefs = ImmutableList.copyOf(typesDef.enumTypesAsJavaList());
         ImmutableList<StructTypeDefinition> structDefs = ImmutableList.copyOf(typesDef.structTypesAsJavaList());
@@ -241,17 +252,12 @@ public class TypeSystem {
             ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs,
             ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs) throws AtlasException {
         TransientTypeSystem transientTypes = new TransientTypeSystem(enumDefs, structDefs, traitDefs, classDefs);
-        Map<String, IDataType> definedTypes = transientTypes.defineTypes();
-        //        LOG.debug("Defined new types " + Arrays.toString(definedTypes.keySet().toArray(new
-        // String[definedTypes.size()])));
-        return definedTypes;
+        return transientTypes.defineTypes();
     }
 
     public DataTypes.ArrayType defineArrayType(IDataType elemType) throws AtlasException {
         assert elemType != null;
         DataTypes.ArrayType dT = new DataTypes.ArrayType(elemType);
-        //        types.put(dT.getName(), dT);
-        //        typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.ARRAY, dT.getName());
         return dT;
     }
 
@@ -259,8 +265,6 @@ public class TypeSystem {
         assert keyType != null;
         assert valueType != null;
         DataTypes.MapType dT = new DataTypes.MapType(keyType, valueType);
-        //        types.put(dT.getName(), dT);
-        //        typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.MAP, dT.getName());
         return dT;
     }
 
@@ -303,15 +307,16 @@ public class TypeSystem {
         final ImmutableList<HierarchicalTypeDefinition<TraitType>> traitDefs;
         final ImmutableList<HierarchicalTypeDefinition<ClassType>> classDefs;
         private final ImmutableList<EnumTypeDefinition> enumDefs;
+
         Map<String, StructTypeDefinition> structNameToDefMap = new HashMap<>();
         Map<String, HierarchicalTypeDefinition<TraitType>> traitNameToDefMap = new HashMap<>();
         Map<String, HierarchicalTypeDefinition<ClassType>> classNameToDefMap = new HashMap<>();
 
-        Set<String> transientTypes;
+        Map<String, IDataType> transientTypes = null;
 
-        List<AttributeInfo> recursiveRefs;
-        List<DataTypes.ArrayType> recursiveArrayTypes;
-        List<DataTypes.MapType> recursiveMapTypes;
+        List<AttributeInfo> recursiveRefs = new ArrayList<>();
+        List<DataTypes.ArrayType> recursiveArrayTypes = new ArrayList<>();
+        List<DataTypes.MapType> recursiveMapTypes = new ArrayList<>();
 
 
         TransientTypeSystem(ImmutableList<EnumTypeDefinition> enumDefs, ImmutableList<StructTypeDefinition> structDefs,
@@ -321,17 +326,13 @@ public class TypeSystem {
             this.structDefs = structDefs;
             this.traitDefs = traitDefs;
             this.classDefs = classDefs;
-            structNameToDefMap = new HashMap<>();
-            traitNameToDefMap = new HashMap<>();
-            classNameToDefMap = new HashMap<>();
-
-            recursiveRefs = new ArrayList<>();
-            recursiveArrayTypes = new ArrayList<>();
-            recursiveMapTypes = new ArrayList<>();
-            transientTypes = new LinkedHashSet<>();
+            transientTypes = new HashMap<>();
         }
 
-        private IDataType dataType(String name) {
+        private IDataType dataType(String name) throws AtlasException {
+            if (transientTypes.containsKey(name)) {
+                return transientTypes.get(name);
+            }
             return TypeSystem.this.types.get(name);
         }
 
@@ -340,52 +341,50 @@ public class TypeSystem {
          * - validate cannot redefine types
          * - setup shallow Type instances to facilitate recursive type graphs
          */
-        private void step1() throws AtlasException {
+        private void step1(boolean update) throws AtlasException {
             for (EnumTypeDefinition eDef : enumDefs) {
                 assert eDef.name != null;
-                if (types.containsKey(eDef.name)) {
+                if (!update && (transientTypes.containsKey(eDef.name) || types.containsKey(eDef.name))) {
                     throw new AtlasException(String.format("Redefinition of type %s not supported", eDef.name));
                 }
 
                 EnumType eT = new EnumType(this, eDef.name, eDef.enumValues);
-                TypeSystem.this.types.put(eDef.name, eT);
-                typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.ENUM, eDef.name);
-                transientTypes.add(eDef.name);
+                transientTypes.put(eDef.name, eT);
             }
 
             for (StructTypeDefinition sDef : structDefs) {
                 assert sDef.typeName != null;
-                if (dataType(sDef.typeName) != null) {
+                if (!update && (transientTypes.containsKey(sDef.typeName) || types.containsKey(sDef.typeName))) {
                     throw new TypeExistsException(String.format("Cannot redefine type %s", sDef.typeName));
                 }
-                TypeSystem.this.types
-                        .put(sDef.typeName, new StructType(this, sDef.typeName, sDef.attributeDefinitions.length));
+                StructType sT = new StructType(this, sDef.typeName, sDef.attributeDefinitions.length);
                 structNameToDefMap.put(sDef.typeName, sDef);
-                transientTypes.add(sDef.typeName);
+                transientTypes.put(sDef.typeName, sT);
             }
 
             for (HierarchicalTypeDefinition<TraitType> traitDef : traitDefs) {
                 assert traitDef.typeName != null;
-                if (types.containsKey(traitDef.typeName)) {
+                if (!update &&
+                        (transientTypes.containsKey(traitDef.typeName) || types.containsKey(traitDef.typeName))) {
                     throw new TypeExistsException(String.format("Cannot redefine type %s", traitDef.typeName));
                 }
-
-                TypeSystem.this.types.put(traitDef.typeName, new TraitType(this, traitDef.typeName, traitDef.superTypes,
-                                traitDef.attributeDefinitions.length));
+                TraitType tT = new TraitType(this, traitDef.typeName, traitDef.superTypes,
+                        traitDef.attributeDefinitions.length);
                 traitNameToDefMap.put(traitDef.typeName, traitDef);
-                transientTypes.add(traitDef.typeName);
+                transientTypes.put(traitDef.typeName, tT);
             }
 
             for (HierarchicalTypeDefinition<ClassType> classDef : classDefs) {
                 assert classDef.typeName != null;
-                if (types.containsKey(classDef.typeName)) {
+                if (!update &&
+                        (transientTypes.containsKey(classDef.typeName) || types.containsKey(classDef.typeName))) {
                     throw new TypeExistsException(String.format("Cannot redefine type %s", classDef.typeName));
                 }
 
-                TypeSystem.this.types.put(classDef.typeName, new ClassType(this, classDef.typeName, classDef.superTypes,
-                                classDef.attributeDefinitions.length));
+                ClassType cT = new ClassType(this, classDef.typeName, classDef.superTypes,
+                        classDef.attributeDefinitions.length);
                 classNameToDefMap.put(classDef.typeName, classDef);
-                transientTypes.add(classDef.typeName);
+                transientTypes.put(classDef.typeName, cT);
             }
         }
 
@@ -438,20 +437,20 @@ public class TypeSystem {
 
         private AttributeInfo constructAttributeInfo(AttributeDefinition attrDef) throws AtlasException {
             AttributeInfo info = new AttributeInfo(this, attrDef, null);
-            if (transientTypes.contains(attrDef.dataTypeName)) {
+            if (transientTypes.keySet().contains(attrDef.dataTypeName)) {
                 recursiveRefs.add(info);
             }
             if (info.dataType().getTypeCategory() == DataTypes.TypeCategory.ARRAY) {
                 DataTypes.ArrayType arrType = (DataTypes.ArrayType) info.dataType();
-                if (transientTypes.contains(arrType.getElemType().getName())) {
+                if (transientTypes.keySet().contains(arrType.getElemType().getName())) {
                     recursiveArrayTypes.add(arrType);
                 }
             }
             if (info.dataType().getTypeCategory() == DataTypes.TypeCategory.MAP) {
                 DataTypes.MapType mapType = (DataTypes.MapType) info.dataType();
-                if (transientTypes.contains(mapType.getKeyType().getName())) {
+                if (transientTypes.keySet().contains(mapType.getKeyType().getName())) {
                     recursiveMapTypes.add(mapType);
-                } else if (transientTypes.contains(mapType.getValueType().getName())) {
+                } else if (transientTypes.keySet().contains(mapType.getValueType().getName())) {
                     recursiveMapTypes.add(mapType);
                 }
             }
@@ -472,8 +471,8 @@ public class TypeSystem {
                 infos[i] = constructAttributeInfo(def.attributeDefinitions[i]);
             }
 
-            StructType type = new StructType(TypeSystem.this, def.typeName, null, infos);
-            TypeSystem.this.types.put(def.typeName, type);
+            StructType type = new StructType(this, def.typeName, infos);
+            transientTypes.put(def.typeName, type);
             return type;
         }
 
@@ -487,11 +486,12 @@ public class TypeSystem {
             try {
                 Constructor<U> cons = cls.getDeclaredConstructor(TypeSystem.class, String.class, ImmutableList.class,
                         AttributeInfo[].class);
-                U type = cons.newInstance(TypeSystem.this, def.typeName, def.superTypes, infos);
-                TypeSystem.this.types.put(def.typeName, type);
+                U type = cons.newInstance(this, def.typeName, def.superTypes, infos);
+                transientTypes.put(def.typeName, type);
                 return type;
             } catch (Exception e) {
-                throw new AtlasException(String.format("Cannot construct Type of MetaType %s", cls.getName()), e);
+                e.printStackTrace();
+                throw new AtlasException(String.format("Cannot construct Type of MetaType %s - %s", cls.getName(), def.typeName), e);
             }
         }
 
@@ -516,17 +516,14 @@ public class TypeSystem {
 
             for (StructTypeDefinition structDef : structDefs) {
                 constructStructureType(structDef);
-                typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.STRUCT, structDef.typeName);
             }
 
             for (TraitType traitType : traitTypes) {
                 constructHierarchicalType(TraitType.class, traitNameToDefMap.get(traitType.getName()));
-                typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.TRAIT, traitType.getName());
             }
 
             for (ClassType classType : classTypes) {
                 constructHierarchicalType(ClassType.class, classNameToDefMap.get(classType.getName()));
-                typeCategoriesToTypeNamesMap.put(DataTypes.TypeCategory.CLASS, classType.getName());
             }
         }
 
@@ -547,35 +544,89 @@ public class TypeSystem {
             }
         }
 
+        /**
+         * Step 5:
+         * - Validate that the update can be done
+         */
+        private void step5() throws TypeUpdateException {
+            //If the type is modified, validate that update can be done
+            for (IDataType newType : transientTypes.values()) {
+                if (TypeSystem.this.types.containsKey(newType.getName())) {
+                    IDataType oldType = TypeSystem.this.types.get(newType.getName());
+                    oldType.validateUpdate(newType);
+                }
+            }
+        }
+
         Map<String, IDataType> defineTypes() throws AtlasException {
-            try {
-                step1();
-                step2();
+            return defineTypes(false);
+        }
 
-                step3();
-                step4();
-            } catch (AtlasException me) {
-                for (String sT : transientTypes) {
-                    types.remove(sT);
-                }
-                throw me;
+        Map<String, IDataType> defineTypes(boolean update) throws AtlasException {
+            step1(update);
+            step2();
+
+            step3();
+            step4();
+
+            if (update) {
+                step5();
             }
 
             Map<String, IDataType> newTypes = new HashMap<>();
 
-            for (String tName : transientTypes) {
-                newTypes.put(tName, dataType(tName));
+            for (Map.Entry<String, IDataType> typeEntry : transientTypes.entrySet()) {
+                String typeName = typeEntry.getKey();
+                IDataType type = typeEntry.getValue();
+
+                //Add/replace the new type in the typesystem
+                TypeSystem.this.types.put(typeName, type);
+                typeCategoriesToTypeNamesMap.put(type.getTypeCategory(), typeName);
+
+                newTypes.put(typeName, type);
             }
             return newTypes;
         }
 
         @Override
         public ImmutableList<String> getTypeNames() {
-            return TypeSystem.this.getTypeNames();
+            Set<String> typeNames = transientTypes.keySet();
+            typeNames.addAll(TypeSystem.this.getTypeNames());
+            return ImmutableList.copyOf(typeNames);
         }
 
+        //get from transient types. Else, from main type system
         @Override
         public <T> T getDataType(Class<T> cls, String name) throws AtlasException {
+            if (transientTypes != null) {
+                if (transientTypes.containsKey(name)) {
+                    try {
+                        return cls.cast(transientTypes.get(name));
+                    } catch (ClassCastException cce) {
+                        throw new AtlasException(cce);
+                    }
+                }
+
+            /*
+             * is this an Array Type?
+             */
+                String arrElemType = TypeUtils.parseAsArrayType(name);
+                if (arrElemType != null) {
+                    IDataType dT = defineArrayType(getDataType(IDataType.class, arrElemType));
+                    return cls.cast(dT);
+                }
+
+            /*
+             * is this a Map Type?
+             */
+                String[] mapType = TypeUtils.parseAsMapType(name);
+                if (mapType != null) {
+                    IDataType dT =
+                            defineMapType(getDataType(IDataType.class, mapType[0]), getDataType(IDataType.class, mapType[1]));
+                    return cls.cast(dT);
+                }
+            }
+
             return TypeSystem.this.getDataType(cls, name);
         }
 
@@ -605,12 +656,12 @@ public class TypeSystem {
 
         @Override
         public DataTypes.ArrayType defineArrayType(IDataType elemType) throws AtlasException {
-            throw new AtlasException("Internal Error: define type called on TrasientTypeSystem");
+            return super.defineArrayType(elemType);
         }
 
         @Override
         public DataTypes.MapType defineMapType(IDataType keyType, IDataType valueType) throws AtlasException {
-            throw new AtlasException("Internal Error: define type called on TrasientTypeSystem");
+            return super.defineMapType(keyType, valueType);
         }
     }
 
@@ -631,7 +682,7 @@ public class TypeSystem {
                 infos[0] = new AttributeInfo(TypeSystem.this, idAttr, null);
                 infos[1] = new AttributeInfo(TypeSystem.this, typNmAttr, null);
 
-                StructType type = new StructType(TypeSystem.this, TYP_NAME, null, infos);
+                StructType type = new StructType(TypeSystem.this, TYP_NAME, infos);
                 TypeSystem.this.types.put(TYP_NAME, type);
 
             } catch (AtlasException me) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUpdateException.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUpdateException.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUpdateException.java
new file mode 100644
index 0000000..33d1cb5
--- /dev/null
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUpdateException.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.typesystem.types;
+
+import org.apache.atlas.AtlasException;
+
+public class TypeUpdateException extends AtlasException {
+    public TypeUpdateException(IDataType newType) {
+        super(newType.getName() + " can't be updated");
+    }
+
+    public TypeUpdateException(IDataType newType, Exception e) {
+        super(newType.getName() + " can't be updated - " + e.getMessage(), e);
+    }
+
+    public TypeUpdateException(String message) {
+        super(message);
+    }
+
+    public TypeUpdateException(IDataType newType, String message) {
+        super(newType.getName() + " can't be updated - " + message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUtils.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUtils.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUtils.java
index 7017a7e..9d2480b 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUtils.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/TypeUtils.java
@@ -19,16 +19,15 @@
 package org.apache.atlas.typesystem.types;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import org.apache.atlas.AtlasException;
-import org.apache.atlas.typesystem.TypesDef;
-import scala.collection.JavaConversions;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -59,23 +58,16 @@ public class TypeUtils {
     }
 
     public static Map<AttributeInfo, List<String>> buildAttrInfoToNameMap(FieldMapping f) {
-        Map<AttributeInfo, List<String>> b = new HashMap<AttributeInfo, List<String>>();
+        Map<AttributeInfo, List<String>> b = new HashMap();
         for (Map.Entry<String, AttributeInfo> e : f.fields.entrySet()) {
             List<String> names = b.get(e.getValue());
             if (names == null) {
-                names = new ArrayList<String>();
+                names = new ArrayList<>();
                 b.put(e.getValue(), names);
             }
             names.add(e.getKey());
         }
-        return ImmutableMap.copyOf(b);
-    }
-
-    public static TypesDef getTypesDef(ImmutableList<EnumTypeDefinition> enums,
-            ImmutableList<StructTypeDefinition> structs, ImmutableList<HierarchicalTypeDefinition<TraitType>> traits,
-            ImmutableList<HierarchicalTypeDefinition<ClassType>> classes) {
-        return new TypesDef(JavaConversions.asScalaBuffer(enums), JavaConversions.asScalaBuffer(structs),
-                JavaConversions.asScalaBuffer(traits), JavaConversions.asScalaBuffer(classes));
+        return b;
     }
 
     protected static class Pair<L, R> {
@@ -87,4 +79,43 @@ public class TypeUtils {
             this.right = right;
         }
     }
+
+    /**
+     * Validates that the old field mapping can be replaced with new field mapping
+     * @param oldFieldMapping
+     * @param newFieldMapping
+     */
+    public static void validateUpdate(FieldMapping oldFieldMapping, FieldMapping newFieldMapping)
+            throws TypeUpdateException {
+        Map<String, AttributeInfo> newFields = newFieldMapping.fields;
+        for (AttributeInfo attribute : oldFieldMapping.fields.values()) {
+            if (newFields.containsKey(attribute.name)) {
+                AttributeInfo newAttribute = newFields.get(attribute.name);
+                //If old attribute is also in new definition, only allowed change is multiplicity change from REQUIRED to OPTIONAL
+                if (!newAttribute.equals(attribute)) {
+                    if (attribute.multiplicity == Multiplicity.REQUIRED
+                            && newAttribute.multiplicity == Multiplicity.OPTIONAL) {
+                        continue;
+                    } else {
+                        throw new TypeUpdateException("Attribute " + attribute.name + " can't be updated");
+                    }
+                }
+
+            } else {
+                //If old attribute is missing in new definition, return false as attributes can't be deleted
+                throw new TypeUpdateException("Old Attribute " + attribute.name + " is missing");
+            }
+        }
+
+        //Only new attributes
+        Set<String> newAttributes = new HashSet<>(ImmutableList.copyOf(newFields.keySet()));
+        newAttributes.removeAll(oldFieldMapping.fields.keySet());
+        for (String attributeName : newAttributes) {
+            AttributeInfo newAttribute = newFields.get(attributeName);
+            //New required attribute can't be added
+            if (newAttribute.multiplicity == Multiplicity.REQUIRED) {
+                throw new TypeUpdateException("Can't add required attribute " + attributeName);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
----------------------------------------------------------------------
diff --git a/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java b/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
index 69956a3..ee82ce5 100755
--- a/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
+++ b/typesystem/src/main/java/org/apache/atlas/typesystem/types/utils/TypesUtil.java
@@ -19,6 +19,7 @@
 package org.apache.atlas.typesystem.types.utils;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.atlas.typesystem.TypesDef;
 import org.apache.atlas.typesystem.types.AttributeDefinition;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.EnumTypeDefinition;
@@ -28,6 +29,7 @@ import org.apache.atlas.typesystem.types.IDataType;
 import org.apache.atlas.typesystem.types.Multiplicity;
 import org.apache.atlas.typesystem.types.StructTypeDefinition;
 import org.apache.atlas.typesystem.types.TraitType;
+import scala.collection.JavaConversions;
 
 /**
  * Types utilities class.
@@ -74,4 +76,11 @@ public class TypesUtil {
             ImmutableList<String> superTypes, AttributeDefinition... attrDefs) {
         return new HierarchicalTypeDefinition<>(ClassType.class, name, superTypes, attrDefs);
     }
+
+    public static TypesDef getTypesDef(ImmutableList<EnumTypeDefinition> enums,
+                                       ImmutableList<StructTypeDefinition> structs, ImmutableList<HierarchicalTypeDefinition<TraitType>> traits,
+                                       ImmutableList<HierarchicalTypeDefinition<ClassType>> classes) {
+        return new TypesDef(JavaConversions.asScalaBuffer(enums), JavaConversions.asScalaBuffer(structs),
+                JavaConversions.asScalaBuffer(traits), JavaConversions.asScalaBuffer(classes));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
index a9fce04..93dd099 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/BaseTest.java
@@ -25,6 +25,7 @@ import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.Struct;
 import org.apache.atlas.typesystem.types.utils.TypesUtil;
+import org.apache.commons.lang.RandomStringUtils;
 import org.testng.annotations.BeforeMethod;
 
 import java.math.BigDecimal;
@@ -96,8 +97,8 @@ public abstract class BaseTest {
         System.out.println("defined recursiveStructType = " + recursiveStructType);
     }
 
-    protected Map<String, IDataType> defineTraits(HierarchicalTypeDefinition... tDefs) throws AtlasException {
-
+    protected Map<String, IDataType> defineTraits(HierarchicalTypeDefinition<TraitType>... tDefs)
+            throws AtlasException {
         return getTypeSystem().defineTraitTypes(tDefs);
     }
 
@@ -168,4 +169,8 @@ public abstract class BaseTest {
 
         return hrDept;
     }
+
+    protected String newName() {
+        return RandomStringUtils.randomAlphanumeric(10);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/ClassTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/ClassTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/ClassTest.java
index d394337..f454641 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/ClassTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/ClassTest.java
@@ -18,14 +18,17 @@
 
 package org.apache.atlas.typesystem.types;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.testng.Assert;
-import org.testng.annotations.Test;
 import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
 
-public class ClassTest extends BaseTest {
+public class ClassTest extends HierarchicalTypeTest<ClassType> {
 
     @BeforeMethod
     public void setup() throws Exception {
@@ -67,6 +70,29 @@ public class ClassTest extends BaseTest {
                 "\t\tlevel : \t\t1\n" +
                 "\t}}]\n" +
                 "}");
+    }
+
+    @Override
+    protected HierarchicalTypeDefinition<ClassType> getTypeDefinition(String name, AttributeDefinition... attributes) {
+        return new HierarchicalTypeDefinition(ClassType.class, name, null, attributes);
+    }
+
+    @Override
+    protected HierarchicalTypeDefinition<ClassType> getTypeDefinition(String name, ImmutableList<String> superTypes,
+                                                                      AttributeDefinition... attributes) {
+        return new HierarchicalTypeDefinition(ClassType.class, name, superTypes, attributes);
+    }
+
+    @Override
+    protected TypesDef getTypesDef(StructTypeDefinition typeDefinition) {
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+                ImmutableList.of((HierarchicalTypeDefinition<ClassType>) typeDefinition));
+    }
 
+    @Override
+    protected TypesDef getTypesDef(HierarchicalTypeDefinition<ClassType>... typeDefinitions) {
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(), ImmutableList.copyOf(typeDefinitions));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/EnumTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/EnumTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/EnumTest.java
index 2735f3c..ea7a798 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/EnumTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/EnumTest.java
@@ -27,9 +27,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.TypesDef;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.testng.Assert;
-import org.testng.annotations.Test;
 import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -59,7 +61,47 @@ public class EnumTest extends BaseTest {
 
         ts.defineEnumType("LockLevel", new EnumValue("DB", 1), new EnumValue("TABLE", 2),
                 new EnumValue("PARTITION", 3));
+    }
+
+    @Test
+    public void testTypeUpdate() throws Exception {
+        TypeSystem ts = getTypeSystem();
+        EnumTypeDefinition etd = new EnumTypeDefinition(newName(), new EnumValue("A", 1));
+        TypesDef typesDef = getTypesDef(etd);
+        ts.defineTypes(typesDef);
+
+        //Allow adding new enum
+        etd = new EnumTypeDefinition(etd.name, new EnumValue("A", 1), new EnumValue("B", 2));
+        typesDef = getTypesDef(etd);
+        ts.updateTypes(typesDef);
+
+        //Don't allow deleting enum
+        etd = new EnumTypeDefinition(etd.name, new EnumValue("A", 1));
+        typesDef = getTypesDef(etd);
+        try {
+            ts.updateTypes(typesDef);
+            Assert.fail("Expected TypeUpdateException");
+        } catch (TypeUpdateException e) {
+            //assert that type is not updated when validation fails
+            EnumType enumType = ts.getDataType(EnumType.class, etd.name);
+            Assert.assertEquals(enumType.values().size(), 2);
+        }
+
+        //Don't allow changing ordinal of existing enum value
+        etd = new EnumTypeDefinition(etd.name, new EnumValue("A", 2));
+        typesDef = getTypesDef(etd);
+        try {
+            ts.updateTypes(typesDef);
+            Assert.fail("Expected TypeUpdateException");
+        } catch (TypeUpdateException e) {
+            //expected
+        }
+    }
 
+    private TypesDef getTypesDef(EnumTypeDefinition enumTypeDefinition) {
+        return TypesUtil.getTypesDef(ImmutableList.of(enumTypeDefinition), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+                ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
     }
 
     protected void fillStruct(Struct s) throws AtlasException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/HierarchicalTypeTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/HierarchicalTypeTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/HierarchicalTypeTest.java
new file mode 100644
index 0000000..4e83492
--- /dev/null
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/HierarchicalTypeTest.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.typesystem.types;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public abstract class HierarchicalTypeTest<T extends HierarchicalType> extends TypeUpdateBaseTest {
+    @Test(enabled = false)
+    public void testTypeUpdate() throws Exception {
+        testTypeUpdateForAttributes();
+
+        //Test super types
+        HierarchicalTypeDefinition classType =
+                getTypeDefinition(newName(), TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE));
+        TypeSystem ts = getTypeSystem();
+        ts.defineTypes(getTypesDef(classType));
+
+        //Add super type with optional attribute
+        HierarchicalTypeDefinition superType =
+                getTypeDefinition(newName(), TypesUtil.createOptionalAttrDef("s", DataTypes.INT_TYPE));
+        classType = getTypeDefinition(classType.typeName, ImmutableList.of(superType.typeName),
+                TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE));
+        ts.updateTypes(getTypesDef(superType, classType));
+
+
+        //Add super type with required attribute should fail
+        HierarchicalTypeDefinition superTypeRequired =
+                getTypeDefinition(newName(), TypesUtil.createRequiredAttrDef("s", DataTypes.INT_TYPE));
+        classType = getTypeDefinition(classType.typeName,
+                ImmutableList.of(superTypeRequired.typeName, superType.typeName),
+                TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE));
+        try {
+            ts.updateTypes(getTypesDef(superTypeRequired, classType));
+            Assert.fail("Expected TypeUpdateException");
+        } catch (TypeUpdateException e) {
+            //expected
+        }
+
+        //Deleting super type should fail
+        classType = getTypeDefinition(classType.typeName, TypesUtil.createRequiredAttrDef("a", DataTypes.INT_TYPE));
+        try {
+            ts.updateTypes(getTypesDef(superType, classType));
+            Assert.fail("Expected TypeUpdateException");
+        } catch (TypeUpdateException e) {
+            //expected
+        }
+    }
+
+    @Override
+    protected abstract HierarchicalTypeDefinition<T> getTypeDefinition(String name, AttributeDefinition... attributes);
+
+    protected abstract HierarchicalTypeDefinition<T> getTypeDefinition(String name, ImmutableList<String> superTypes,
+                                                                       AttributeDefinition... attributes);
+
+    @Override
+    protected abstract TypesDef getTypesDef(StructTypeDefinition typeDefinition);
+
+    protected abstract TypesDef getTypesDef(HierarchicalTypeDefinition<T>... typeDefinitions);
+
+    @Override
+    protected int getNumberOfFields(TypeSystem ts, String typeName) throws Exception {
+        return ts.getDataType(HierarchicalType.class, typeName).numFields;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/StructTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/StructTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/StructTest.java
index 17ce4ea..b49e5ce 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/StructTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/StructTest.java
@@ -18,14 +18,17 @@
 
 package org.apache.atlas.typesystem.types;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.Struct;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.testng.Assert;
-import org.testng.annotations.Test;
 import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
 
-public class StructTest extends BaseTest {
+public class StructTest extends TypeUpdateBaseTest {
 
     StructType structType;
     StructType recursiveStructType;
@@ -78,4 +81,25 @@ public class StructTest extends BaseTest {
                 "}");
     }
 
+    @Test
+    public void testTypeUpdate() throws Exception {
+        testTypeUpdateForAttributes();
+    }
+
+    @Override
+    protected int getNumberOfFields(TypeSystem ts, String typeName) throws Exception {
+        return ts.getDataType(StructType.class, typeName).numFields;
+    }
+
+    @Override
+    protected StructTypeDefinition getTypeDefinition(String name, AttributeDefinition... attributes) {
+        return new StructTypeDefinition(name, attributes);
+    }
+
+    @Override
+    protected TypesDef getTypesDef(StructTypeDefinition typeDefinition) {
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.of(typeDefinition),
+                ImmutableList.<HierarchicalTypeDefinition<TraitType>>of(),
+                ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/bf5672c5/typesystem/src/test/java/org/apache/atlas/typesystem/types/TraitTest.java
----------------------------------------------------------------------
diff --git a/typesystem/src/test/java/org/apache/atlas/typesystem/types/TraitTest.java b/typesystem/src/test/java/org/apache/atlas/typesystem/types/TraitTest.java
index 8330ecb..4482816 100755
--- a/typesystem/src/test/java/org/apache/atlas/typesystem/types/TraitTest.java
+++ b/typesystem/src/test/java/org/apache/atlas/typesystem/types/TraitTest.java
@@ -23,9 +23,12 @@ import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.IStruct;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.Struct;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
+
 import java.util.HashMap;
 import java.util.Map;
 
@@ -33,7 +36,7 @@ import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAt
 import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
 import static org.apache.atlas.typesystem.types.utils.TypesUtil.createTraitTypeDef;
 
-public class TraitTest extends BaseTest {
+public class TraitTest extends HierarchicalTypeTest<TraitType> {
 
 
     @BeforeMethod
@@ -213,8 +216,30 @@ public class TraitTest extends BaseTest {
                 "\tA.C.D.c : \t3\n" +
                 "\tA.C.D.d : \t3\n" +
                 "}");
+    }
+
+    @Override
+    protected HierarchicalTypeDefinition<TraitType> getTypeDefinition(String name, AttributeDefinition... attributes) {
+        return new HierarchicalTypeDefinition(TraitType.class, name, null, attributes);
+    }
 
+    @Override
+    protected HierarchicalTypeDefinition<TraitType> getTypeDefinition(String name, ImmutableList<String> superTypes,
+                                                                      AttributeDefinition... attributes) {
+        return new HierarchicalTypeDefinition(TraitType.class, name, superTypes, attributes);
     }
 
+    @Override
+    protected TypesDef getTypesDef(StructTypeDefinition typeDefinition) {
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of((HierarchicalTypeDefinition<TraitType>) typeDefinition),
+                ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
+    }
+
+    @Override
+    protected TypesDef getTypesDef(HierarchicalTypeDefinition<TraitType>... typeDefinitions) {
+        return TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.copyOf(typeDefinitions), ImmutableList.<HierarchicalTypeDefinition<ClassType>>of());
+    }
 }