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/11/14 21:02:01 UTC

[2/2] incubator-atlas git commit: ATLAS-1257 Map Entity REST APIs to ATLAS v1 backend (sumasai)

ATLAS-1257 Map Entity REST APIs to ATLAS v1 backend (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/33d60746
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/33d60746
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/33d60746

Branch: refs/heads/master
Commit: 33d60746ab6be129251dabae7a3b104c8533e095
Parents: 758b3d4
Author: Suma Shivaprasad <su...@gmail.com>
Authored: Mon Nov 14 13:01:39 2016 -0800
Committer: Suma Shivaprasad <su...@gmail.com>
Committed: Mon Nov 14 13:01:39 2016 -0800

----------------------------------------------------------------------
 distro/src/bin/atlas_stop.py                    |   4 +-
 .../java/org/apache/atlas/AtlasErrorCode.java   |   8 +-
 .../atlas/exception/AtlasBaseException.java     |  11 +
 .../atlas/model/instance/AtlasEntity.java       |  44 +++-
 .../atlas/model/instance/AtlasObjectId.java     |   2 +-
 .../model/instance/EntityMutationResponse.java  |   2 +-
 .../org/apache/atlas/type/AtlasTypeUtil.java    |  11 +-
 .../test/java/org/apache/atlas/TestUtilsV2.java |  19 +-
 release-log.txt                                 |   1 +
 .../atlas/services/DefaultMetadataService.java  |  32 ++-
 .../service/DefaultMetadataServiceTest.java     |   6 +-
 .../apache/atlas/services/MetadataService.java  |  25 ++-
 .../web/adapters/AtlasArrayFormatConverter.java |  78 +++++++
 .../web/adapters/AtlasEnumFormatConverter.java  |  40 ++++
 .../atlas/web/adapters/AtlasFormatAdapter.java  |  31 +++
 .../web/adapters/AtlasFormatConverters.java     |  88 ++++++++
 .../adapters/AtlasFormatConvertersModule.java   |  48 ++++
 .../web/adapters/AtlasInstanceRestAdapters.java | 156 +++++++++++++
 .../web/adapters/AtlasMapFormatConverter.java   |  74 ++++++
 .../adapters/AtlasPrimitiveFormatConverter.java |  51 +++++
 .../v1/ReferenceableToAtlasEntityConverter.java | 111 +++++++++
 .../v1/StructToAtlasStructConverter.java        | 114 ++++++++++
 .../v1/TraitToAtlasClassificationConverter.java |  64 ++++++
 .../v2/AtlasClassificationToTraitConverter.java |  53 +++++
 .../v2/AtlasEntityToReferenceableConverter.java | 108 +++++++++
 .../v2/AtlasStructToStructConverter.java        | 124 ++++++++++
 .../atlas/web/listeners/GuiceServletConfig.java |   4 +-
 .../atlas/web/resources/EntityResource.java     |   6 +-
 .../org/apache/atlas/web/rest/EntitiesREST.java |  78 ++++++-
 .../org/apache/atlas/web/rest/EntityRest.java   | 225 +++++++++++++++++--
 .../atlas/web/adapters/TestEntitiesREST.java    | 216 ++++++++++++++++++
 .../atlas/web/adapters/TestEntityREST.java      | 181 +++++++++++++++
 32 files changed, 1956 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/distro/src/bin/atlas_stop.py
----------------------------------------------------------------------
diff --git a/distro/src/bin/atlas_stop.py b/distro/src/bin/atlas_stop.py
index a2b1c31..a25d25a 100755
--- a/distro/src/bin/atlas_stop.py
+++ b/distro/src/bin/atlas_stop.py
@@ -64,9 +64,9 @@ def main():
     if mc.is_hbase_local(confdir):
         mc.run_hbase_action(mc.hbaseBinDir(atlas_home), "stop", None, None, True)
 		
-    #after 30 seconds kill it 
-    time.sleep(30)
     if mc.exist_pid(pid):
+        #after 30 seconds kill it
+        time.sleep(30)
         try:
             sys.stderr.write("did not stop gracefully after 30 seconds seconds: killing with SIGKILL\n")
             os.kill(pid, SIGKILL)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
index 8e0d164..6a35d2b 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -49,6 +49,7 @@ public enum AtlasErrorCode {
     INSTANCE_GUID_NOT_FOUND(404, "ATLAS4045E", "Given instance guid {0} is invalid"),
     INSTANCE_LINEAGE_INVALID_PARAMS(404, "ATLAS4046E", "Invalid lineage query parameters passed {0}: {1}"),
     INSTANCE_LINEAGE_QUERY_FAILED(404, "ATLAS4047E", "Instance lineage query failed {0}"),
+    INSTANCE_ALREADY_EXISTS(405, "ATLAS4051E", "Given entity with guid/name {0} already exists"),
 
     TYPE_ALREADY_EXISTS(409, "ATLAS4091E", "Given type {0} already exists"),
     TYPE_HAS_REFERENCES(409, "ATLAS4092E", "Given type {0} has references"),
@@ -57,9 +58,12 @@ public enum AtlasErrorCode {
     INTERNAL_ERROR(500, "ATLAS5001E", "Internal server error {0}"),
     INDEX_CREATION_FAILED(500, "ATLAS5002E", "Index creation failed for {0}"),
     INDEX_ROLLBACK_FAILED(500, "ATLAS5003E", "Index rollback failed for {0}"),
+
     PATCH_NOT_APPLICABLE_FOR_TYPE(500, "ATLAS5004E", "{0} - invalid patch for type {1}"),
-    PATCH_FOR_UNKNOWN_TYPE(500, "ATLAS5005E", "{0} - patch references unknown type {1}")
-    ;
+    PATCH_FOR_UNKNOWN_TYPE(500, "ATLAS5005E", "{0} - patch references unknown type {1}"),
+    ATTRIBUTE_UNIQUE_INVALID(400, "ATLAS40015E", "Type {0} with unique attribute {1} does not exist"),
+    TYPE_NAME_INVALID(400, "ATLAS40016E", "Type {0} with name {1} does not exist"),
+    TYPE_CATEGORY_INVALID(400, "ATLAS40017E", "Type Category {0} does not match {1} or is invalid");
 
     private String errorCode;
     private String errorMessage;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/exception/AtlasBaseException.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/exception/AtlasBaseException.java b/intg/src/main/java/org/apache/atlas/exception/AtlasBaseException.java
index b88d4e8..5538702 100644
--- a/intg/src/main/java/org/apache/atlas/exception/AtlasBaseException.java
+++ b/intg/src/main/java/org/apache/atlas/exception/AtlasBaseException.java
@@ -20,6 +20,7 @@ package org.apache.atlas.exception;
 import org.apache.atlas.AtlasErrorCode;
 
 import javax.ws.rs.core.Response;
+import java.util.List;
 
 /**
  * Base Exception class for Atlas API.
@@ -33,6 +34,11 @@ public class AtlasBaseException extends Exception {
         this.atlasErrorCode = errorCode;
     }
 
+    public AtlasBaseException(final AtlasErrorCode errorCode, final List<String> params) {
+        super(errorCode.getFormattedErrorMessage(params.toArray(new String[params.size()])));
+        this.atlasErrorCode = errorCode;
+    }
+
     public AtlasBaseException() {
         this(AtlasErrorCode.INTERNAL_ERROR);
     }
@@ -69,6 +75,11 @@ public class AtlasBaseException extends Exception {
         this.atlasErrorCode = AtlasErrorCode.INTERNAL_ERROR;
     }
 
+    public AtlasBaseException(final AtlasErrorCode errorCode, Throwable cause, final List<String> params) {
+        super(errorCode.getFormattedErrorMessage(params.toArray(new String[params.size()])), cause);
+        this.atlasErrorCode = errorCode;
+    }
+
     public AtlasErrorCode getAtlasErrorCode() {
         return atlasErrorCode;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntity.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntity.java b/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntity.java
index 6727f27..5288cbf 100644
--- a/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntity.java
+++ b/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntity.java
@@ -21,6 +21,8 @@ import java.io.Serializable;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -33,6 +35,8 @@ import org.apache.atlas.model.typedef.AtlasEntityDef;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
 import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+
+import org.codehaus.jackson.annotate.JsonIgnore;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
@@ -61,6 +65,9 @@ public class AtlasEntity extends AtlasStruct implements Serializable {
     private Date   updateTime = null;
     private Long   version    = null;
 
+    @JsonIgnore
+    private static AtomicLong s_nextId = new AtomicLong(System.nanoTime());
+
     public AtlasEntity() {
         this(null, null);
     }
@@ -76,7 +83,7 @@ public class AtlasEntity extends AtlasStruct implements Serializable {
     public AtlasEntity(String typeName, Map<String, Object> attributes) {
         super(typeName, attributes);
 
-        setGuid(null);
+        setGuid(nextInternalId());
         setStatus(null);
         setCreatedBy(null);
         setUpdatedBy(null);
@@ -239,4 +246,39 @@ public class AtlasEntity extends AtlasStruct implements Serializable {
             super(list, startIndex, pageSize, totalCount, sortType, sortBy);
         }
     }
+
+    @JsonIgnore
+    public boolean validate(String id) {
+        try {
+            long l = Long.parseLong(id);
+            return l < 0;
+        } catch (NumberFormatException ne) {
+            return false;
+        }
+    }
+
+    @JsonIgnore
+    public boolean isUnassigned() {
+        return guid != null && guid.length() > 0 && guid.charAt(0) == '-';
+    }
+
+    @JsonIgnore
+    public boolean isAssigned() {
+        return isAssigned(guid);
+    }
+
+    @JsonIgnore
+    public static boolean isAssigned(String guid) {
+        try {
+            UUID.fromString(guid);
+        } catch (IllegalArgumentException e) {
+            return false;
+        }
+
+        return true;
+    }
+
+    private String nextInternalId() {
+        return "-" + Long.toString(s_nextId.getAndIncrement());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java b/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
index 5921fea..bedd4d9 100644
--- a/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
+++ b/intg/src/main/java/org/apache/atlas/model/instance/AtlasObjectId.java
@@ -31,6 +31,7 @@ import org.apache.atlas.model.SearchFilter.SortType;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
 import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
@@ -140,7 +141,6 @@ public class AtlasObjectId  implements Serializable {
         return toString(new StringBuilder()).toString();
     }
 
-
     /**
      * REST serialization friendly list.
      */

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/model/instance/EntityMutationResponse.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/instance/EntityMutationResponse.java b/intg/src/main/java/org/apache/atlas/model/instance/EntityMutationResponse.java
index a6c75ed..72f4118 100644
--- a/intg/src/main/java/org/apache/atlas/model/instance/EntityMutationResponse.java
+++ b/intg/src/main/java/org/apache/atlas/model/instance/EntityMutationResponse.java
@@ -61,7 +61,7 @@ public class EntityMutationResponse {
         this.entitiesMutated = opVsEntityMap;
     }
 
-    List<AtlasEntityHeader> getEntitiesByOperation(EntityMutations.EntityOperation op) {
+    public List<AtlasEntityHeader> getEntitiesByOperation(EntityMutations.EntityOperation op) {
         if ( entitiesMutated != null) {
             return entitiesMutated.get(op);
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
index a53b11d..6eed60f 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
@@ -32,13 +32,14 @@ import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+
 import java.util.Map;
 import java.util.Set;
+import java.util.Arrays;
 
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_PREFIX;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_SUFFIX;
@@ -112,21 +113,21 @@ public class AtlasTypeUtil {
     public static AtlasAttributeDef createOptionalAttrDef(String name, AtlasType dataType) {
         return new AtlasAttributeDef(name, dataType.getTypeName(), true,
             Cardinality.SINGLE, 0, 1,
-            true, false,
+            false, false,
             Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
     }
 
     public static AtlasAttributeDef createOptionalAttrDef(String name, String dataType) {
         return new AtlasAttributeDef(name, dataType, true,
             Cardinality.SINGLE, 0, 1,
-            true, false,
+            false, false,
             Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
     }
 
     public static AtlasAttributeDef createRequiredAttrDef(String name, String dataType) {
         return new AtlasAttributeDef(name, dataType, false,
             Cardinality.SINGLE, 1, 1,
-            false, false,
+            false, true,
             Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
     }
 
@@ -147,7 +148,7 @@ public class AtlasTypeUtil {
     public static AtlasAttributeDef createRequiredAttrDef(String name, AtlasType dataType) {
         return new AtlasAttributeDef(name, dataType.getTypeName(), false,
             Cardinality.SINGLE, 1, 1,
-            false, false,
+            false, true,
             Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
----------------------------------------------------------------------
diff --git a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
index bae9d3b..ea6d410 100755
--- a/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
+++ b/intg/src/test/java/org/apache/atlas/TestUtilsV2.java
@@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.atlas.model.instance.AtlasEntity;
-import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.model.instance.AtlasStruct;
 import org.apache.atlas.model.typedef.AtlasClassificationDef;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
 import org.apache.atlas.model.typedef.AtlasEnumDef;
@@ -35,6 +35,8 @@ import org.apache.commons.lang.RandomStringUtils;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.sql.Struct;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
@@ -432,7 +434,8 @@ public final class TestUtilsV2 {
                 AtlasTypeUtil.createClassTypeDef(COLUMN_TYPE, COLUMN_TYPE + "_description",
                         ImmutableSet.<String>of(),
                         AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
-                        AtlasTypeUtil.createRequiredAttrDef("type", "string"));
+                        AtlasTypeUtil.createRequiredAttrDef("type", "string")
+                        );
 
         AtlasStructDef partitionDefinition = new AtlasStructDef("partition_struct_type", "partition_struct_type" + _description, "1.0",
                 Arrays.asList(AtlasTypeUtil.createRequiredAttrDef("name", "string")));
@@ -614,8 +617,18 @@ public final class TestUtilsV2 {
         entity.setAttribute("description", "random table");
         entity.setAttribute("type", "type");
         entity.setAttribute("tableType", "MANAGED");
-        entity.setAttribute("database", new AtlasObjectId(DATABASE_TYPE, dbId));
+        entity.setAttribute("database", dbId);
         entity.setAttribute("created", new Date());
+
+        Map<String, Object> partAttributes = new HashMap<String, Object>() {{
+            put("name", "part0");
+        }};
+        final AtlasStruct partitionStruct  = new AtlasStruct("partition_struct_type", partAttributes);
+
+        entity.setAttribute("partitions", new ArrayList<AtlasStruct>() {{ add(partitionStruct); }});
+        entity.setAttribute("parametersMap", new java.util.HashMap<String, String>() {{
+            put("key1", "value1");
+        }});
         return entity;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index dc9c36f..da45215 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -9,6 +9,7 @@ ATLAS-1060 Add composite indexes for exact match performance improvements for al
 ATLAS-1127 Modify creation and modification timestamps to Date instead of Long(sumasai)
 
 ALL CHANGES:
+ATLAS-1257 Map Entity REST APIs to ATLAS v1 backend (sumasai)
 ATLAS-1279 Remove QueryPlan attribute from Hive Process entity (svimal2106)
 ATLAS-1234 Lineage REST API - v2 (sarath.kum4r@gmail.com via mneethiraj)
 ATLAS-1276 fix for webapp test failures (ayubkhan via mneethiraj)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/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 69e8d12..d2793d2 100755
--- a/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
+++ b/repository/src/main/java/org/apache/atlas/services/DefaultMetadataService.java
@@ -292,14 +292,29 @@ public class DefaultMetadataService implements MetadataService, ActiveStateChang
      * @return entity definition as JSON
      */
     @Override
-    public String getEntityDefinition(String guid) throws AtlasException {
+    public String getEntityDefinitionJson(String guid) throws AtlasException {
         guid = ParamChecker.notEmpty(guid, "entity id");
 
         final ITypedReferenceableInstance instance = repository.getEntityDefinition(guid);
         return InstanceSerialization.toJson(instance, true);
     }
 
-    private ITypedReferenceableInstance getEntityDefinitionReference(String entityType, String attribute, String value)
+    /**
+     * Return the definition for the given guid.
+     *
+     * @param guid guid
+     * @return entity definition as JSON
+     */
+    @Override
+    public ITypedReferenceableInstance getEntityDefinition(String guid) throws AtlasException {
+        guid = ParamChecker.notEmpty(guid, "entity id");
+
+        final ITypedReferenceableInstance instance = repository.getEntityDefinition(guid);
+        return instance;
+    }
+
+    @Override
+    public ITypedReferenceableInstance getEntityDefinitionReference(String entityType, String attribute, String value)
             throws AtlasException {
         validateTypeExists(entityType);
         validateUniqueAttribute(entityType, attribute);
@@ -356,6 +371,19 @@ public class DefaultMetadataService implements MetadataService, ActiveStateChang
         return entityResult;
     }
 
+    /**
+     * Updates an entity, instance of the type based on the guid set.
+     *
+     * @param entityInstanceDefinitions
+     * @return guids - json array of guids
+     */
+    @Override
+    public AtlasClient.EntityResult updateEntities(ITypedReferenceableInstance[] entityInstanceDefinitions) throws AtlasException {
+        AtlasClient.EntityResult entityResult = repository.updateEntities(entityInstanceDefinitions);
+        onEntitiesAddedUpdated(entityResult);
+        return entityResult;
+    }
+
     private void onEntitiesAddedUpdated(AtlasClient.EntityResult entityResult) throws AtlasException {
         onEntitiesAdded(entityResult.getCreatedEntities());
         onEntitiesUpdated(entityResult.getUpdateEntities());

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
index 37c2374..d659c0f 100644
--- a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
@@ -221,7 +221,7 @@ public class DefaultMetadataServiceTest {
 
         //Verify that get entity definition returns actual values with reserved characters
         Referenceable instance =
-                InstanceSerialization.fromJsonReferenceable(metadataService.getEntityDefinition(id), true);
+                InstanceSerialization.fromJsonReferenceable(metadataService.getEntityDefinitionJson(id), true);
         assertReferenceableEquals(instance, entity);
 
         //Verify that search with reserved characters works - for string attribute
@@ -693,7 +693,7 @@ public class DefaultMetadataServiceTest {
         serdeInstance.setNull("description");
         updateInstance(table);
         tableDefinitionJson =
-            metadataService.getEntityDefinition(tableId._getId());
+            metadataService.getEntityDefinitionJson(tableId._getId());
         tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
         Assert.assertNull(((Struct) tableDefinition.get("serde1")).get("description"));
     }
@@ -733,7 +733,7 @@ public class DefaultMetadataServiceTest {
         metadataService.updateEntityAttributeByGuid(tableId._getId(), "database", dbId);
 
         String tableDefinitionJson =
-            metadataService.getEntityDefinition(tableId._getId());
+            metadataService.getEntityDefinitionJson(tableId._getId());
         Referenceable tableDefinition = InstanceSerialization.fromJsonReferenceable(tableDefinitionJson, true);
 
         assertEquals(dbId, (((Id) tableDefinition.get("database"))._getId()));

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/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 2351891..e653184 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
@@ -101,13 +101,27 @@ public interface MetadataService {
      */
     List<String> createEntities(ITypedReferenceableInstance[] typedInstances) throws AtlasException;
 
+
     /**
      * Return the definition for the given guid.
      *
      * @param guid guid
      * @return entity definition as JSON
      */
-    String getEntityDefinition(String guid) throws AtlasException;
+    String getEntityDefinitionJson(String guid) throws AtlasException;
+
+    ITypedReferenceableInstance getEntityDefinition(String guid) throws AtlasException;
+
+
+    /**
+     * Return the definition given type and attribute. The attribute has to be unique attribute for the type
+     * @param entityType - type name
+     * @param attribute - attribute name
+     * @param value - attribute value
+     * @return
+     * @throws AtlasException
+     */
+    ITypedReferenceableInstance getEntityDefinitionReference(String entityType, String attribute, String value) throws AtlasException;
 
     /**
      * Return the definition given type and attribute. The attribute has to be unique attribute for the type
@@ -155,6 +169,15 @@ public interface MetadataService {
      */
     AtlasClient.EntityResult updateEntities(String entityJson) throws AtlasException;
 
+
+    /**
+     * Batch API - Adds/Updates the given entity id(guid).
+     *
+     * @param entityJson entity json
+     * @return json array of guids of entities created/updated
+     */
+    AtlasClient.EntityResult updateEntities(ITypedReferenceableInstance[] iTypedReferenceableInstances) throws AtlasException;
+
     // Trait management functions
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
new file mode 100644
index 0000000..5485efa
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasArrayFormatConverter.java
@@ -0,0 +1,78 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.type.AtlasArrayType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import static org.apache.atlas.web.adapters.AtlasFormatConverters.VERSION_V1;
+import static org.apache.atlas.web.adapters.AtlasFormatConverters.VERSION_V2;
+
+public class AtlasArrayFormatConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2);
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1);
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.ARRAY;
+    }
+
+    @Override
+    public Object convert(String sourceVersion, String targetVersion, AtlasType type, final Object source) throws AtlasBaseException {
+        Collection newCollection = null;
+        if ( source != null ) {
+            if (AtlasFormatConverters.isArrayListType(source.getClass())) {
+                newCollection = new ArrayList();
+            } else if (AtlasFormatConverters.isSetType(source.getClass())) {
+                newCollection = new LinkedHashSet();
+            }
+
+            AtlasArrayType arrType = (AtlasArrayType) type;
+            AtlasType elemType = arrType.getElementType();
+
+            Collection originalList = (Collection) source;
+            for (Object elem : originalList) {
+                AtlasFormatAdapter elemConverter = registry.getConverter(sourceVersion, targetVersion, elemType.getTypeCategory());
+                Object convertedVal = elemConverter.convert(sourceVersion, targetVersion, elemType, elem);
+
+                newCollection.add(convertedVal);
+            }
+        }
+        return newCollection;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
new file mode 100644
index 0000000..ec0661d
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasEnumFormatConverter.java
@@ -0,0 +1,40 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+
+import javax.inject.Inject;
+
+public class AtlasEnumFormatConverter extends AtlasPrimitiveFormatConverter {
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        super.init(registry);
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.ENUM;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatAdapter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatAdapter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatAdapter.java
new file mode 100644
index 0000000..c146d72
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatAdapter.java
@@ -0,0 +1,31 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.type.AtlasType;
+
+public interface AtlasFormatAdapter {
+
+    Object convert(String sourceVersion, String targetVersion, AtlasType type, Object source) throws AtlasBaseException;
+
+    TypeCategory getTypeCategory();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverters.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverters.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverters.java
new file mode 100644
index 0000000..5238a75
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConverters.java
@@ -0,0 +1,88 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Singleton
+public class AtlasFormatConverters {
+
+    public static String VERSION_V1 = "v1";
+    public static String VERSION_V2 = "v2";
+
+    private Map<String, AtlasFormatAdapter> registry = new HashMap<>();
+
+    public void registerConverter(AtlasFormatAdapter adapter, String sourceVersion, String targetVersion) {
+        registry.put(getKey(sourceVersion, targetVersion, adapter.getTypeCategory()), adapter);
+    }
+
+    public AtlasFormatAdapter getConverter(String sourceVersion, String targetVersion, TypeCategory typeCategory) throws AtlasBaseException {
+        if (registry.containsKey(getKey(sourceVersion, targetVersion, typeCategory))) {
+            return registry.get(getKey(sourceVersion, targetVersion, typeCategory));
+        }
+        throw new AtlasBaseException(AtlasErrorCode.INTERNAL_ERROR, "Could not find the converter for this type " + typeCategory);
+    }
+
+    public static boolean isArrayListType(Class c) {
+        return List.class.isAssignableFrom(c);
+    }
+
+    public static boolean isSetType(Class c) {
+        return Set.class.isAssignableFrom(c);
+    }
+
+    public static boolean isPrimitiveType(final Class c) {
+        if (c != null) {
+            if (Number.class.isAssignableFrom(c)) {
+                return true;
+            }
+
+            if (String.class.isAssignableFrom(c)) {
+                return true;
+            }
+
+            if (Date.class.isAssignableFrom(c)) {
+                return true;
+            }
+
+            return c.isPrimitive();
+        }
+        return false;
+    }
+
+    public static boolean isMapType(Object o) {
+        if ( o != null ) {
+            return Map.class.isAssignableFrom(o.getClass());
+        }
+        return false;
+    }
+
+    String getKey(String sourceVersion, String targetVersion, TypeCategory typeCategory) {
+        return sourceVersion + "-to-" + targetVersion + "-" + typeCategory.name();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConvertersModule.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConvertersModule.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConvertersModule.java
new file mode 100644
index 0000000..c3dfb61
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasFormatConvertersModule.java
@@ -0,0 +1,48 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.multibindings.Multibinder;
+import org.apache.atlas.web.adapters.v1.ReferenceableToAtlasEntityConverter;
+import org.apache.atlas.web.adapters.v1.StructToAtlasStructConverter;
+import org.apache.atlas.web.adapters.v1.TraitToAtlasClassificationConverter;
+import org.apache.atlas.web.adapters.v2.AtlasClassificationToTraitConverter;
+import org.apache.atlas.web.adapters.v2.AtlasEntityToReferenceableConverter;
+import org.apache.atlas.web.adapters.v2.AtlasStructToStructConverter;
+
+public class AtlasFormatConvertersModule extends AbstractModule {
+
+  protected void configure() {
+      Multibinder<AtlasFormatAdapter> multibinder
+          = Multibinder.newSetBinder(binder(), AtlasFormatAdapter.class);
+      multibinder.addBinding().to(AtlasStructToStructConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(AtlasEntityToReferenceableConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(AtlasClassificationToTraitConverter.class).asEagerSingleton();
+
+      multibinder.addBinding().to(AtlasPrimitiveFormatConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(AtlasEnumFormatConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(AtlasMapFormatConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(AtlasArrayFormatConverter.class).asEagerSingleton();
+
+      multibinder.addBinding().to(ReferenceableToAtlasEntityConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(StructToAtlasStructConverter.class).asEagerSingleton();
+      multibinder.addBinding().to(TraitToAtlasClassificationConverter.class).asEagerSingleton();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
new file mode 100644
index 0000000..dc42f10
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasInstanceRestAdapters.java
@@ -0,0 +1,156 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasEntityWithAssociations;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+import org.apache.atlas.model.instance.EntityMutations;
+import org.apache.atlas.services.MetadataService;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.atlas.typesystem.IStruct;
+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.exception.EntityNotFoundException;
+import org.apache.atlas.typesystem.exception.TypeNotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+@Singleton
+public class AtlasInstanceRestAdapters {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AtlasInstanceRestAdapters.class);
+
+    @Inject
+    private AtlasTypeRegistry typeRegistry;
+
+    @Inject
+    private AtlasFormatConverters instanceFormatters;
+
+    @Inject
+    private MetadataService metadataService;
+
+    public ITypedReferenceableInstance[] getITypedReferenceables(List<AtlasEntity> entities) throws AtlasBaseException {
+        ITypedReferenceableInstance[] entitiesInOldFormat = new ITypedReferenceableInstance[entities.size()];
+
+        for (int i = 0; i < entities.size(); i++) {
+            ITypedReferenceableInstance typedInstance = getITypedReferenceable(entities.get(i));
+            entitiesInOldFormat[i] = typedInstance;
+        }
+
+        return entitiesInOldFormat;
+    }
+
+    public ITypedReferenceableInstance getITypedReferenceable(AtlasEntity entity) throws AtlasBaseException {
+        AtlasFormatAdapter entityFormatter = instanceFormatters.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.ENTITY);
+        AtlasType entityType = typeRegistry.getType(entity.getTypeName());
+
+        Referenceable ref = (Referenceable) entityFormatter.convert(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, entityType, entity);
+        try {
+            return metadataService.getTypedReferenceableInstance(ref);
+        } catch (AtlasException e) {
+            LOG.error("Exception while getting a typed reference for the entity ", e);
+            throw toAtlasBaseException(e);
+        }
+    }
+
+    public Referenceable getReferenceable(AtlasEntity entity) throws AtlasBaseException {
+        AtlasFormatAdapter entityFormatter = instanceFormatters.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.ENTITY);
+        AtlasType entityType = typeRegistry.getType(entity.getTypeName());
+
+        return (Referenceable) entityFormatter.convert(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, entityType, entity);
+    }
+
+    public ITypedStruct getTrait(AtlasClassification classification) throws AtlasBaseException {
+        AtlasFormatAdapter formatter = instanceFormatters.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.CLASSIFICATION);
+        AtlasType clsType = typeRegistry.getType(classification.getTypeName());
+
+        Struct ref = (Struct) formatter.convert(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, clsType, classification);
+        try {
+            return metadataService.createTraitInstance(ref);
+        } catch (AtlasException e) {
+            LOG.error("Exception while getting a typed reference for the entity ", e);
+            throw toAtlasBaseException(e);
+        }
+    }
+
+    public AtlasClassification getClassification(IStruct classification) throws AtlasBaseException {
+        AtlasFormatAdapter formatter = instanceFormatters.getConverter(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, TypeCategory.CLASSIFICATION);
+        AtlasType clsType = typeRegistry.getType(classification.getTypeName());
+
+        return (AtlasClassification) formatter.convert(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, clsType, classification);
+    }
+
+    public AtlasEntityWithAssociations getAtlasEntity(IReferenceableInstance referenceable) throws AtlasBaseException {
+        AtlasFormatAdapter entityFormatter = instanceFormatters.getConverter(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, TypeCategory.ENTITY);
+        AtlasType entityType = typeRegistry.getType(referenceable.getTypeName());
+
+        return (AtlasEntityWithAssociations) entityFormatter.convert(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, entityType, referenceable);
+    }
+
+
+    public static EntityMutationResponse toEntityMutationResponse(AtlasClient.EntityResult result) {
+        EntityMutationResponse response = new EntityMutationResponse();
+        for (String guid : result.getCreatedEntities()) {
+            AtlasEntityHeader header = new AtlasEntityHeader();
+            header.setGuid(guid);
+            response.addEntity(EntityMutations.EntityOperation.CREATE_OR_UPDATE, header);
+        }
+
+        for (String guid : result.getUpdateEntities()) {
+            AtlasEntityHeader header = new AtlasEntityHeader();
+            header.setGuid(guid);
+            response.addEntity(EntityMutations.EntityOperation.CREATE_OR_UPDATE, header);
+        }
+
+        for (String guid : result.getDeletedEntities()) {
+            AtlasEntityHeader header = new AtlasEntityHeader();
+            header.setGuid(guid);
+            response.addEntity(EntityMutations.EntityOperation.DELETE, header);
+        }
+        return response;
+    }
+
+    public static AtlasBaseException toAtlasBaseException(AtlasException e) {
+        if ( e instanceof EntityNotFoundException) {
+            return new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND, e);
+        }
+
+        if ( e instanceof TypeNotFoundException) {
+            return new AtlasBaseException(AtlasErrorCode.TYPE_NAME_NOT_FOUND, e);
+        }
+
+        return new AtlasBaseException(e);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
new file mode 100644
index 0000000..761c8a7
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasMapFormatConverter.java
@@ -0,0 +1,74 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.type.AtlasMapType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+
+import javax.inject.Inject;
+import java.util.HashMap;
+import java.util.Map;
+
+public class AtlasMapFormatConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2);
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1);
+    }
+
+    @Override
+    public Map convert(String sourceVersion, String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+       Map newMap = new HashMap<>();
+
+        if ( source != null) {
+            Map origMap = (Map) source;
+            for (Object key : origMap.keySet()) {
+
+
+                AtlasMapType mapType = (AtlasMapType) type;
+                AtlasType keyType = mapType.getKeyType();
+                AtlasType valueType = mapType.getValueType();
+                AtlasFormatAdapter keyConverter = registry.getConverter(sourceVersion, targetVersion, keyType.getTypeCategory());
+                Object convertedKey = keyConverter.convert(sourceVersion, targetVersion, keyType, key);
+                Object val = origMap.get(key);
+
+                if (val != null) {
+                    AtlasFormatAdapter valueConverter = registry.getConverter(sourceVersion, targetVersion, valueType.getTypeCategory());
+                    newMap.put(convertedKey, valueConverter.convert(sourceVersion, targetVersion, valueType, val));
+                } else {
+                    newMap.put(convertedKey, val);
+                }
+            }
+        }
+        return newMap;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.MAP;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
new file mode 100644
index 0000000..66be58e
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/AtlasPrimitiveFormatConverter.java
@@ -0,0 +1,51 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+
+import javax.inject.Inject;
+import java.util.HashMap;
+import java.util.Map;
+
+public class AtlasPrimitiveFormatConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2);
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+       return type.getNormalizedValue(source);
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.PRIMITIVE;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v1/ReferenceableToAtlasEntityConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v1/ReferenceableToAtlasEntityConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/ReferenceableToAtlasEntityConverter.java
new file mode 100644
index 0000000..ba77c67
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/ReferenceableToAtlasEntityConverter.java
@@ -0,0 +1,111 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v1;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntityWithAssociations;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.atlas.typesystem.IStruct;
+import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ReferenceableToAtlasEntityConverter implements AtlasFormatAdapter {
+
+    protected AtlasTypeRegistry typeRegistry;
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public ReferenceableToAtlasEntityConverter(AtlasTypeRegistry typeRegistry) {
+        this.typeRegistry = typeRegistry;
+    }
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+        AtlasEntityWithAssociations result = null;
+        if ( source != null) {
+            if ( isId(source)) {
+                Id idObj = (Id) source;
+                result = new AtlasEntityWithAssociations(idObj.getTypeName());
+                setId(idObj, result);
+            } else if (isEntityType(source) ) {
+
+                IReferenceableInstance entity = (IReferenceableInstance) source;
+
+                //Resolve attributes
+                StructToAtlasStructConverter converter = (StructToAtlasStructConverter) registry.getConverter(sourceVersion, targetVersion, TypeCategory.STRUCT);
+                result =  new AtlasEntityWithAssociations(entity.getTypeName(), converter.convertAttributes((AtlasEntityType) type, entity));
+
+                //Id
+                setId(entity, result);
+
+                //Resolve traits
+                List<AtlasClassification> classifications = new ArrayList<>();
+                for (String traitName : entity.getTraits()) {
+                    IStruct trait = entity.getTrait(traitName);
+                    AtlasClassificationType traitType = (AtlasClassificationType) typeRegistry.getType(traitName);
+                    AtlasClassification clsInstance =  new AtlasClassification(traitType.getTypeName(), converter.convertAttributes(traitType, trait));
+                    classifications.add(clsInstance);
+                }
+                result.setClassifications(classifications);
+            }
+        }
+        return result;
+    }
+
+    private boolean isEntityType(Object o) {
+        if ( o != null && o instanceof IReferenceableInstance) {
+            return true;
+        }
+        return false;
+    }
+
+    private boolean isId(Object o) {
+        if ( o != null && o instanceof Id) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.ENTITY;
+    }
+
+
+    private void setId(IReferenceableInstance entity, AtlasEntity result) {
+        result.setGuid(entity.getId()._getId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v1/StructToAtlasStructConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v1/StructToAtlasStructConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/StructToAtlasStructConverter.java
new file mode 100644
index 0000000..15c41c8
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/StructToAtlasStructConverter.java
@@ -0,0 +1,114 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v1;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.IStruct;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+import org.apache.atlas.web.adapters.AtlasInstanceRestAdapters;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@Singleton
+public class StructToAtlasStructConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+
+        if (source != null) {
+            if (isStructType(source)) {
+                IStruct entity = (IStruct) source;
+                //Resolve attributes
+                StructToAtlasStructConverter converter = (StructToAtlasStructConverter) registry.getConverter(sourceVersion, targetVersion, TypeCategory.STRUCT);
+                return new AtlasStruct(type.getTypeName(), converter.convertAttributes((AtlasStructType) type, entity));
+            }
+
+        }
+        return null;
+    }
+
+    private boolean isStructType(Object o) {
+        if (o != null && o instanceof IStruct) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.STRUCT;
+    }
+
+    public Map<String, Object> convertAttributes(AtlasStructType structType, Object entity) throws AtlasBaseException {
+        Collection<AtlasStructDef.AtlasAttributeDef> attributeDefs;
+
+        if (structType.getTypeCategory() == TypeCategory.STRUCT) {
+            attributeDefs = structType.getStructDef().getAttributeDefs();
+        } else if (structType.getTypeCategory() == TypeCategory.CLASSIFICATION) {
+            attributeDefs = ((AtlasClassificationType)structType).getAllAttributeDefs().values();
+        } else if (structType.getTypeCategory() == TypeCategory.ENTITY) {
+            attributeDefs = ((AtlasEntityType)structType).getAllAttributeDefs().values();
+        } else {
+            attributeDefs = Collections.emptyList();
+        }
+
+        Map<String, Object> newAttrMap = new HashMap<>();
+        for (AtlasStructDef.AtlasAttributeDef attrDef : attributeDefs) {
+            AtlasType attrType = structType.getAttributeType(attrDef.getName());
+            AtlasFormatAdapter attrConverter = registry.getConverter(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, attrType.getTypeCategory());
+
+            Object attrVal = null;
+            if ( AtlasFormatConverters.isMapType(entity)) {
+                attrVal = ((Map)entity).get(attrDef.getName());
+            } else {
+                try {
+                    attrVal = ((IStruct)entity).get(attrDef.getName());
+                } catch (AtlasException e) {
+                    throw AtlasInstanceRestAdapters.toAtlasBaseException(e);
+                }
+            }
+            final Object convertedVal = attrConverter.convert(AtlasFormatConverters.VERSION_V1, AtlasFormatConverters.VERSION_V2, attrType, attrVal);
+            newAttrMap.put(attrDef.getName(), convertedVal);
+        }
+
+        return newAttrMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v1/TraitToAtlasClassificationConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v1/TraitToAtlasClassificationConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/TraitToAtlasClassificationConverter.java
new file mode 100644
index 0000000..6ac9013
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v1/TraitToAtlasClassificationConverter.java
@@ -0,0 +1,64 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v1;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.IStruct;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+import org.apache.atlas.web.adapters.AtlasInstanceRestAdapters;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@Singleton
+public class TraitToAtlasClassificationConverter extends StructToAtlasStructConverter {
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        super.init(registry);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+        AtlasStruct struct = (AtlasStruct) super.convert(sourceVersion, targetVersion, type, source);
+        if ( struct != null) {
+            return new AtlasClassification(struct.getTypeName(), struct.getAttributes());
+        }
+        return null;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.CLASSIFICATION;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasClassificationToTraitConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasClassificationToTraitConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasClassificationToTraitConverter.java
new file mode 100644
index 0000000..ece483c
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasClassificationToTraitConverter.java
@@ -0,0 +1,53 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v2;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.typesystem.Struct;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@Singleton
+public class AtlasClassificationToTraitConverter extends AtlasStructToStructConverter {
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        super.init(registry);
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.CLASSIFICATION;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasEntityToReferenceableConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasEntityToReferenceableConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasEntityToReferenceableConverter.java
new file mode 100644
index 0000000..fab051f
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasEntityToReferenceableConverter.java
@@ -0,0 +1,108 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v2;
+
+
+import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasObjectId;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import javax.inject.Inject;
+import java.util.Map;
+
+public class AtlasEntityToReferenceableConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+
+        if ( source != null) {
+            //JSOn unmarshalling gives us a Map instead of AtlasObjectId or AtlasEntity
+            if ( AtlasFormatConverters.isMapType(source)) {
+                //Could be an entity or an Id
+                Map srcMap = (Map) source;
+                String idStr = (String)srcMap.get(AtlasObjectId.KEY_GUID);
+                String typeName = type.getTypeName();
+
+                if (StringUtils.isEmpty(idStr)) {
+                    throw new AtlasBaseException(AtlasErrorCode.INSTANCE_GUID_NOT_FOUND);
+                }
+
+                if (MapUtils.isEmpty((Map)srcMap.get(AtlasStructToStructConverter.ATTRIBUTES_PROPERTY_KEY))) {
+                    //Convert to Id
+                    Id id = new Id(idStr, 0, typeName);
+                    return id;
+                } else {
+                    final Map attrMap = (Map) srcMap.get(AtlasStructToStructConverter.ATTRIBUTES_PROPERTY_KEY);
+                    //Resolve attributes
+                    AtlasStructToStructConverter converter = (AtlasStructToStructConverter) registry.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.STRUCT);
+                    return new Referenceable(idStr, typeName, converter.convertAttributes((AtlasEntityType)type, attrMap));
+
+                }
+            } else {
+                if ( isEntityType(source) ) {
+                    AtlasEntity entity = (AtlasEntity) source;
+                    String id = entity.getGuid();
+                    //Resolve attributes
+                    AtlasStructToStructConverter converter = (AtlasStructToStructConverter) registry.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.STRUCT);
+                    return new Referenceable(id, entity.getTypeName(), converter.convertAttributes((AtlasEntityType)type, entity));
+
+                } else if (isTransientId(source)) {
+                    return new Referenceable((String) source, type.getTypeName(), null);
+                }
+            }
+        }
+        return null;
+    }
+
+    private boolean isEntityType(Object o) {
+        if ( o != null && (o instanceof AtlasEntity)) {
+            return true;
+        }
+        return false;
+    }
+
+    private boolean isTransientId(Object o) {
+        if ( o != null && (o instanceof String)) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.ENTITY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasStructToStructConverter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasStructToStructConverter.java b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasStructToStructConverter.java
new file mode 100644
index 0000000..f0b4b12
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/adapters/v2/AtlasStructToStructConverter.java
@@ -0,0 +1,124 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.web.adapters.v2;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.TypeCategory;
+import org.apache.atlas.model.instance.AtlasStruct;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasType;
+import org.apache.atlas.typesystem.Struct;
+import org.apache.atlas.web.adapters.AtlasFormatAdapter;
+import org.apache.atlas.web.adapters.AtlasFormatConverters;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@Singleton
+public class AtlasStructToStructConverter implements AtlasFormatAdapter {
+
+    protected AtlasFormatConverters registry;
+
+    public static final String ATTRIBUTES_PROPERTY_KEY = "attributes";
+
+    @Inject
+    public void init(AtlasFormatConverters registry) throws AtlasBaseException {
+        this.registry = registry;
+        registry.registerConverter(this, AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1);
+    }
+
+    @Override
+    public Object convert(final String sourceVersion, final String targetVersion, final AtlasType type, final Object source) throws AtlasBaseException {
+
+        if (source != null) {
+            //Json unmarshalling gives us a Map instead of AtlasObjectId or AtlasEntity
+            if (AtlasFormatConverters.isMapType(source)) {
+                //Could be an entity or an Id
+                Map srcMap = (Map) source;
+                final Map attrMap = (Map) srcMap.get(ATTRIBUTES_PROPERTY_KEY);
+
+                if ( attrMap != null) {
+                    //Resolve attributes
+                    AtlasStructToStructConverter converter = (AtlasStructToStructConverter) registry.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.STRUCT);
+                    return new Struct(type.getTypeName(), converter.convertAttributes((AtlasStructType)type, attrMap));
+                }
+
+            } else if (isStructType(source)) {
+
+                AtlasStruct entity = (AtlasStruct) source;
+                //Resolve attributes
+                AtlasStructToStructConverter converter = (AtlasStructToStructConverter) registry.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, TypeCategory.STRUCT);
+                return new Struct(type.getTypeName(), converter.convertAttributes((AtlasStructType) type, entity));
+            }
+        }
+
+        return null;
+
+    }
+
+    private boolean isStructType(Object o) {
+        if (o != null && o instanceof AtlasStruct) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public TypeCategory getTypeCategory() {
+        return TypeCategory.STRUCT;
+    }
+
+    public Map<String, Object> convertAttributes(AtlasStructType structType, Object entity) throws AtlasBaseException {
+        Collection<AtlasStructDef.AtlasAttributeDef> attributeDefs;
+
+        if (structType.getTypeCategory() == TypeCategory.STRUCT) {
+            attributeDefs = structType.getStructDef().getAttributeDefs();
+        } else if (structType.getTypeCategory() == TypeCategory.CLASSIFICATION) {
+            attributeDefs = ((AtlasClassificationType)structType).getAllAttributeDefs().values();
+        } else if (structType.getTypeCategory() == TypeCategory.ENTITY) {
+            attributeDefs = ((AtlasEntityType)structType).getAllAttributeDefs().values();
+        } else {
+            attributeDefs = Collections.emptyList();
+        }
+
+        Map<String, Object> newAttrMap = new HashMap<>();
+        for (AtlasStructDef.AtlasAttributeDef attrDef : attributeDefs) {
+            AtlasType attrType = structType.getAttributeType(attrDef.getName());
+
+            AtlasFormatAdapter attrConverter = registry.getConverter(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, attrType.getTypeCategory());
+
+            Object attrVal = null;
+            if ( AtlasFormatConverters.isMapType(entity)) {
+                attrVal = ((Map)entity).get(attrDef.getName());
+            } else {
+                attrVal = ((AtlasStruct)entity).getAttribute(attrDef.getName());
+            }
+            final Object convertedVal = attrConverter.convert(AtlasFormatConverters.VERSION_V2, AtlasFormatConverters.VERSION_V1, attrType, attrVal);
+            newAttrMap.put(attrDef.getName(), convertedVal);
+        }
+
+        return newAttrMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java b/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
index 0c2e0ec..c245596 100755
--- a/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
+++ b/webapp/src/main/java/org/apache/atlas/web/listeners/GuiceServletConfig.java
@@ -33,11 +33,11 @@ import org.apache.atlas.notification.NotificationModule;
 import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.service.Services;
+import org.apache.atlas.web.adapters.AtlasFormatConvertersModule;
 import org.apache.atlas.web.filters.ActiveServerFilter;
 import org.apache.atlas.web.filters.AuditFilter;
 import org.apache.atlas.web.service.ActiveInstanceElectorModule;
 import org.apache.atlas.web.service.ServiceModule;
-import org.apache.commons.collections.iterators.EnumerationIterator;
 import org.apache.commons.configuration.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -74,7 +74,7 @@ public class GuiceServletConfig extends GuiceServletContextListener {
             LoginProcessor loginProcessor = new LoginProcessor();
             loginProcessor.login();
 
-            injector = Guice.createInjector(Stage.PRODUCTION, getRepositoryModule(), new ActiveInstanceElectorModule(),
+            injector = Guice.createInjector(Stage.PRODUCTION,  getRepositoryModule(), new AtlasFormatConvertersModule(), new ActiveInstanceElectorModule(),
                     new NotificationModule(), new ServiceModule(), new JerseyServletModule() {
 
                         private Configuration appConfiguration = null;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/33d60746/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
index bbf01a6..230265b 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/EntityResource.java
@@ -179,7 +179,7 @@ public class EntityResource {
         response.put(AtlasClient.ENTITIES, new JSONObject(entityResult.toString()).get(AtlasClient.ENTITIES));
         String sampleEntityId = getSample(entityResult);
         if (sampleEntityId != null) {
-            String entityDefinition = metadataService.getEntityDefinition(sampleEntityId);
+            String entityDefinition = metadataService.getEntityDefinitionJson(sampleEntityId);
             response.put(AtlasClient.DEFINITION, new JSONObject(entityDefinition));
         }
         return response;
@@ -470,7 +470,7 @@ public class EntityResource {
 
             LOG.debug("Fetching entity definition for guid={} ", guid);
             guid = ParamChecker.notEmpty(guid, "guid cannot be null");
-            final String entityDefinition = metadataService.getEntityDefinition(guid);
+            final String entityDefinition = metadataService.getEntityDefinitionJson(guid);
 
             JSONObject response = new JSONObject();
             response.put(AtlasClient.REQUEST_ID, Servlets.getRequestId());
@@ -653,7 +653,7 @@ public class EntityResource {
                 perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "EntityResource.getTraitDefinitionsForEntity(" + guid + ")");
             }
             LOG.debug("Fetching all trait definitions for entity={}", guid);
-            final String entityDefinition = metadataService.getEntityDefinition(guid);
+            final String entityDefinition = metadataService.getEntityDefinitionJson(guid);
 
             Referenceable entity = InstanceSerialization.fromJsonReferenceable(entityDefinition, true);
             JSONArray traits = new JSONArray();