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/10/10 22:37:54 UTC

[6/8] incubator-atlas git commit: ATLAS-694 Update Atlas code to use graph abstraction layer (jnhagelb via sumasai)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
old mode 100755
new mode 100644
index c3252c5..06e2b7c
--- a/pom.xml
+++ b/pom.xml
@@ -455,6 +455,7 @@
         <titan.storage.backend>berkeleyje</titan.storage.backend>
         <titan.index.backend>elasticsearch</titan.index.backend>
         <entity.repository.impl>org.apache.atlas.repository.audit.InMemoryEntityAuditRepository</entity.repository.impl>
+	<graphdb.backend.impl>org.apache.atlas.repository.graphdb.titan0.Titan0GraphDatabase</graphdb.backend.impl>
         <atlas.surefire.options></atlas.surefire.options>
     </properties>
 
@@ -900,6 +901,57 @@
             </dependency>
 
             <!-- Graph DB -->
+            <dependency>
+                <groupId>com.tinkerpop.blueprints</groupId>
+                <artifactId>blueprints-core</artifactId>
+                <version>${tinkerpop.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>com.thinkaurelius.titan</groupId>
+                <artifactId>titan-core</artifactId>
+                <version>${titan.version}</version>
+                <exclusions>
+                    <!-- rexster does not work with servlet-api -->
+                    <exclusion>
+                        <groupId>com.tinkerpop.rexster</groupId>
+                        <artifactId>rexster-core</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>com.tinkerpop.rexster</groupId>
+                        <artifactId>rexster-server</artifactId>
+                    </exclusion>
+                    <!-- asm 4.0 does not work with jersey asm 3.1 -->
+                    <exclusion>
+                        <groupId>com.tinkerpop</groupId>
+                        <artifactId>frames</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>com.esotericsoftware.reflectasm</groupId>
+                        <artifactId>reflectasm</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>org.ow2.asm</groupId>
+                        <artifactId>asm</artifactId>
+                    </exclusion>
+                    <exclusion> <!-- GPL license imported from ganglia -->
+                        <groupId>org.acplt</groupId>
+                        <artifactId>oncrpc</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+
+            <dependency>
+                <groupId>com.thinkaurelius.titan</groupId>
+                <artifactId>titan-berkeleyje</artifactId>
+                <version>${titan.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>com.thinkaurelius.titan</groupId>
+                <artifactId>titan-hbase</artifactId>
+                <version>${titan.version}</version>
+            </dependency>
             
             <dependency>
                 <groupId>org.apache.hbase</groupId>
@@ -954,6 +1006,7 @@
                 </exclusions>
             </dependency>
 
+
             <dependency>
                 <groupId>com.vividsolutions</groupId>
                 <artifactId>jts</artifactId>
@@ -1027,12 +1080,6 @@
 
             <dependency>
                 <groupId>org.apache.atlas</groupId>
-                <artifactId>atlas-graphdb-titan0</artifactId>
-                <version>${project.version}</version>
-            </dependency>
-
-            <dependency>
-                <groupId>org.apache.atlas</groupId>
                 <artifactId>atlas-server-api</artifactId>
                 <version>${project.version}</version>
             </dependency>
@@ -1083,6 +1130,17 @@
                 <type>war</type>
             </dependency>
 
+	    <!-- use titan 0.5.4 by default -->
+            <dependency>
+                <groupId>org.apache.atlas</groupId>
+                <artifactId>atlas-graphdb-impls</artifactId>
+                <version>${project.version}</version>
+                <type>pom</type>
+		  <!-- exclusions should be added here for all of the non-titan0 
+                 implementations -->
+                <scope>test</scope>
+	    </dependency>
+
             <dependency>
                 <groupId>org.apache.atlas</groupId>
                 <artifactId>hive-bridge</artifactId>
@@ -1653,9 +1711,12 @@
                     <excludeSubProjects>true</excludeSubProjects>
                     <excludes>
                         <exclude>**/dependency-reduced-pom.xml</exclude>
+			<exclude>**/javax.script.ScriptEngineFactory</exclude>
                         <exclude>.reviewboardrc</exclude>
                         <exclude>3party-licenses/**</exclude>
+                        <exclude>**/.cache</exclude>
                         <exclude>**/.cache-main</exclude>
+                        <exclude>**/.cache-tests</exclude>
                         <exclude>**/.checkstyle</exclude>
                         <exclude>*.txt</exclude>
                         <exclude>**/*.json</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index d49a9ca..494d147 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-694  Update Atlas code to use graph abstraction layer (jnhagelb via sumasai)
 ATLAS-1215 Atlas UI not working in firefox due to fix in ATLAS-1199 (kevalbhatt)
 ATLAS-1171 Structured, high-level public APIs - Fix JAXB issues with PList, SearchFilter (mneethiraj via sumasai)
 ATLAS-1206 Atlas UI not working with IE or Chrome on Windows OS in Kerberos mode (nixonrodrigues via sumasai)
@@ -50,7 +51,7 @@ ATLAS-1104 Get outgoing edges by label doesn't work in some cases (shwethags)
 ATLAS-1106 Fix Build failure due to wrong version in graphdb/common pom (sumasai)
 ATLAS-1105 Disable HiveLiteralRewriterTest since its not used currently (sumasai)
 ATLAS-1103 : UI: Search type list is not refreshed (Kalyanikashikar via sumasai)
-ATLAS-693 Titan 0.5.4 implementation of the graph db abstraction {jnhagelb via dkantor)
+ATLAS-693 Titan 0.5.4 implementation of the graph db abstraction (jnhagelb via dkantor)
 ATLAS-1099 UI : multiple tag assign button hides wrongly (Kalyanikashikar via sumasai)
 ATLAS-1087 Provide an option to turn off persisting entity definition in audits (sumasai, shwethags)
 ATLAS-1097 Fix a potential NPE issue flagged by Coverity scan (mneethiraj via shwethags)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/pom.xml
----------------------------------------------------------------------
diff --git a/repository/pom.xml b/repository/pom.xml
index 7a34331..cdc8429 100755
--- a/repository/pom.xml
+++ b/repository/pom.xml
@@ -52,11 +52,15 @@
             <artifactId>atlas-graphdb-api</artifactId>
         </dependency>
 
-        <dependency>
+	
+	<dependency>
             <groupId>org.apache.atlas</groupId>
-            <artifactId>atlas-graphdb-titan0</artifactId>
+            <artifactId>atlas-graphdb-impls</artifactId>
+	    <type>pom</type>
+            <scope>test</scope>
         </dependency>
 
+
         <dependency>
             <groupId>joda-time</groupId>
             <artifactId>joda-time</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/GraphTransactionInterceptor.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/GraphTransactionInterceptor.java b/repository/src/main/java/org/apache/atlas/GraphTransactionInterceptor.java
index fff8925..1f8affe 100644
--- a/repository/src/main/java/org/apache/atlas/GraphTransactionInterceptor.java
+++ b/repository/src/main/java/org/apache/atlas/GraphTransactionInterceptor.java
@@ -17,11 +17,10 @@
 
 package org.apache.atlas;
 
-import com.google.inject.Inject;
-import com.thinkaurelius.titan.core.TitanGraph;
 import org.aopalliance.intercept.MethodInterceptor;
 import org.aopalliance.intercept.MethodInvocation;
-import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.exception.SchemaNotFoundException;
 import org.slf4j.Logger;
@@ -29,30 +28,27 @@ import org.slf4j.LoggerFactory;
 
 public class GraphTransactionInterceptor implements MethodInterceptor {
     private static final Logger LOG = LoggerFactory.getLogger(GraphTransactionInterceptor.class);
-    private TitanGraph titanGraph;
-
-    @Inject
-    GraphProvider<TitanGraph> graphProvider;
+    private AtlasGraph graph;
 
     @Override
     public Object invoke(MethodInvocation invocation) throws Throwable {
-        if (titanGraph == null) {
-            titanGraph = graphProvider.get();
+        
+        if (graph == null) {
+            graph = AtlasGraphProvider.getGraphInstance();
         }
 
         try {
             Object response = invocation.proceed();
-            titanGraph.commit();
+            graph.commit();
             LOG.info("graph commit");
             return response;
         } catch (Throwable t) {
-            titanGraph.rollback();
-
             if (logException(t)) {
                 LOG.error("graph rollback due to exception ", t);
             } else {
                 LOG.error("graph rollback due to exception " + t.getClass().getSimpleName() + ":" + t.getMessage());
             }
+            graph.rollback();
             throw t;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java b/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
index f1ef140..d0792d7 100755
--- a/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
+++ b/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
@@ -18,16 +18,9 @@
 
 package org.apache.atlas;
 
-import com.google.inject.Binder;
-import com.google.inject.Singleton;
-import com.google.inject.matcher.Matchers;
-import com.google.inject.multibindings.Multibinder;
-import com.google.inject.throwingproviders.ThrowingProviderBinder;
-import com.thinkaurelius.titan.core.TitanGraph;
-
 import org.aopalliance.intercept.MethodInterceptor;
-import org.apache.atlas.discovery.DiscoveryService;
 import org.apache.atlas.discovery.DataSetLineageService;
+import org.apache.atlas.discovery.DiscoveryService;
 import org.apache.atlas.discovery.LineageService;
 import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
 import org.apache.atlas.listener.EntityChangeListener;
@@ -35,13 +28,9 @@ import org.apache.atlas.listener.TypesChangeListener;
 import org.apache.atlas.repository.MetadataRepository;
 import org.apache.atlas.repository.audit.EntityAuditListener;
 import org.apache.atlas.repository.audit.EntityAuditRepository;
-import org.apache.atlas.repository.audit.HBaseBasedAuditRepository;
 import org.apache.atlas.repository.graph.DeleteHandler;
 import org.apache.atlas.repository.graph.GraphBackedMetadataRepository;
 import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
-import org.apache.atlas.repository.graph.GraphProvider;
-import org.apache.atlas.repository.graph.SoftDeleteHandler;
-import org.apache.atlas.repository.graph.TitanGraphProvider;
 import org.apache.atlas.repository.typestore.GraphBackedTypeStore;
 import org.apache.atlas.repository.typestore.ITypeStore;
 import org.apache.atlas.service.Service;
@@ -51,10 +40,15 @@ import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.services.ReservedTypesRegistrar;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.TypeSystemProvider;
-import org.apache.atlas.typesystem.types.cache.DefaultTypeCache;
 import org.apache.atlas.typesystem.types.cache.TypeCache;
+import org.apache.atlas.util.AtlasRepositoryConfiguration;
 import org.apache.commons.configuration.Configuration;
 
+import com.google.inject.Binder;
+import com.google.inject.Singleton;
+import com.google.inject.matcher.Matchers;
+import com.google.inject.multibindings.Multibinder;
+
 /**
  * Guice module for Repository module.
  */
@@ -62,9 +56,6 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
 
     @Override
     protected void configure() {
-        // special wiring for Titan Graph
-        ThrowingProviderBinder.create(binder()).bind(GraphProvider.class, TitanGraph.class).to(TitanGraphProvider.class)
-                .asEagerSingleton();
 
         // allow for dynamic binding of the metadata repo & graph service
         // bind the MetadataRepositoryService interface to an implementation
@@ -95,9 +86,9 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
         Configuration configuration = getConfiguration();
         bindAuditRepository(binder(), configuration);
 
-        bind(DeleteHandler.class).to(getDeleteHandlerImpl(configuration)).asEagerSingleton();
+        bind(DeleteHandler.class).to((Class<? extends DeleteHandler>) AtlasRepositoryConfiguration.getDeleteHandlerImpl()).asEagerSingleton();
 
-        bind(TypeCache.class).to(getTypeCache(configuration)).asEagerSingleton();
+        bind(TypeCache.class).to((Class<? extends TypeCache>) AtlasRepositoryConfiguration.getTypeCache()).asEagerSingleton();
 
         //Add EntityAuditListener as EntityChangeListener
         Multibinder<EntityChangeListener> entityChangeListenerBinder =
@@ -119,7 +110,7 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
 
     protected void bindAuditRepository(Binder binder, Configuration configuration) {
 
-        Class<? extends EntityAuditRepository> auditRepoImpl = getAuditRepositoryImpl(getConfiguration());
+        Class<? extends EntityAuditRepository> auditRepoImpl = AtlasRepositoryConfiguration.getAuditRepositoryImpl();
 
         //Map EntityAuditRepository interface to configured implementation
         binder.bind(EntityAuditRepository.class).to(auditRepoImpl).asEagerSingleton();
@@ -132,40 +123,4 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
         }
     }
 
-
-    private static final String AUDIT_REPOSITORY_IMPLEMENTATION_PROPERTY = "atlas.EntityAuditRepository.impl";
-
-    private Class<? extends EntityAuditRepository> getAuditRepositoryImpl(Configuration configuration) {
-        try {
-            return ApplicationProperties.getClass(configuration,
-                    AUDIT_REPOSITORY_IMPLEMENTATION_PROPERTY, HBaseBasedAuditRepository.class.getName(), EntityAuditRepository.class);
-        } catch (AtlasException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static final String DELETE_HANDLER_IMPLEMENTATION_PROPERTY = "atlas.DeleteHandler.impl";
-
-    private Class<? extends DeleteHandler> getDeleteHandlerImpl(Configuration configuration) {
-        try {
-            return ApplicationProperties.getClass(configuration,
-                    DELETE_HANDLER_IMPLEMENTATION_PROPERTY, SoftDeleteHandler.class.getName(), DeleteHandler.class);
-        } catch (AtlasException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static final String TYPE_CACHE_IMPLEMENTATION_PROPERTY = "atlas.TypeCache.impl";
-
-    protected Class<? extends TypeCache> getTypeCache(Configuration configuration) {
-
-        // Get the type cache implementation class from Atlas configuration.
-        try {
-            return ApplicationProperties.getClass(configuration, TYPE_CACHE_IMPLEMENTATION_PROPERTY,
-                DefaultTypeCache.class.getName(), TypeCache.class);
-        } catch (AtlasException e) {
-            throw new RuntimeException("Error getting TypeCache implementation class", e);
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/discovery/DataSetLineageService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/DataSetLineageService.java b/repository/src/main/java/org/apache/atlas/discovery/DataSetLineageService.java
index c216469..4359264 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/DataSetLineageService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/DataSetLineageService.java
@@ -18,7 +18,9 @@
 
 package org.apache.atlas.discovery;
 
-import com.thinkaurelius.titan.core.TitanGraph;
+import javax.inject.Inject;
+import javax.inject.Singleton;
+
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
@@ -31,7 +33,8 @@ import org.apache.atlas.query.InputLineageClosureQuery;
 import org.apache.atlas.query.OutputLineageClosureQuery;
 import org.apache.atlas.query.QueryParams;
 import org.apache.atlas.repository.MetadataRepository;
-import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.exception.SchemaNotFoundException;
 import org.apache.atlas.typesystem.persistence.ReferenceableInstance;
@@ -39,13 +42,11 @@ import org.apache.atlas.utils.ParamChecker;
 import org.apache.commons.configuration.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import scala.Option;
 import scala.Some;
 import scala.collection.immutable.List;
 
-import javax.inject.Inject;
-import javax.inject.Singleton;
-
 /**
  * Hive implementation of Lineage service interface.
  */
@@ -80,14 +81,14 @@ public class DataSetLineageService implements LineageService {
     }
 
 
-    private final TitanGraph titanGraph;
+    private final AtlasGraph graph;
     private final DefaultGraphPersistenceStrategy graphPersistenceStrategy;
     private final GraphBackedDiscoveryService discoveryService;
 
     @Inject
-    DataSetLineageService(GraphProvider<TitanGraph> graphProvider, MetadataRepository metadataRepository,
+    DataSetLineageService(MetadataRepository metadataRepository,
                           GraphBackedDiscoveryService discoveryService) throws DiscoveryException {
-        this.titanGraph = graphProvider.get();
+        this.graph = AtlasGraphProvider.getGraphInstance();
         this.graphPersistenceStrategy = new DefaultGraphPersistenceStrategy(metadataRepository);
         this.discoveryService = discoveryService;
     }
@@ -136,7 +137,7 @@ public class DataSetLineageService implements LineageService {
                 inputsQuery = new InputLineageClosureQuery(AtlasClient.DATA_SET_SUPER_TYPE, SELECT_INSTANCE_GUID,
                 guid, HIVE_PROCESS_TYPE_NAME,
                 HIVE_PROCESS_INPUT_ATTRIBUTE_NAME, HIVE_PROCESS_OUTPUT_ATTRIBUTE_NAME, Option.empty(),
-                SELECT_ATTRIBUTES, true, graphPersistenceStrategy, titanGraph);
+                SELECT_ATTRIBUTES, true, graphPersistenceStrategy, graph);
         return inputsQuery.graph().toInstanceJson();
     }
 
@@ -153,7 +154,7 @@ public class DataSetLineageService implements LineageService {
         OutputLineageClosureQuery outputsQuery =
                 new OutputLineageClosureQuery(AtlasClient.DATA_SET_SUPER_TYPE, SELECT_INSTANCE_GUID, guid, HIVE_PROCESS_TYPE_NAME,
                         HIVE_PROCESS_INPUT_ATTRIBUTE_NAME, HIVE_PROCESS_OUTPUT_ATTRIBUTE_NAME, Option.empty(),
-                        SELECT_ATTRIBUTES, true, graphPersistenceStrategy, titanGraph);
+                        SELECT_ATTRIBUTES, true, graphPersistenceStrategy, graph);
         return outputsQuery.graph().toInstanceJson();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java b/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
index b17eec7..1133b48 100755
--- a/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/graph/DefaultGraphPersistenceStrategy.java
@@ -18,9 +18,10 @@
 
 package org.apache.atlas.discovery.graph;
 
-import com.google.common.collect.ImmutableCollection;
-import com.google.common.collect.ImmutableList;
-import com.thinkaurelius.titan.core.TitanVertex;
+import java.util.List;
+
+import javax.inject.Inject;
+
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.query.Expressions;
 import org.apache.atlas.query.GraphPersistenceStrategies;
@@ -29,8 +30,12 @@ import org.apache.atlas.query.IntSequence;
 import org.apache.atlas.query.TypeUtils;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.MetadataRepository;
+import org.apache.atlas.repository.RepositoryException;
 import org.apache.atlas.repository.graph.GraphBackedMetadataRepository;
 import org.apache.atlas.repository.graph.GraphHelper;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.repository.graphdb.GremlinVersion;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.persistence.Id;
@@ -44,8 +49,8 @@ import org.apache.atlas.typesystem.types.TypeSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.inject.Inject;
-import java.util.List;
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.ImmutableList;
 
 /**
  * Default implementation of GraphPersistenceStrategy.
@@ -95,8 +100,8 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
     }
 
     @Override
-    public List<String> traitNames(TitanVertex vertex) {
-        return GraphHelper.getTraitNames(vertex);
+    public List<String> traitNames(AtlasVertex AtlasVertex) {
+        return GraphHelper.getTraitNames(AtlasVertex);
     }
 
     @Override
@@ -105,7 +110,7 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
     }
 
     @Override
-    public Id getIdFromVertex(String dataTypeName, TitanVertex vertex) {
+    public Id getIdFromVertex(String dataTypeName, AtlasVertex vertex) {
         return GraphHelper.getIdFromVertex(dataTypeName, vertex);
     }
 
@@ -133,16 +138,16 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
                 break;
 
             case STRUCT:
-                TitanVertex structVertex = (TitanVertex) value;
+                AtlasVertex structVertex = (AtlasVertex) value;
                 StructType structType = (StructType) dataType;
                 ITypedStruct structInstance = structType.createInstance();
 
                 TypeSystem.IdType idType = TypeSystem.getInstance().getIdType();
 
                 if (dataType.getName().equals(idType.getName())) {
-                    structInstance.set(idType.typeNameAttrName(), GraphHelper.getProperty(structVertex, typeAttributeName()));
-                    structInstance.set(idType.idAttrName(), GraphHelper.getProperty(structVertex, idAttributeName()));
-                    String stateValue = GraphHelper.getProperty(structVertex, stateAttributeName());
+                    structInstance.set(idType.typeNameAttrName(), GraphHelper.getSingleValuedProperty(structVertex, typeAttributeName(), String.class));
+                    structInstance.set(idType.idAttrName(), GraphHelper.getSingleValuedProperty(structVertex, idAttributeName(), String.class));
+                    String stateValue = GraphHelper.getSingleValuedProperty(structVertex, stateAttributeName(), String.class);
                     if (stateValue != null) {
                         structInstance.set(idType.stateAttrName(), stateValue);
                     }
@@ -153,7 +158,7 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
                 return dataType.convert(structInstance, Multiplicity.OPTIONAL);
 
             case TRAIT:
-                TitanVertex traitVertex = (TitanVertex) value;
+                AtlasVertex traitVertex = (AtlasVertex) value;
                 TraitType traitType = (TraitType) dataType;
                 ITypedStruct traitInstance = traitType.createInstance();
                 // todo - this is not right, we should load the Instance associated with this
@@ -165,9 +170,9 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
                 break;
 
             case CLASS:
-                TitanVertex classVertex = (TitanVertex) value;
+                AtlasVertex classVertex = (AtlasVertex) value;
                 ITypedReferenceableInstance classInstance = metadataRepository.getGraphToInstanceMapper()
-                    .mapGraphToTypedInstance(classVertex.<String>getProperty(Constants.GUID_PROPERTY_KEY),
+                    .mapGraphToTypedInstance(GraphHelper.getSingleValuedProperty(classVertex, Constants.GUID_PROPERTY_KEY, String.class),
                         classVertex);
                 return dataType.convert(classInstance, Multiplicity.OPTIONAL);
 
@@ -210,6 +215,11 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
     public String gremlinCompOp(Expressions.ComparisonExpression op) {
         return GraphPersistenceStrategies$class.gremlinCompOp(this, op);
     }
+    
+    @Override
+    public String gremlinPrimitiveOp(Expressions.ComparisonExpression op) {
+        return GraphPersistenceStrategies$class.gremlinPrimitiveOp(this, op);
+    }
 
     @Override
     public String loopObjectExpression(IDataType<?> dataType) {
@@ -250,5 +260,30 @@ public class DefaultGraphPersistenceStrategy implements GraphPersistenceStrategi
     public boolean addGraphVertexPrefix(scala.collection.Traversable<String> preStatements) {
         return GraphPersistenceStrategies$class.addGraphVertexPrefix(this, preStatements);
     }
+    
+    @Override
+    public GremlinVersion getSupportedGremlinVersion() {
+        return GraphPersistenceStrategies$class.getSupportedGremlinVersion(this);
+    }
+
+    @Override
+    public String generatePersisentToLogicalConversionExpression(String expr, IDataType<?> t) {
+        return GraphPersistenceStrategies$class.generatePersisentToLogicalConversionExpression(this,expr, t);
+    }
+
+    @Override
+    public String initialQueryCondition() {
+        return GraphPersistenceStrategies$class.initialQueryCondition(this);
+    }
+
+    @Override
+    public boolean isPropertyValueConversionNeeded(IDataType<?> t) {
+        return GraphPersistenceStrategies$class.isPropertyValueConversionNeeded(this, t);
+    }
+
+    @Override
+    public AtlasGraph getGraph() throws RepositoryException {
+        return metadataRepository.getGraph();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java b/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
index 0c029bb..c0cc25c 100755
--- a/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/graph/GraphBackedDiscoveryService.java
@@ -18,13 +18,17 @@
 
 package org.apache.atlas.discovery.graph;
 
-import com.thinkaurelius.titan.core.TitanEdge;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.TitanIndexQuery;
-import com.thinkaurelius.titan.core.TitanProperty;
-import com.thinkaurelius.titan.core.TitanVertex;
-import com.tinkerpop.blueprints.Direction;
-import com.tinkerpop.blueprints.Vertex;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.script.ScriptException;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.discovery.DiscoveryException;
@@ -39,29 +43,21 @@ import org.apache.atlas.query.QueryParser;
 import org.apache.atlas.query.QueryProcessor;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.MetadataRepository;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.repository.graph.GraphHelper;
-import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import scala.util.Either;
 import scala.util.parsing.combinator.Parsers;
 
-import javax.inject.Inject;
-import javax.inject.Singleton;
-import javax.script.Bindings;
-import javax.script.ScriptEngine;
-import javax.script.ScriptEngineManager;
-import javax.script.ScriptException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 /**
  * Graph backed implementation of Search.
  */
@@ -70,19 +66,19 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
 
     private static final Logger LOG = LoggerFactory.getLogger(GraphBackedDiscoveryService.class);
 
-    private final TitanGraph titanGraph;
+    private final AtlasGraph graph;
     private final DefaultGraphPersistenceStrategy graphPersistenceStrategy;
 
     public final static String SCORE = "score";
 
     @Inject
-    GraphBackedDiscoveryService(GraphProvider<TitanGraph> graphProvider, MetadataRepository metadataRepository)
+    GraphBackedDiscoveryService(MetadataRepository metadataRepository)
     throws DiscoveryException {
-        this.titanGraph = graphProvider.get();
+        this.graph = AtlasGraphProvider.getGraphInstance();
         this.graphPersistenceStrategy = new DefaultGraphPersistenceStrategy(metadataRepository);
     }
 
-    //Refer http://s3.thinkaurelius.com/docs/titan/0.5.4/index-backends.html for indexed query
+    //For titan 0.5.4, refer to http://s3.thinkaurelius.com/docs/titan/0.5.4/index-backends.html for indexed query
     //http://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl-query-string-query
     // .html#query-string-syntax for query syntax
     @Override
@@ -90,8 +86,7 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
     public String searchByFullText(String query, QueryParams queryParams) throws DiscoveryException {
         String graphQuery = String.format("v.\"%s\":(%s)", Constants.ENTITY_TEXT_PROPERTY_KEY, query);
         LOG.debug("Full text query: {}", graphQuery);
-        Iterator<TitanIndexQuery.Result<Vertex>> results =
-                titanGraph.indexQuery(Constants.FULLTEXT_INDEX, graphQuery).vertices().iterator();
+        Iterator<AtlasIndexQuery.Result<?, ?>> results =graph.indexQuery(Constants.FULLTEXT_INDEX, graphQuery).vertices();
         JSONArray response = new JSONArray();
 
         int index = 0;
@@ -101,8 +96,9 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
         }
 
         while (results.hasNext() && response.length() < queryParams.limit()) {
-            TitanIndexQuery.Result<Vertex> result = results.next();
-            Vertex vertex = result.getElement();
+            
+            AtlasIndexQuery.Result<?,?> result = results.next();
+            AtlasVertex<?,?> vertex = result.getVertex();
 
             JSONObject row = new JSONObject();
             String guid = GraphHelper.getIdFromVertex(vertex);
@@ -157,7 +153,7 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
         LOG.debug("Query = {}", validatedExpression);
         LOG.debug("Expression Tree = {}", validatedExpression.treeString());
         LOG.debug("Gremlin Query = {}", gremlinQuery.queryStr());
-        return new GremlinEvaluator(gremlinQuery, graphPersistenceStrategy, titanGraph).evaluate();
+        return new GremlinEvaluator(gremlinQuery, graphPersistenceStrategy, graph).evaluate();
     }
 
     /**
@@ -173,72 +169,59 @@ public class GraphBackedDiscoveryService implements DiscoveryService {
     @GraphTransaction
     public List<Map<String, String>> searchByGremlin(String gremlinQuery) throws DiscoveryException {
         LOG.debug("Executing gremlin query={}", gremlinQuery);
-        ScriptEngineManager manager = new ScriptEngineManager();
-        ScriptEngine engine = manager.getEngineByName("gremlin-groovy");
-
-        if(engine == null) {
-            throw new DiscoveryException("gremlin-groovy: engine not found");
-        }
-
-        Bindings bindings = engine.createBindings();
-        bindings.put("g", titanGraph);
-
         try {
-            Object o = engine.eval(gremlinQuery, bindings);
+            Object o = graph.executeGremlinScript(gremlinQuery, false);
             return extractResult(o);
         } catch (ScriptException se) {
             throw new DiscoveryException(se);
         }
     }
-
+    
     private List<Map<String, String>> extractResult(final Object o) throws DiscoveryException {
         List<Map<String, String>> result = new ArrayList<>();
         if (o instanceof List) {
             List l = (List) o;
-            for (Object r : l) {
-
+            
+            for (Object value : l) {
                 Map<String, String> oRow = new HashMap<>();
-                if (r instanceof Map) {
-                    @SuppressWarnings("unchecked") Map<Object, Object> iRow = (Map) r;
+                if (value instanceof Map) {
+                    @SuppressWarnings("unchecked") Map<Object, Object> iRow = (Map) value;
                     for (Map.Entry e : iRow.entrySet()) {
                         Object k = e.getKey();
                         Object v = e.getValue();
                         oRow.put(k.toString(), v.toString());
                     }
-                } else if (r instanceof TitanVertex) {
-                    TitanVertex vertex = (TitanVertex) r;
-                    oRow.put("id", vertex.getId().toString());
-                    Iterable<TitanProperty> ps = vertex.getProperties();
-                    for (TitanProperty tP : ps) {
-                        String pName = tP.getPropertyKey().getName();
-                        Object pValue = vertex.getProperty(pName);
-                        if (pValue != null) {
-                            oRow.put(pName, pValue.toString());
+                } else if (value instanceof AtlasVertex) {
+                    AtlasVertex<?,?> vertex = (AtlasVertex<?,?>)value;
+                    for (String key : vertex.getPropertyKeys()) {
+                        Object propertyValue = GraphHelper.getProperty(vertex,  key);
+                        if (propertyValue != null) {
+                            oRow.put(key, propertyValue.toString());
                         }
                     }
-
-                } else if (r instanceof String) {
-                    oRow.put("", r.toString());
-                } else if (r instanceof TitanEdge) {
-                    TitanEdge edge = (TitanEdge) r;
+   
+                } else if (value instanceof String) {
+                    oRow.put("", value.toString());
+                } else if(value instanceof AtlasEdge) {
+                    AtlasEdge edge = (AtlasEdge) value;
                     oRow.put("id", edge.getId().toString());
                     oRow.put("label", edge.getLabel());
-                    oRow.put("inVertex", edge.getVertex(Direction.IN).getId().toString());
-                    oRow.put("outVertex", edge.getVertex(Direction.OUT).getId().toString());
-                    Set<String> propertyKeys = edge.getPropertyKeys();
-                    for (String propertyKey : propertyKeys) {
-                        oRow.put(propertyKey, edge.getProperty(propertyKey).toString());
+                    oRow.put("inVertex", edge.getInVertex().getId().toString());
+                    oRow.put("outVertex", edge.getOutVertex().getId().toString());
+                    for (String propertyKey : edge.getPropertyKeys()) {
+                        oRow.put(propertyKey, GraphHelper.getProperty(edge, propertyKey).toString());
                     }
                 } else {
-                    throw new DiscoveryException(String.format("Cannot process result %s", o.toString()));
+                    throw new DiscoveryException(String.format("Cannot process result %s", String.valueOf(value)));
                 }
-
+    
                 result.add(oRow);
             }
-        } else {
+        }
+        else {
             result.add(new HashMap<String, String>() {{
                 put("result", o.toString());
-            }});
+            }}); 
         }
         return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/repository/graph/AtlasGraphProvider.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/AtlasGraphProvider.java b/repository/src/main/java/org/apache/atlas/repository/graph/AtlasGraphProvider.java
new file mode 100755
index 0000000..f2bfc6d
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/AtlasGraphProvider.java
@@ -0,0 +1,72 @@
+/**
+ * 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.repository.graph;
+
+import org.apache.atlas.repository.RepositoryException;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.GraphDatabase;
+import org.apache.atlas.util.AtlasRepositoryConfiguration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Provides access to the AtlasGraph
+ *
+ */
+public class AtlasGraphProvider implements IAtlasGraphProvider {
+
+    private static volatile GraphDatabase<?,?> graphDb_;    
+
+    public static <V, E> AtlasGraph<V, E> getGraphInstance() {
+        GraphDatabase<?,?> db = getGraphDatabase();      
+        AtlasGraph<?, ?> graph = db.getGraph();
+        return (AtlasGraph<V, E>) graph;
+
+    }
+
+    private static <V, E> GraphDatabase<?,?> getGraphDatabase() {
+
+        try {
+            if (graphDb_ == null) {
+                synchronized(AtlasGraphProvider.class) {
+                    if(graphDb_ == null) {
+                        Class implClass = AtlasRepositoryConfiguration.getGraphDatabaseImpl();
+                        graphDb_ = (GraphDatabase<V, E>) implClass.newInstance();
+                    }
+                }
+            }         
+            return graphDb_;
+        }
+        catch (IllegalAccessException e) {
+            throw new RuntimeException("Error initializing graph database", e);
+        } catch (InstantiationException e) {
+            throw new RuntimeException("Error initializing graph database", e);
+        }
+    }
+
+    @VisibleForTesting
+    public static void cleanup() {
+        getGraphDatabase().cleanup();
+    }
+
+    @Override
+    public AtlasGraph get() throws RepositoryException {
+        return getGraphInstance();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
index 92f98c6..fb014f2 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/DeleteHandler.java
@@ -18,14 +18,23 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.tinkerpop.blueprints.Direction;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.Vertex;
+import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX;
+import static org.apache.atlas.repository.graph.GraphHelper.string;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RequestContext;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.GraphHelper.VertexInfo;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.exception.NullRequiredAttributeException;
 import org.apache.atlas.typesystem.persistence.Id;
 import org.apache.atlas.typesystem.types.AttributeInfo;
@@ -38,16 +47,6 @@ import org.apache.atlas.typesystem.types.TypeSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.atlas.repository.graph.GraphHelper.EDGE_LABEL_PREFIX;
-import static org.apache.atlas.repository.graph.GraphHelper.string;
-
 public abstract class DeleteHandler {
     public static final Logger LOG = LoggerFactory.getLogger(DeleteHandler.class);
 
@@ -71,12 +70,12 @@ public abstract class DeleteHandler {
      * @param instanceVertices
      * @throws AtlasException
      */
-    public void deleteEntities(List<Vertex> instanceVertices) throws AtlasException {
+    public void deleteEntities(List<AtlasVertex> instanceVertices) throws AtlasException {
        RequestContext requestContext = RequestContext.get();
 
-       Set<Vertex> deletionCandidateVertices = new HashSet<>();
+       Set<AtlasVertex> deletionCandidateVertices = new HashSet<>();
 
-       for (Vertex instanceVertex : instanceVertices) {
+       for (AtlasVertex instanceVertex : instanceVertices) {
             String guid = GraphHelper.getIdFromVertex(instanceVertex);
             Id.EntityState state = GraphHelper.getState(instanceVertex);
             if (requestContext.getDeletedEntityIds().contains(guid) || state == Id.EntityState.DELETED) {
@@ -96,13 +95,13 @@ public abstract class DeleteHandler {
        }
 
        // Delete traits and vertices.
-       for (Vertex deletionCandidateVertex : deletionCandidateVertices) {
+       for (AtlasVertex deletionCandidateVertex : deletionCandidateVertices) {
            deleteAllTraits(deletionCandidateVertex);
            deleteTypeVertex(deletionCandidateVertex, false);
        }
     }
 
-    protected abstract void deleteEdge(Edge edge, boolean force) throws AtlasException;
+    protected abstract void deleteEdge(AtlasEdge edge, boolean force) throws AtlasException;
 
     /**
      * Deletes a type vertex - can be entity(class type) or just vertex(struct/trait type)
@@ -110,7 +109,7 @@ public abstract class DeleteHandler {
      * @param typeCategory
      * @throws AtlasException
      */
-    protected void deleteTypeVertex(Vertex instanceVertex, DataTypes.TypeCategory typeCategory, boolean force) throws AtlasException {
+    protected void deleteTypeVertex(AtlasVertex instanceVertex, DataTypes.TypeCategory typeCategory, boolean force) throws AtlasException {
         switch (typeCategory) {
         case STRUCT:
         case TRAIT:
@@ -131,7 +130,7 @@ public abstract class DeleteHandler {
      * @param instanceVertex
      * @throws AtlasException
      */
-    protected void deleteTypeVertex(Vertex instanceVertex, boolean force) throws AtlasException {
+    protected void deleteTypeVertex(AtlasVertex instanceVertex, boolean force) throws AtlasException {
         LOG.debug("Deleting {}", string(instanceVertex));
         String typeName = GraphHelper.getTypeName(instanceVertex);
         IDataType type = typeSystem.getDataType(IDataType.class, typeName);
@@ -158,10 +157,10 @@ public abstract class DeleteHandler {
                 DataTypes.TypeCategory elementTypeCategory = elementType.getTypeCategory();
                 if (elementTypeCategory == DataTypes.TypeCategory.STRUCT ||
                         elementTypeCategory == DataTypes.TypeCategory.CLASS) {
-                    Iterator<Edge> edges = graphHelper.getOutGoingEdgesByLabel(instanceVertex, edgeLabel);
+                    Iterator<AtlasEdge> edges = graphHelper.getOutGoingEdgesByLabel(instanceVertex, edgeLabel);
                     if (edges != null) {
                         while (edges.hasNext()) {
-                            Edge edge = edges.next();
+                            AtlasEdge edge = edges.next();
                             deleteEdgeReference(edge, elementType.getTypeCategory(), attributeInfo.isComposite, false);
                         }
                     }
@@ -176,7 +175,7 @@ public abstract class DeleteHandler {
 
                 if (valueTypeCategory == DataTypes.TypeCategory.STRUCT ||
                         valueTypeCategory == DataTypes.TypeCategory.CLASS) {
-                    List<String> keys = GraphHelper.getProperty(instanceVertex, propertyName);
+                    List<String> keys = GraphHelper.getListProperty(instanceVertex, propertyName);
                     if (keys != null) {
                         for (String key : keys) {
                             String mapEdgeLabel = GraphHelper.getQualifiedNameForMapKey(edgeLabel, key);
@@ -199,7 +198,7 @@ public abstract class DeleteHandler {
      * @return returns true if the edge reference is hard deleted
      * @throws AtlasException
      */
-    public boolean deleteEdgeReference(Edge edge, DataTypes.TypeCategory typeCategory, boolean isComposite,
+    public boolean deleteEdgeReference(AtlasEdge edge, DataTypes.TypeCategory typeCategory, boolean isComposite,
                                     boolean forceDeleteStructTrait) throws AtlasException {
         LOG.debug("Deleting {}", string(edge));
         boolean forceDelete =
@@ -210,14 +209,14 @@ public abstract class DeleteHandler {
             //If the vertex is of type struct/trait, delete the edge and then the reference vertex as the vertex is not shared by any other entities.
             //If the vertex is of type class, and its composite attribute, this reference vertex' lifecycle is controlled
             //through this delete, hence delete the edge and the reference vertex.
-            Vertex vertexForDelete = edge.getVertex(Direction.IN);
+            AtlasVertex vertexForDelete = edge.getInVertex();
 
             //If deleting the edge and then the in vertex, reverse attribute shouldn't be updated
             deleteEdge(edge, false, forceDelete);
             deleteTypeVertex(vertexForDelete, typeCategory, forceDelete);
         } else {
-            //If the vertex is of type class, and its not a composite attributes, the reference vertex' lifecycle is not controlled
-            //through this delete. Hence just remove the reference edge. Leave the reference vertex as is
+            //If the vertex is of type class, and its not a composite attributes, the reference AtlasVertex' lifecycle is not controlled
+            //through this delete. Hence just remove the reference edge. Leave the reference AtlasVertex as is
 
             //If deleting just the edge, reverse attribute should be updated for any references
             //For example, for the department type system, if the person's manager edge is deleted, subordinates of manager should be updated
@@ -226,20 +225,20 @@ public abstract class DeleteHandler {
         return !softDelete || forceDelete;
     }
 
-    public void deleteEdgeReference(Vertex outVertex, String edgeLabel, DataTypes.TypeCategory typeCategory,
+    public void deleteEdgeReference(AtlasVertex outVertex, String edgeLabel, DataTypes.TypeCategory typeCategory,
                                     boolean isComposite) throws AtlasException {
-        Edge edge = graphHelper.getEdgeForLabel(outVertex, edgeLabel);
+        AtlasEdge edge = graphHelper.getEdgeForLabel(outVertex, edgeLabel);
         if (edge != null) {
             deleteEdgeReference(edge, typeCategory, isComposite, false);
         }
     }
 
-    protected void deleteEdge(Edge edge, boolean updateReverseAttribute, boolean force) throws AtlasException {
+    protected void deleteEdge(AtlasEdge edge, boolean updateReverseAttribute, boolean force) throws AtlasException {
         //update reverse attribute
         if (updateReverseAttribute) {
             AttributeInfo attributeInfo = getAttributeForEdge(edge.getLabel());
             if (attributeInfo.reverseAttributeName != null) {
-                deleteEdgeBetweenVertices(edge.getVertex(Direction.IN), edge.getVertex(Direction.OUT),
+                deleteEdgeBetweenVertices(edge.getInVertex(), edge.getOutVertex(),
                         attributeInfo.reverseAttributeName);
             }
         }
@@ -247,25 +246,25 @@ public abstract class DeleteHandler {
         deleteEdge(edge, force);
     }
 
-    protected void deleteVertex(Vertex instanceVertex, boolean force) throws AtlasException {
+    protected void deleteVertex(AtlasVertex instanceVertex, boolean force) throws AtlasException {
         //Update external references(incoming edges) to this vertex
         LOG.debug("Setting the external references to {} to null(removing edges)", string(instanceVertex));
-        Iterator<Edge> edges = instanceVertex.getEdges(Direction.IN).iterator();
+        Iterator<AtlasEdge> edges = instanceVertex.getEdges(AtlasEdgeDirection.IN).iterator();
 
         while(edges.hasNext()) {
-            Edge edge = edges.next();
+            AtlasEdge edge = edges.next();
             Id.EntityState edgeState = GraphHelper.getState(edge);
             if (edgeState == Id.EntityState.ACTIVE) {
                 //Delete only the active edge references
                 AttributeInfo attribute = getAttributeForEdge(edge.getLabel());
                 //TODO use delete edge instead??
-                deleteEdgeBetweenVertices(edge.getVertex(Direction.OUT), edge.getVertex(Direction.IN), attribute.name);
+                deleteEdgeBetweenVertices(edge.getOutVertex(), edge.getInVertex(), attribute.name);
             }
         }
         _deleteVertex(instanceVertex, force);
     }
 
-    protected abstract void _deleteVertex(Vertex instanceVertex, boolean force);
+    protected abstract void _deleteVertex(AtlasVertex instanceVertex, boolean force);
 
     /**
      * Deletes the edge between outvertex and inVertex. The edge is for attribute attributeName of outVertex
@@ -274,7 +273,7 @@ public abstract class DeleteHandler {
      * @param attributeName
      * @throws AtlasException
      */
-    protected void deleteEdgeBetweenVertices(Vertex outVertex, Vertex inVertex, String attributeName) throws AtlasException {
+    protected void deleteEdgeBetweenVertices(AtlasVertex outVertex, AtlasVertex inVertex, String attributeName) throws AtlasException {
         LOG.debug("Removing edge from {} to {} with attribute name {}", string(outVertex), string(inVertex),
                 attributeName);
         String typeName = GraphHelper.getTypeName(outVertex);
@@ -289,7 +288,7 @@ public abstract class DeleteHandler {
         AttributeInfo attributeInfo = getFieldMapping(type).fields.get(attributeName);
         String propertyName = GraphHelper.getQualifiedFieldName(type, attributeName);
         String edgeLabel = EDGE_LABEL_PREFIX + propertyName;
-        Edge edge = null;
+        AtlasEdge edge = null;
 
         switch (attributeInfo.dataType().getTypeCategory()) {
         case CLASS:
@@ -308,17 +307,17 @@ public abstract class DeleteHandler {
 
         case ARRAY:
             //If its array attribute, find the right edge between the two vertices and update array property
-            List<String> elements = GraphHelper.getProperty(outVertex, propertyName);
+            List<String> elements = GraphHelper.getListProperty(outVertex, propertyName);
             if (elements != null) {
                 elements = new ArrayList<>(elements);   //Make a copy, else list.remove reflects on titan.getProperty()
                 for (String elementEdgeId : elements) {
-                    Edge elementEdge = graphHelper.getEdgeByEdgeId(outVertex, edgeLabel, elementEdgeId);
+                    AtlasEdge elementEdge = graphHelper.getEdgeByEdgeId(outVertex, edgeLabel, elementEdgeId);
                     if (elementEdge == null) {
                         continue;
                     }
 
-                    Vertex elementVertex = elementEdge.getVertex(Direction.IN);
-                    if (elementVertex.getId().toString().equals(inVertex.getId().toString())) {
+                    AtlasVertex elementVertex = elementEdge.getInVertex();
+                    if (elementVertex.equals(inVertex)) {
                         edge = elementEdge;
 
                         //TODO element.size includes deleted items as well. should exclude
@@ -349,15 +348,15 @@ public abstract class DeleteHandler {
 
         case MAP:
             //If its map attribute, find the right edge between two vertices and update map property
-            List<String> keys = GraphHelper.getProperty(outVertex, propertyName);
+            List<String> keys = GraphHelper.getListProperty(outVertex, propertyName);
             if (keys != null) {
                 keys = new ArrayList<>(keys);   //Make a copy, else list.remove reflects on titan.getProperty()
                 for (String key : keys) {
                     String keyPropertyName = GraphHelper.getQualifiedNameForMapKey(propertyName, key);
-                    String mapEdgeId = GraphHelper.getProperty(outVertex, keyPropertyName);
-                    Edge mapEdge = graphHelper.getEdgeByEdgeId(outVertex, keyPropertyName, mapEdgeId);
+                    String mapEdgeId = GraphHelper.getSingleValuedProperty(outVertex, keyPropertyName, String.class);
+                    AtlasEdge mapEdge = graphHelper.getEdgeByEdgeId(outVertex, keyPropertyName, mapEdgeId);
                     if(mapEdge != null) {
-                        Vertex mapVertex = mapEdge.getVertex(Direction.IN);
+                        AtlasVertex mapVertex = mapEdge.getInVertex();
                         if (mapVertex.getId().toString().equals(inVertex.getId().toString())) {
                             //TODO keys.size includes deleted items as well. should exclude
                             if (attributeInfo.multiplicity.nullAllowed() || keys.size() > attributeInfo.multiplicity.lower) {
@@ -427,7 +426,7 @@ public abstract class DeleteHandler {
      * @param instanceVertex
      * @throws AtlasException
      */
-    private void deleteAllTraits(Vertex instanceVertex) throws AtlasException {
+    private void deleteAllTraits(AtlasVertex instanceVertex) throws AtlasException {
         List<String> traitNames = GraphHelper.getTraitNames(instanceVertex);
         LOG.debug("Deleting traits {} for {}", traitNames, string(instanceVertex));
         String typeName = GraphHelper.getTypeName(instanceVertex);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java b/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
index b342e27..053e8ac 100644
--- a/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/FullTextMapper.java
@@ -17,7 +17,7 @@
  */
 package org.apache.atlas.repository.graph;
 
-import com.tinkerpop.blueprints.Vertex;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.ITypedInstance;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
@@ -49,7 +49,7 @@ public class FullTextMapper {
         instanceCache = new HashMap<>();
     }
 
-    public String mapRecursive(Vertex instanceVertex, boolean followReferences) throws AtlasException {
+    public String mapRecursive(AtlasVertex instanceVertex, boolean followReferences) throws AtlasException {
         String guid = GraphHelper.getIdFromVertex(instanceVertex);
         ITypedReferenceableInstance typedReference;
         if (instanceCache.containsKey(guid)) {
@@ -121,7 +121,7 @@ public class FullTextMapper {
         case CLASS:
             if (followReferences) {
                 String refGuid = ((ITypedReferenceableInstance) value).getId()._getId();
-                Vertex refVertex = graphHelper.getVertexForGUID(refGuid);
+                AtlasVertex refVertex = graphHelper.getVertexForGUID(refGuid);
                 return mapRecursive(refVertex, false);
             }
             break;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
index 263ea46..691a12c 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepository.java
@@ -18,13 +18,12 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.google.common.base.Preconditions;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.GraphQuery;
-import com.tinkerpop.blueprints.Vertex;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.GraphTransaction;
@@ -32,6 +31,10 @@ import org.apache.atlas.RequestContext;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.MetadataRepository;
 import org.apache.atlas.repository.RepositoryException;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
 import org.apache.atlas.typesystem.exception.EntityExistsException;
@@ -46,10 +49,9 @@ import org.apache.atlas.typesystem.types.TypeSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
 
 /**
  * An implementation backed by a Graph database provided
@@ -64,16 +66,16 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
 
     private static final GraphHelper graphHelper = GraphHelper.getInstance();
 
-    private final TitanGraph titanGraph;
+    private final AtlasGraph graph;
 
     private DeleteHandler deleteHandler;
 
     private GraphToTypedInstanceMapper graphToInstanceMapper;
 
     @Inject
-    public GraphBackedMetadataRepository(GraphProvider<TitanGraph> graphProvider, DeleteHandler deleteHandler) {
-        this.titanGraph = graphProvider.get();
-        graphToInstanceMapper = new GraphToTypedInstanceMapper(titanGraph);
+    public GraphBackedMetadataRepository(DeleteHandler deleteHandler) {
+        this.graph = AtlasGraphProvider.getGraphInstance();
+        graphToInstanceMapper = new GraphToTypedInstanceMapper(graph);
         this.deleteHandler = deleteHandler;
     }
 
@@ -148,7 +150,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
     public ITypedReferenceableInstance getEntityDefinition(String guid) throws RepositoryException, EntityNotFoundException {
         LOG.debug("Retrieving entity with guid={}", guid);
 
-        Vertex instanceVertex = graphHelper.getVertexForGUID(guid);
+        AtlasVertex instanceVertex = graphHelper.getVertexForGUID(guid);
 
         try {
             return graphToInstanceMapper.mapGraphToTypedInstance(guid, instanceVertex);
@@ -164,7 +166,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         LOG.debug("Retrieving entity with type={} and {}={}", entityType, attribute, value);
         IDataType type = typeSystem.getDataType(IDataType.class, entityType);
         String propertyKey = getFieldNameInVertex(type, attribute);
-        Vertex instanceVertex = graphHelper.findVertex(propertyKey, value,
+        AtlasVertex instanceVertex = graphHelper.findVertex(propertyKey, value,
                 Constants.ENTITY_TYPE_PROPERTY_KEY, entityType,
                 Constants.STATE_PROPERTY_KEY, Id.EntityState.ACTIVE.name());
 
@@ -176,15 +178,15 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
     @GraphTransaction
     public List<String> getEntityList(String entityType) throws RepositoryException {
         LOG.debug("Retrieving entity list for type={}", entityType);
-        GraphQuery query = titanGraph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, entityType);
-        Iterator<Vertex> results = query.vertices().iterator();
+        AtlasGraphQuery query = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, entityType);
+        Iterator<AtlasVertex> results = query.vertices().iterator();
         if (!results.hasNext()) {
             return Collections.emptyList();
         }
 
         ArrayList<String> entityList = new ArrayList<>();
         while (results.hasNext()) {
-            Vertex vertex = results.next();
+            AtlasVertex vertex = results.next();
             entityList.add(GraphHelper.getIdFromVertex(vertex));
         }
 
@@ -202,7 +204,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
     @GraphTransaction
     public List<String> getTraitNames(String guid) throws AtlasException {
         LOG.debug("Retrieving trait names for entity={}", guid);
-        Vertex instanceVertex = graphHelper.getVertexForGUID(guid);
+        AtlasVertex instanceVertex = graphHelper.getVertexForGUID(guid);
         return GraphHelper.getTraitNames(instanceVertex);
     }
 
@@ -222,7 +224,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         LOG.debug("Adding a new trait={} for entity={}", traitName, guid);
 
         try {
-            Vertex instanceVertex = graphHelper.getVertexForGUID(guid);
+            AtlasVertex instanceVertex = graphHelper.getVertexForGUID(guid);
 
             // add the trait instance as a new vertex
             final String typeName = GraphHelper.getTypeName(instanceVertex);
@@ -256,7 +258,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
     public void deleteTrait(String guid, String traitNameToBeDeleted) throws TraitNotFoundException, EntityNotFoundException, RepositoryException {
         LOG.debug("Deleting trait={} from entity={}", traitNameToBeDeleted, guid);
         
-        Vertex instanceVertex = graphHelper.getVertexForGUID(guid);
+        AtlasVertex instanceVertex = graphHelper.getVertexForGUID(guid);
 
         List<String> traitNames = GraphHelper.getTraitNames(instanceVertex);
         if (!traitNames.contains(traitNameToBeDeleted)) {
@@ -267,7 +269,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         try {
             final String entityTypeName = GraphHelper.getTypeName(instanceVertex);
             String relationshipLabel = GraphHelper.getTraitLabel(entityTypeName, traitNameToBeDeleted);
-            Edge edge = graphHelper.getEdgeForLabel(instanceVertex, relationshipLabel);
+            AtlasEdge edge = graphHelper.getEdgeForLabel(instanceVertex, relationshipLabel);
             if(edge != null) {
                 deleteHandler.deleteEdgeReference(edge, DataTypes.TypeCategory.TRAIT, false, true);
 
@@ -281,7 +283,7 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
     }
 
     
-    private void updateTraits(Vertex instanceVertex, List<String> traitNames) {
+    private void updateTraits(AtlasVertex instanceVertex, List<String> traitNames) {
         // remove the key
         instanceVertex.removeProperty(Constants.TRAIT_NAMES_PROPERTY_KEY);
 
@@ -332,14 +334,14 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
             throw new IllegalArgumentException("guids must be non-null and non-empty");
         }
 
-        List<Vertex> vertices = new ArrayList<>(guids.size());
+        List<AtlasVertex> vertices = new ArrayList<>(guids.size());
         for (String guid : guids) {
             if (guid == null) {
                 LOG.warn("deleteEntities: Ignoring null guid");
                 continue;
             }
             try {
-                Vertex instanceVertex = graphHelper.getVertexForGUID(guid);
+                AtlasVertex instanceVertex = graphHelper.getVertexForGUID(guid);
                 vertices.add(instanceVertex);
             } catch (EntityNotFoundException e) {
                 // Entity does not exist - treat as non-error, since the caller
@@ -360,4 +362,8 @@ public class GraphBackedMetadataRepository implements MetadataRepository {
         return new AtlasClient.EntityResult(requestContext.getCreatedEntityIds(),
                 requestContext.getUpdatedEntityIds(), requestContext.getDeletedEntityIds());
     }
+
+    public AtlasGraph getGraph() {
+        return AtlasGraphProvider.getGraphInstance();
+    }
 }