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:50 UTC

[2/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/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryTest.java
index 2541541..7a57518 100755
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedMetadataRepositoryTest.java
@@ -18,15 +18,26 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
-import com.tinkerpop.blueprints.Compare;
-import com.tinkerpop.blueprints.Direction;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.GraphQuery;
-import com.tinkerpop.blueprints.Vertex;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createUniqueRequiredAttrDef;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import javax.inject.Inject;
+
 import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.RequestContext;
@@ -35,6 +46,12 @@ import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
 import org.apache.atlas.query.QueryParams;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.RepositoryException;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.IStruct;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.ITypedStruct;
@@ -60,26 +77,11 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
-import scala.actors.threadpool.Arrays;
 
-import javax.inject.Inject;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createUniqueRequiredAttrDef;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNotEquals;
-import static org.testng.Assert.assertNotNull;
-import static org.testng.Assert.assertTrue;
+import scala.actors.threadpool.Arrays;
 
 /**
  * GraphBackedMetadataRepository test
@@ -91,9 +93,6 @@ import static org.testng.Assert.assertTrue;
 public class GraphBackedMetadataRepositoryTest {
 
     @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
-    @Inject
     private GraphBackedMetadataRepository repositoryService;
 
     @Inject
@@ -108,7 +107,7 @@ public class GraphBackedMetadataRepositoryTest {
         typeSystem = TypeSystem.getInstance();
         typeSystem.reset();
 
-        new GraphBackedSearchIndexer(graphProvider);
+        new GraphBackedSearchIndexer();
 
         TestUtils.defineDeptEmployeeTypes(typeSystem);
         TestUtils.createHiveTypes(typeSystem);
@@ -116,23 +115,13 @@ public class GraphBackedMetadataRepositoryTest {
 
     @BeforeMethod
     public void setupContext() {
-        RequestContext.createContext();
+        TestUtils.resetRequestContext();
     }
 
     @AfterClass
     public void tearDown() throws Exception {
         TypeSystem.getInstance().reset();
-        try {
-            //TODO - Fix failure during shutdown while using BDB
-            graphProvider.get().shutdown();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+        AtlasGraphProvider.cleanup();
     }
 
     @Test
@@ -187,9 +176,10 @@ public class GraphBackedMetadataRepositoryTest {
     }
 
     private boolean assertEdge(String id, String typeName) throws Exception {
-        TitanGraph graph = graphProvider.get();
-        Vertex vertex = (Vertex)graph.query().has(Constants.GUID_PROPERTY_KEY, id).vertices().iterator().next();
-        Iterable<Edge> edges = vertex.getEdges(Direction.OUT, Constants.INTERNAL_PROPERTY_KEY_PREFIX + typeName + ".ref");
+        AtlasGraph graph = TestUtils.getGraph();
+        Iterable<AtlasVertex> vertices = graph.query().has(Constants.GUID_PROPERTY_KEY, id).vertices();
+        AtlasVertex AtlasVertex = vertices.iterator().next();
+        Iterable<AtlasEdge> edges = AtlasVertex.getEdges(AtlasEdgeDirection.OUT, Constants.INTERNAL_PROPERTY_KEY_PREFIX + typeName + ".ref");
         if (!edges.iterator().hasNext()) {
             ITypedReferenceableInstance entity = repositoryService.getEntityDefinition(id);
             assertNotNull(entity.get("ref"));
@@ -290,9 +280,9 @@ public class GraphBackedMetadataRepositoryTest {
 
     @GraphTransaction
     String getGUID() {
-        Vertex tableVertex = getTableEntityVertex();
+        AtlasVertex tableVertex = getTableEntityVertex();
 
-        String guid = tableVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+        String guid = GraphHelper.getSingleValuedProperty(tableVertex, Constants.GUID_PROPERTY_KEY, String.class);
         if (guid == null) {
             Assert.fail();
         }
@@ -300,12 +290,12 @@ public class GraphBackedMetadataRepositoryTest {
     }
 
     @GraphTransaction
-    Vertex getTableEntityVertex() {
-        TitanGraph graph = graphProvider.get();
-        GraphQuery query = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, Compare.EQUAL, TestUtils.TABLE_TYPE);
-        Iterator<Vertex> results = query.vertices().iterator();
+    AtlasVertex getTableEntityVertex() {
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphQuery query = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, ComparisionOperator.EQUAL, TestUtils.TABLE_TYPE);
+        Iterator<AtlasVertex> results = query.vertices().iterator();
         // returning one since guid should be unique
-        Vertex tableVertex = results.hasNext() ? results.next() : null;
+        AtlasVertex tableVertex = results.hasNext() ? results.next() : null;
         if (tableVertex == null) {
             Assert.fail();
         }
@@ -365,8 +355,8 @@ public class GraphBackedMetadataRepositoryTest {
     @Test(dependsOnMethods = "testGetTraitNames")
     public void testAddTrait() throws Exception {
         final String aGUID = getGUID();
-        Vertex vertex = GraphHelper.getInstance().getVertexForGUID(aGUID);
-        Long modificationTimestampPreUpdate = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
+        AtlasVertex AtlasVertex = GraphHelper.getInstance().getVertexForGUID(aGUID);
+        Long modificationTimestampPreUpdate = GraphHelper.getSingleValuedProperty(AtlasVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class);
         Assert.assertNotNull(modificationTimestampPreUpdate);
 
         List<String> traitNames = repositoryService.getTraitNames(aGUID);
@@ -388,7 +378,7 @@ public class GraphBackedMetadataRepositoryTest {
         
         // Verify modification timestamp was updated.
         GraphHelper.getInstance().getVertexForGUID(aGUID);
-        Long modificationTimestampPostUpdate = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
+        Long modificationTimestampPostUpdate = GraphHelper.getSingleValuedProperty(AtlasVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class);
         Assert.assertNotNull(modificationTimestampPostUpdate);
     }
 
@@ -406,7 +396,7 @@ public class GraphBackedMetadataRepositoryTest {
 
         repositoryService.addTrait(aGUID, traitInstance);
 
-        TestUtils.dumpGraph(graphProvider.get());
+        TestUtils.dumpGraph(TestUtils.getGraph());
 
         // refresh trait names
         List<String> traitNames = repositoryService.getTraitNames(aGUID);
@@ -437,8 +427,8 @@ public class GraphBackedMetadataRepositoryTest {
     @Test(dependsOnMethods = "testAddTrait")
     public void testDeleteTrait() throws Exception {
         final String aGUID = getGUID();
-        Vertex vertex = GraphHelper.getInstance().getVertexForGUID(aGUID);
-        Long modificationTimestampPreUpdate = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
+        AtlasVertex AtlasVertex = GraphHelper.getInstance().getVertexForGUID(aGUID);
+        Long modificationTimestampPreUpdate = GraphHelper.getSingleValuedProperty(AtlasVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class);
         Assert.assertNotNull(modificationTimestampPreUpdate);
 
         List<String> traitNames = repositoryService.getTraitNames(aGUID);
@@ -457,7 +447,7 @@ public class GraphBackedMetadataRepositoryTest {
         
         // Verify modification timestamp was updated.
         GraphHelper.getInstance().getVertexForGUID(aGUID);
-        Long modificationTimestampPostUpdate = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY);
+        Long modificationTimestampPostUpdate = GraphHelper.getSingleValuedProperty(AtlasVertex, Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class);
         Assert.assertNotNull(modificationTimestampPostUpdate);
         Assert.assertTrue(modificationTimestampPostUpdate > modificationTimestampPreUpdate);
     }
@@ -477,20 +467,20 @@ public class GraphBackedMetadataRepositoryTest {
 
     @Test(dependsOnMethods = "testCreateEntity")
     public void testGetIdFromVertex() throws Exception {
-        Vertex tableVertex = getTableEntityVertex();
+        AtlasVertex tableVertex = getTableEntityVertex();
 
-        String guid = tableVertex.getProperty(Constants.GUID_PROPERTY_KEY);
+        String guid = GraphHelper.getSingleValuedProperty(tableVertex, Constants.GUID_PROPERTY_KEY, String.class);
         if (guid == null) {
             Assert.fail();
         }
 
-        Id expected = new Id(guid, tableVertex.<Integer>getProperty(Constants.VERSION_PROPERTY_KEY), TestUtils.TABLE_TYPE);
+        Id expected = new Id(guid, GraphHelper.getSingleValuedProperty(tableVertex, Constants.VERSION_PROPERTY_KEY, Integer.class), TestUtils.TABLE_TYPE);
         Assert.assertEquals(GraphHelper.getIdFromVertex(TestUtils.TABLE_TYPE, tableVertex), expected);
     }
 
     @Test(dependsOnMethods = "testCreateEntity")
     public void testGetTypeName() throws Exception {
-        Vertex tableVertex = getTableEntityVertex();
+        AtlasVertex tableVertex = getTableEntityVertex();
         Assert.assertEquals(GraphHelper.getTypeName(tableVertex), TestUtils.TABLE_TYPE);
     }
 
@@ -559,7 +549,7 @@ public class GraphBackedMetadataRepositoryTest {
         String dslQuery = "hive_table as t where name = 'bar' "
             + "database where name = 'foo' and description = 'foo database' select t";
 
-        TestUtils.dumpGraph(graphProvider.get());
+        TestUtils.dumpGraph(TestUtils.getGraph());
 
         System.out.println("Executing dslQuery = " + dslQuery);
         String jsonResults = discoveryService.searchByDSL(dslQuery, queryParams);
@@ -592,7 +582,7 @@ public class GraphBackedMetadataRepositoryTest {
         //but with elasticsearch, doesn't work without sleep. why??
         long sleepInterval = 1000;
 
-        TestUtils.dumpGraph(graphProvider.get());
+        TestUtils.dumpGraph(TestUtils.getGraph());
 
         //person in hr department whose name is john
         Thread.sleep(sleepInterval);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositoryHardDeleteTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositoryHardDeleteTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositoryHardDeleteTest.java
index 79b48b5..7929505 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositoryHardDeleteTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositoryHardDeleteTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.tinkerpop.blueprints.Vertex;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
@@ -99,7 +99,7 @@ public class GraphBackedRepositoryHardDeleteTest extends GraphBackedMetadataRepo
     }
 
     @Override
-    protected void assertVerticesDeleted(List<Vertex> vertices) {
+    protected void assertVerticesDeleted(List<AtlasVertex> vertices) {
         assertEquals(vertices.size(), 0);
     }
 
@@ -171,10 +171,10 @@ public class GraphBackedRepositoryHardDeleteTest extends GraphBackedMetadataRepo
         assertNull(mapOwnerInstance.get("map"));
         assertNull(mapOwnerInstance.get("biMap"));
 
-        Vertex mapOwnerVertex = GraphHelper.getInstance().getVertexForGUID(mapOwnerGuid);
-        Object object = mapOwnerVertex.getProperty("MapOwner.map.value1");
+        AtlasVertex mapOwnerVertex = GraphHelper.getInstance().getVertexForGUID(mapOwnerGuid);
+        Object object = mapOwnerVertex.getProperty("MapOwner.map.value1", String.class);
         assertNull(object);
-        object = mapOwnerVertex.getProperty("MapOwner.biMap.value1");
+        object = mapOwnerVertex.getProperty("MapOwner.biMap.value1", String.class);
         assertNull(object);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositorySoftDeleteTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositorySoftDeleteTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositorySoftDeleteTest.java
index a0af487..93926ef 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositorySoftDeleteTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedRepositorySoftDeleteTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.tinkerpop.blueprints.Vertex;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
@@ -116,9 +116,9 @@ public class GraphBackedRepositorySoftDeleteTest extends GraphBackedMetadataRepo
     }
 
     @Override
-    protected void assertVerticesDeleted(List<Vertex> vertices) {
-        for (Vertex vertex : vertices) {
-            assertEquals(vertex.getProperty(Constants.STATE_PROPERTY_KEY), Id.EntityState.DELETED.name());
+    protected void assertVerticesDeleted(List<AtlasVertex> vertices) {
+        for (AtlasVertex vertex : vertices) {
+            assertEquals(GraphHelper.getSingleValuedProperty(vertex, Constants.STATE_PROPERTY_KEY, String.class), Id.EntityState.DELETED.name());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerMockTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerMockTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerMockTest.java
index f3680de..398ea62 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerMockTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerMockTest.java
@@ -18,36 +18,33 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.schema.TitanManagement;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.ha.HAConfiguration;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.IndexException;
 import org.apache.atlas.repository.RepositoryException;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphManagement;
 import org.apache.commons.configuration.Configuration;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyZeroInteractions;
-import static org.mockito.Mockito.when;
-
-public class GraphBackedSearchIndexerMockTest {
+public class GraphBackedSearchIndexerMockTest implements IAtlasGraphProvider {
 
     @Mock
     private Configuration configuration;
 
     @Mock
-    private GraphProvider<TitanGraph> graphProvider;
+    private AtlasGraph graph;
 
     @Mock
-    private TitanGraph titanGraph;
-
-    @Mock
-    private TitanManagement titanManagement;
+    private AtlasGraphManagement management;
 
     @BeforeMethod
     public void setup() {
@@ -56,26 +53,24 @@ public class GraphBackedSearchIndexerMockTest {
 
     @Test
     public void testSearchIndicesAreInitializedOnConstructionWhenHAIsDisabled() throws IndexException, RepositoryException {
-        when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY, false)).thenReturn(false);
-        when(graphProvider.get()).thenReturn(titanGraph);
-        when(titanGraph.getManagementSystem()).thenReturn(titanManagement);
-        when(titanManagement.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
+        when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY, false)).thenReturn(false);        
+        when(graph.getManagementSystem()).thenReturn(management);
+        when(management.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
 
-        GraphBackedSearchIndexer graphBackedSearchIndexer = new GraphBackedSearchIndexer(graphProvider, configuration);
+        GraphBackedSearchIndexer graphBackedSearchIndexer = new GraphBackedSearchIndexer(this, configuration);
 
-        verify(titanManagement).containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY);
+        verify(management).containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY);
     }
 
     @Test
     public void testSearchIndicesAreNotInitializedOnConstructionWhenHAIsEnabled() throws IndexException, RepositoryException {
         when(configuration.containsKey(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY)).thenReturn(true);
         when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY)).thenReturn(true);
-        when(graphProvider.get()).thenReturn(titanGraph);
-        when(titanGraph.getManagementSystem()).thenReturn(titanManagement);
-        when(titanManagement.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
+        when(graph.getManagementSystem()).thenReturn(management);
+        when(management.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
 
-        new GraphBackedSearchIndexer(graphProvider, configuration);
-        verifyZeroInteractions(titanManagement);
+        new GraphBackedSearchIndexer(this, configuration);
+        verifyZeroInteractions(management);
 
     }
 
@@ -83,13 +78,18 @@ public class GraphBackedSearchIndexerMockTest {
     public void testIndicesAreReinitializedWhenServerBecomesActive() throws AtlasException {
         when(configuration.containsKey(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY)).thenReturn(true);
         when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY)).thenReturn(true);
-        when(graphProvider.get()).thenReturn(titanGraph);
-        when(titanGraph.getManagementSystem()).thenReturn(titanManagement);
-        when(titanManagement.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
+        when(graph.getManagementSystem()).thenReturn(management);
+        when(management.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)).thenReturn(true);
 
-        GraphBackedSearchIndexer graphBackedSearchIndexer = new GraphBackedSearchIndexer(graphProvider, configuration);
+        GraphBackedSearchIndexer graphBackedSearchIndexer = new GraphBackedSearchIndexer(this, configuration);
         graphBackedSearchIndexer.instanceIsActive();
 
-        verify(titanManagement).containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY);
+        verify(management).containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY);
     }
+    
+
+    @Override
+      public AtlasGraph get() {
+          return graph;
+      }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerTest.java
index 3291e72..1716ac5 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexerTest.java
@@ -18,16 +18,22 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.google.inject.Inject;
-import com.thinkaurelius.titan.core.PropertyKey;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.schema.TitanGraphIndex;
-import com.thinkaurelius.titan.core.schema.TitanManagement;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.Vertex;
+import static junit.framework.Assert.assertTrue;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+
+import java.util.Arrays;
+import java.util.Set;
+
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.TestUtils;
 import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphIndex;
+import org.apache.atlas.repository.graphdb.AtlasGraphManagement;
+import org.apache.atlas.repository.graphdb.AtlasPropertyKey;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes;
 import org.apache.atlas.typesystem.types.EnumType;
@@ -39,115 +45,133 @@ import org.apache.commons.lang.RandomStringUtils;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import java.util.Arrays;
-
-import static junit.framework.Assert.assertTrue;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertNotNull;
+import com.google.inject.Inject;
 
 @Guice(modules = RepositoryMetadataModule.class)
 public class GraphBackedSearchIndexerTest {
 
-    @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
+ 
     @Inject
     private GraphBackedSearchIndexer graphBackedSearchIndexer;
 
     @Test
     public void verifySystemMixedIndexes() {
-        TitanGraph titanGraph = graphProvider.get();
-        TitanManagement managementSystem = titanGraph.getManagementSystem();
-
-        TitanGraphIndex vertexIndex = managementSystem.getGraphIndex(Constants.VERTEX_INDEX);
-        assertNotNull(vertexIndex);
-        assertTrue(vertexIndex.isMixedIndex());
-        assertTrue(Vertex.class.isAssignableFrom(vertexIndex.getIndexedElement()));
-
-        TitanGraphIndex edgeIndex = managementSystem.getGraphIndex(Constants.EDGE_INDEX);
-        assertNotNull(edgeIndex);
-        assertTrue(edgeIndex.isMixedIndex());
-        assertTrue(Edge.class.isAssignableFrom(edgeIndex.getIndexedElement()));
-
-        verifyVertexIndexContains(managementSystem, Constants.STATE_PROPERTY_KEY);
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphManagement managementSystem = graph.getManagementSystem();
+        try {
+            AtlasGraphIndex vertexIndex = managementSystem.getGraphIndex(Constants.VERTEX_INDEX);
+            assertNotNull(vertexIndex);
+            assertTrue(vertexIndex.isMixedIndex());
+            assertFalse(vertexIndex.isEdgeIndex());
+            assertTrue(vertexIndex.isVertexIndex());
+            
+            AtlasGraphIndex edgeIndex = managementSystem.getGraphIndex(Constants.EDGE_INDEX);
+            assertNotNull(edgeIndex);
+            assertTrue(edgeIndex.isMixedIndex());
+            assertTrue(edgeIndex.isEdgeIndex());
+            assertFalse(edgeIndex.isVertexIndex());
+           
+    
+            verifyVertexIndexContains(managementSystem, Constants.STATE_PROPERTY_KEY);
+        }
+        finally {
+            managementSystem.rollback();
+        }
     }
 
     @Test
     public void verifySystemCompositeIndexes() {
-        TitanGraph titanGraph = graphProvider.get();
-        TitanManagement managementSystem = titanGraph.getManagementSystem();
-
-        verifySystemCompositeIndex(managementSystem, Constants.GUID_PROPERTY_KEY, true);
-        verifyVertexIndexContains(managementSystem, Constants.GUID_PROPERTY_KEY);
-
-        verifySystemCompositeIndex(managementSystem, Constants.ENTITY_TYPE_PROPERTY_KEY, false);
-        verifyVertexIndexContains(managementSystem, Constants.ENTITY_TYPE_PROPERTY_KEY);
-
-        verifySystemCompositeIndex(managementSystem, Constants.SUPER_TYPES_PROPERTY_KEY, false);
-        verifyVertexIndexContains(managementSystem, Constants.SUPER_TYPES_PROPERTY_KEY);
-
-        verifySystemCompositeIndex(managementSystem, Constants.TRAIT_NAMES_PROPERTY_KEY, false);
-        verifyVertexIndexContains(managementSystem, Constants.TRAIT_NAMES_PROPERTY_KEY);
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphManagement managementSystem = graph.getManagementSystem();
+        try {
+            verifySystemCompositeIndex(managementSystem, Constants.GUID_PROPERTY_KEY, true);
+            verifyVertexIndexContains(managementSystem, Constants.GUID_PROPERTY_KEY);
+    
+            verifySystemCompositeIndex(managementSystem, Constants.ENTITY_TYPE_PROPERTY_KEY, false);
+            verifyVertexIndexContains(managementSystem, Constants.ENTITY_TYPE_PROPERTY_KEY);
+    
+            verifySystemCompositeIndex(managementSystem, Constants.SUPER_TYPES_PROPERTY_KEY, false);
+            verifyVertexIndexContains(managementSystem, Constants.SUPER_TYPES_PROPERTY_KEY);
+    
+            verifySystemCompositeIndex(managementSystem, Constants.TRAIT_NAMES_PROPERTY_KEY, false);
+            verifyVertexIndexContains(managementSystem, Constants.TRAIT_NAMES_PROPERTY_KEY);
+        }
+        finally {
+            managementSystem.rollback();
+        }
     }
 
     @Test
     public void verifyFullTextIndex() {
-        TitanGraph titanGraph = graphProvider.get();
-        TitanManagement managementSystem = titanGraph.getManagementSystem();
-
-        TitanGraphIndex fullTextIndex = managementSystem.getGraphIndex(Constants.FULLTEXT_INDEX);
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphManagement managementSystem = graph.getManagementSystem();
+        try {
+        AtlasGraphIndex fullTextIndex = managementSystem.getGraphIndex(Constants.FULLTEXT_INDEX);
         assertTrue(fullTextIndex.isMixedIndex());
 
         Arrays.asList(fullTextIndex.getFieldKeys()).contains(
                 managementSystem.getPropertyKey(Constants.ENTITY_TEXT_PROPERTY_KEY));
+        }
+        finally {
+            managementSystem.rollback();
+        }
     }
 
     @Test
     public void verifyTypeStoreIndexes() {
-        TitanGraph titanGraph = graphProvider.get();
-        TitanManagement managementSystem = titanGraph.getManagementSystem();
-
-        verifySystemCompositeIndex(managementSystem, Constants.TYPENAME_PROPERTY_KEY, true);
-        verifyVertexIndexContains(managementSystem, Constants.TYPENAME_PROPERTY_KEY);
-
-        verifySystemCompositeIndex(managementSystem, Constants.VERTEX_TYPE_PROPERTY_KEY, false);
-        verifyVertexIndexContains(managementSystem, Constants.VERTEX_TYPE_PROPERTY_KEY);
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphManagement managementSystem = graph.getManagementSystem();
+        try {
+            verifySystemCompositeIndex(managementSystem, Constants.TYPENAME_PROPERTY_KEY, true);
+            verifyVertexIndexContains(managementSystem, Constants.TYPENAME_PROPERTY_KEY);
+    
+            verifySystemCompositeIndex(managementSystem, Constants.VERTEX_TYPE_PROPERTY_KEY, false);
+            verifyVertexIndexContains(managementSystem, Constants.VERTEX_TYPE_PROPERTY_KEY);
+        }
+        finally {
+            managementSystem.rollback();
+        }
+        
     }
 
     @Test
     public void verifyUserDefinedTypeIndex() throws AtlasException {
-        TitanGraph titanGraph = graphProvider.get();
-        TitanManagement managementSystem = titanGraph.getManagementSystem();
-
-        TypeSystem typeSystem = TypeSystem.getInstance();
-
-        String enumName = "randomEnum" + RandomStringUtils.randomAlphanumeric(10);
-        EnumType managedType = typeSystem.defineEnumType(enumName, new EnumValue("randomEnumValue", 0));
-
-        HierarchicalTypeDefinition<ClassType> databaseTypeDefinition =
-                createClassTypeDef("Database", "Database type description", null,
-                        TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
-                        TypesUtil.createRequiredAttrDef("managedType", managedType));
-
-        ClassType databaseType = typeSystem.defineClassType(databaseTypeDefinition);
-        graphBackedSearchIndexer.onAdd(Arrays.asList(databaseType));
-
-        verifySystemCompositeIndex(managementSystem, "Database.name" + Constants.ENTITY_TYPE_PROPERTY_KEY, false);
-        verifyVertexIndexContains(managementSystem, "Database.name" + Constants.ENTITY_TYPE_PROPERTY_KEY);
-        verifySystemCompositeIndex(managementSystem, "Database.name" + Constants.SUPER_TYPES_PROPERTY_KEY, false);
-
-        verifyVertexIndexContains(managementSystem, "Database.managedType");
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasGraphManagement managementSystem = graph.getManagementSystem();
+        try {
+            TypeSystem typeSystem = TypeSystem.getInstance();
+    
+            String enumName = "randomEnum" + RandomStringUtils.randomAlphanumeric(10);
+            EnumType managedType = typeSystem.defineEnumType(enumName, new EnumValue("randomEnumValue", 0));
+    
+            HierarchicalTypeDefinition<ClassType> databaseTypeDefinition =
+                    createClassTypeDef("Database", "Database type description", null,
+                            TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
+                            TypesUtil.createRequiredAttrDef("managedType", managedType));
+                
+            ClassType databaseType = typeSystem.defineClassType(databaseTypeDefinition);
+            graphBackedSearchIndexer.onAdd(Arrays.asList(databaseType));
+    
+            verifySystemCompositeIndex(managementSystem, "Database.name" + Constants.ENTITY_TYPE_PROPERTY_KEY, false);
+            verifyVertexIndexContains(managementSystem, "Database.name" + Constants.ENTITY_TYPE_PROPERTY_KEY);
+            verifySystemCompositeIndex(managementSystem, "Database.name" + Constants.SUPER_TYPES_PROPERTY_KEY, false);
+    
+            verifyVertexIndexContains(managementSystem, "Database.managedType");
+        }
+        finally {
+            //search indexer uses its own titan management transaction
+            managementSystem.rollback();
+        }
     }
 
-    private void verifyVertexIndexContains(TitanManagement managementSystem, String indexName) {
-        TitanGraphIndex vertexIndex = managementSystem.getGraphIndex(Constants.VERTEX_INDEX);
-        PropertyKey[] fieldKeys = vertexIndex.getFieldKeys();
+    private void verifyVertexIndexContains(AtlasGraphManagement managementSystem, String indexName) {
+        AtlasGraphIndex vertexIndex = managementSystem.getGraphIndex(Constants.VERTEX_INDEX);
+        Set<AtlasPropertyKey> fieldKeys = vertexIndex.getFieldKeys();
         Arrays.asList(fieldKeys).contains(managementSystem.getPropertyKey(indexName));
     }
 
-    private void verifySystemCompositeIndex(TitanManagement managementSystem, String indexName, boolean isUnique) {
-        TitanGraphIndex systemIndex = managementSystem.getGraphIndex(indexName);
+    private void verifySystemCompositeIndex(AtlasGraphManagement managementSystem, String indexName, boolean isUnique) {
+        AtlasGraphIndex systemIndex = managementSystem.getGraphIndex(indexName);
         assertNotNull(systemIndex);
         assertTrue(systemIndex.isCompositeIndex());
         if (isUnique) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperMockTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperMockTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperMockTest.java
index 5ebc2f7..dbd4bf1 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperMockTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperMockTest.java
@@ -17,51 +17,51 @@
  */
 package org.apache.atlas.repository.graph;
 
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.TitanVertex;
-import com.tinkerpop.blueprints.Direction;
-import com.tinkerpop.blueprints.Edge;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+import java.util.Iterator;
+
 import org.apache.atlas.repository.RepositoryException;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.mockito.MockitoAnnotations;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.util.Iterator;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.testng.Assert.assertEquals;
-
 public class GraphHelperMockTest {
 
     private GraphHelper graphHelperInstance;
 
-    private TitanGraph graph;
+    private AtlasGraph graph;
 
     @BeforeClass
     public void setup() {
         MockitoAnnotations.initMocks(this);
-        graph = mock(TitanGraph.class);
+        graph = mock(AtlasGraph.class);
         graphHelperInstance = GraphHelper.getInstance(graph);
     }
 
     @Test(expectedExceptions = RepositoryException.class)
     public void testGetOrCreateEdgeLabelWithMaxRetries() throws Exception {
         final String edgeLabel = "testLabel";
-        TitanVertex v1 = mock(TitanVertex.class);
-        TitanVertex v2 = mock(TitanVertex.class);
+        AtlasVertex v1 = mock(AtlasVertex.class);
+        AtlasVertex v2 = mock(AtlasVertex.class);
 
-        Iterable noEdgesIterable = new Iterable<Edge>() {
+        Iterable noEdgesIterable = new Iterable<AtlasEdge>() {
             @Override
-            public Iterator<Edge> iterator() {
-                return new Iterator<Edge>() {
+            public Iterator<AtlasEdge> iterator() {
+                return new Iterator<AtlasEdge>() {
                     @Override
                     public boolean hasNext() {
                         return false;
                     }
 
                     @Override
-                    public Edge next() {
+                    public AtlasEdge next() {
                         return null;
                     }
 
@@ -71,33 +71,33 @@ public class GraphHelperMockTest {
                 };
             }
         };
-        when(v2.getEdges(Direction.IN)).thenReturn(noEdgesIterable);
-        when(v1.getEdges(Direction.OUT)).thenReturn(noEdgesIterable);
+        when(v2.getEdges(AtlasEdgeDirection.IN)).thenReturn(noEdgesIterable);
+        when(v1.getEdges(AtlasEdgeDirection.OUT)).thenReturn(noEdgesIterable);
 
         when(v1.getId()).thenReturn(new String("1234"));
         when(v2.getId()).thenReturn(new String("5678"));
-        when(graph.addEdge(null, v1, v2, edgeLabel)).thenThrow(new RuntimeException("Unique property constraint violated"));
+        when(graph.addEdge(v1, v2, edgeLabel)).thenThrow(new RuntimeException("Unique property constraint violated"));
         graphHelperInstance.getOrCreateEdge(v1, v2, edgeLabel);
     }
 
     @Test
     public void testGetOrCreateEdgeLabelWithRetries() throws Exception {
         final String edgeLabel = "testLabel";
-        TitanVertex v1 = mock(TitanVertex.class);
-        TitanVertex v2 = mock(TitanVertex.class);
-        Edge edge = mock(Edge.class);
+        AtlasVertex v1 = mock(AtlasVertex.class);
+        AtlasVertex v2 = mock(AtlasVertex.class);
+        AtlasEdge edge = mock(AtlasEdge.class);
 
-        Iterable noEdgesIterable = new Iterable<Edge>() {
+        Iterable noEdgesIterable = new Iterable<AtlasEdge>() {
             @Override
-            public Iterator<Edge> iterator() {
-                return new Iterator<Edge>() {
+            public Iterator<AtlasEdge> iterator() {
+                return new Iterator<AtlasEdge>() {
                     @Override
                     public boolean hasNext() {
                         return false;
                     }
 
                     @Override
-                    public Edge next() {
+                    public AtlasEdge next() {
                         return null;
                     }
 
@@ -107,15 +107,15 @@ public class GraphHelperMockTest {
                 };
             }
         };
-        when(v2.getEdges(Direction.IN)).thenReturn(noEdgesIterable);
-        when(v1.getEdges(Direction.OUT)).thenReturn(noEdgesIterable);
+        when(v2.getEdges(AtlasEdgeDirection.IN)).thenReturn(noEdgesIterable);
+        when(v1.getEdges(AtlasEdgeDirection.OUT)).thenReturn(noEdgesIterable);
 
         when(v1.getId()).thenReturn(new String("v1"));
         when(v2.getId()).thenReturn(new String("v2"));
         when(edge.getId()).thenReturn(new String("edge"));
-        when(graph.addEdge(null, v1, v2, edgeLabel))
+        when(graph.addEdge(v1, v2, edgeLabel))
                 .thenThrow(new RuntimeException("Unique property constraint violated")).thenReturn(edge);
-        Edge redge = graphHelperInstance.getOrCreateEdge(v1, v2, edgeLabel);
+        AtlasEdge redge = graphHelperInstance.getOrCreateEdge(v1, v2, edgeLabel);
         assertEquals(edge, redge);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
index ad34aae..fe15014 100644
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphHelperTest.java
@@ -25,8 +25,8 @@ import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
@@ -35,6 +35,9 @@ import javax.inject.Inject;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.repository.graph.GraphHelper.VertexInfo;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.testng.Assert;
@@ -44,16 +47,9 @@ import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.TitanVertex;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.Vertex;
-
 @Guice(modules = RepositoryMetadataModule.class)
 public class GraphHelperTest {
-    @Inject
-    private GraphProvider<TitanGraph> graphProvider;
+
 
     @DataProvider(name = "encodeDecodeTestData")
     private Object[][] createTestData() {
@@ -81,25 +77,14 @@ public class GraphHelperTest {
         typeSystem = TypeSystem.getInstance();
         typeSystem.reset();
 
-        new GraphBackedSearchIndexer(graphProvider);
+        new GraphBackedSearchIndexer();
 
         TestUtils.defineDeptEmployeeTypes(typeSystem);
     }
 
     @AfterClass
-    public void tearDown() throws Exception {
-        TypeSystem.getInstance().reset();
-        try {
-            //TODO - Fix failure during shutdown while using BDB
-            graphProvider.get().shutdown();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+    public void tearDown() {
+        AtlasGraphProvider.cleanup();
     }
 
     @Test
@@ -116,7 +101,7 @@ public class GraphHelperTest {
                 deptGuid = guid;
             }
         }
-        Vertex deptVertex = GraphHelper.getInstance().getVertexForGUID(deptGuid);
+        AtlasVertex deptVertex = GraphHelper.getInstance().getVertexForGUID(deptGuid);
         Set<VertexInfo> compositeVertices = GraphHelper.getInstance().getCompositeVertices(deptVertex);
         HashMap<String, VertexInfo> verticesByGuid = new HashMap<>();
         for (VertexInfo vertexInfo: compositeVertices) {
@@ -141,14 +126,13 @@ public class GraphHelperTest {
 
     @Test
     public void testGetOutgoingEdgesByLabel() throws Exception {
-        TitanGraph graph = graphProvider.get();
-        TitanVertex v1 = graph.addVertex();
-        TitanVertex v2 = graph.addVertex();
-
-        v1.addEdge("l1", v2);
-        v1.addEdge("l2", v2);
+        AtlasGraph graph = TestUtils.getGraph();
+        AtlasVertex v1 = graph.addVertex();
+        AtlasVertex v2 = graph.addVertex();
+        graph.addEdge(v1, v2, "l1");
+        graph.addEdge(v1, v2, "l2");
 
-        Iterator<Edge> iterator = GraphHelper.getInstance().getOutGoingEdgesByLabel(v1, "l1");
+        Iterator<AtlasEdge> iterator = GraphHelper.getInstance().getOutGoingEdgesByLabel(v1, "l1");
         assertTrue(iterator.hasNext());
         assertTrue(iterator.hasNext());
         assertNotNull(iterator.next());

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/graph/GraphRepoMapperScaleTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/graph/GraphRepoMapperScaleTest.java b/repository/src/test/java/org/apache/atlas/repository/graph/GraphRepoMapperScaleTest.java
index 0a870d8..49ef551 100755
--- a/repository/src/test/java/org/apache/atlas/repository/graph/GraphRepoMapperScaleTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/graph/GraphRepoMapperScaleTest.java
@@ -18,18 +18,23 @@
 
 package org.apache.atlas.repository.graph;
 
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.TitanIndexQuery;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
-import com.tinkerpop.blueprints.Compare;
-import com.tinkerpop.blueprints.GraphQuery;
-import com.tinkerpop.blueprints.Predicate;
-import com.tinkerpop.blueprints.Vertex;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+
+import javax.inject.Inject;
+
+import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.GraphTransaction;
 import org.apache.atlas.RepositoryMetadataModule;
-import org.apache.atlas.RequestContext;
 import org.apache.atlas.TestUtils;
 import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.ITypedReferenceableInstance;
 import org.apache.atlas.typesystem.Referenceable;
 import org.apache.atlas.typesystem.Struct;
@@ -45,11 +50,6 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import javax.inject.Inject;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-
 @Test
 @Guice(modules = RepositoryMetadataModule.class)
 public class GraphRepoMapperScaleTest {
@@ -58,9 +58,6 @@ public class GraphRepoMapperScaleTest {
     private static final String TABLE_NAME = "bar";
 
     @Inject
-    GraphProvider<TitanGraph> graphProvider;
-
-    @Inject
     private GraphBackedMetadataRepository repositoryService;
 
     @Inject
@@ -73,6 +70,9 @@ public class GraphRepoMapperScaleTest {
     @BeforeClass
     @GraphTransaction
     public void setUp() throws Exception {
+        //force up front graph initialization
+        TestUtils.getGraph();
+        searchIndexer = new GraphBackedSearchIndexer(new AtlasGraphProvider(), ApplicationProperties.get());
         //Make sure we can cleanup the index directory
         Collection<IDataType> typesAdded = TestUtils.createHiveTypes(typeSystem);
         searchIndexer.onAdd(typesAdded);
@@ -80,23 +80,13 @@ public class GraphRepoMapperScaleTest {
 
     @BeforeMethod
     public void setupContext() {
-        RequestContext.createContext();
+        TestUtils.resetRequestContext();
     }
 
     @AfterClass
     public void tearDown() throws Exception {
         TypeSystem.getInstance().reset();
-        try {
-            //TODO - Fix failure during shutdown while using BDB
-            graphProvider.get().shutdown();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+        AtlasGraphProvider.cleanup();
     }
 
     @Test
@@ -130,7 +120,7 @@ public class GraphRepoMapperScaleTest {
 
         searchWithOutIndex("hive_table.name", "bar-999");
         searchWithIndex("hive_table.name", "bar-999");
-        searchWithIndex("hive_table.created", Compare.GREATER_THAN_EQUAL, TestUtils.TEST_DATE_IN_LONG, 1000);
+        searchWithIndex("hive_table.created", ComparisionOperator.GREATER_THAN_EQUAL, TestUtils.TEST_DATE_IN_LONG, 1000);
 
         for (int index = 500; index < 600; index++) {
             searchWithIndex("hive_table.name", "bar-" + index);
@@ -140,12 +130,13 @@ public class GraphRepoMapperScaleTest {
     }
 
     private void searchWithOutIndex(String key, String value) {
-        TitanGraph graph = graphProvider.get();
+        AtlasGraph graph = TestUtils.getGraph();
         long start = System.currentTimeMillis();
         int count = 0;
         try {
-            GraphQuery query = graph.query().has(key, Compare.EQUAL, value);
-            for (Vertex ignored : query.vertices()) {
+            AtlasGraphQuery query = graph.query().has(key, ComparisionOperator.EQUAL, value);
+            Iterable<AtlasVertex> result = query.vertices();
+            for (AtlasVertex ignored : result) {
                 count++;
             }
         } finally {
@@ -154,29 +145,33 @@ public class GraphRepoMapperScaleTest {
         }
     }
 
+
     private void searchWithIndex(String key, String value) {
-        TitanGraph graph = graphProvider.get();
+        AtlasGraph graph = TestUtils.getGraph();
         long start = System.currentTimeMillis();
         int count = 0;
         try {
             String queryString = "v.\"" + key + "\":(" + value + ")";
-            TitanIndexQuery query = graph.indexQuery(Constants.VERTEX_INDEX, queryString);
-            for (TitanIndexQuery.Result<Vertex> ignored : query.vertices()) {
+            AtlasIndexQuery query = graph.indexQuery(Constants.VERTEX_INDEX, queryString);
+            Iterator<AtlasIndexQuery.Result> result = query.vertices();
+            while(result.hasNext()) {
+                result.next();
                 count++;
             }
         } finally {
             System.out.println("Search on [" + key + "=" + value + "] returned results: " + count + ", took " + (
-                System.currentTimeMillis() - start) + " ms");
+                    System.currentTimeMillis() - start) + " ms");
         }
     }
-
-    private void  searchWithIndex(String key, Predicate searchPredicate, Object value, int expectedResults) {
-        TitanGraph graph = graphProvider.get();
+    
+    private void searchWithIndex(String key, ComparisionOperator op, Object value, int expectedResults) {
+        AtlasGraph graph = TestUtils.getGraph();
         long start = System.currentTimeMillis();
         int count = 0;
         try {
-            GraphQuery query = graph.query().has(key, searchPredicate, value);
-            for (Vertex ignored : query.vertices()) {
+            AtlasGraphQuery query = graph.query().has(key, op, value);
+            Iterable<AtlasVertex> itrble = query.vertices();
+            for (AtlasVertex ignored : itrble) {
                 count++;
             }
         } finally {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java b/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
index 90e622a..000f2f4 100755
--- a/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/typestore/GraphBackedTypeStoreTest.java
@@ -18,19 +18,26 @@
 
 package org.apache.atlas.repository.typestore;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
-import com.tinkerpop.blueprints.Direction;
-import com.tinkerpop.blueprints.Edge;
-import com.tinkerpop.blueprints.Vertex;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createStructTypeDef;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
 
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
+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.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.typesystem.TypesDef;
 import org.apache.atlas.typesystem.types.AttributeDefinition;
 import org.apache.atlas.typesystem.types.ClassType;
@@ -53,25 +60,15 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import javax.inject.Inject;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createStructTypeDef;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 @Guice(modules = RepositoryMetadataModule.class)
 public class GraphBackedTypeStoreTest {
+    
     private static final String DESCRIPTION = "_description";
 
     @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
-    @Inject
     private ITypeStore typeStore;
 
     private TypeSystem ts;
@@ -86,18 +83,10 @@ public class GraphBackedTypeStoreTest {
     @AfterClass
     public void tearDown() throws Exception {
         ts.reset();
-        try {
-            graphProvider.get().shutdown();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        } catch(Exception e) {
-            e.printStackTrace();
-        }
+        AtlasGraphProvider.cleanup();
     }
 
+
     @Test
     public void testStore() throws AtlasException {
         ImmutableList<String> typeNames = ts.getTypeNames();
@@ -112,10 +101,10 @@ public class GraphBackedTypeStoreTest {
     }
 
     private void dumpGraph() {
-        TitanGraph graph = graphProvider.get();
-        for (Vertex v : graph.getVertices()) {
+        AtlasGraph<?, ?> graph = TestUtils.getGraph();
+        for (AtlasVertex<?,?> v : graph.getVertices()) {
             System.out.println("****v = " + GraphHelper.vertexString(v));
-            for (Edge e : v.getEdges(Direction.OUT)) {
+            for (AtlasEdge<?,?> e : v.getEdges(AtlasEdgeDirection.OUT)) {
                 System.out.println("****e = " + GraphHelper.edgeString(e));
             }
         }
@@ -215,17 +204,17 @@ public class GraphBackedTypeStoreTest {
         // ATLAS-474: verify that type update did not write duplicate edges to the type store.
         if (typeStore instanceof GraphBackedTypeStore) {
             GraphBackedTypeStore gbTypeStore = (GraphBackedTypeStore) typeStore;
-            Vertex typeVertex = gbTypeStore.findVertex(TypeCategory.CLASS, "Department");
+            AtlasVertex typeVertex = gbTypeStore.findVertex(TypeCategory.CLASS, "Department");
             int edgeCount = countOutgoingEdges(typeVertex, gbTypeStore.getEdgeLabel("Department", "employees"));
-            Assert.assertEquals(edgeCount, 1, "Should only be 1 edge for employees attribute on Department type vertex");
+            Assert.assertEquals(edgeCount, 1, "Should only be 1 edge for employees attribute on Department type AtlasVertex");
         }
     }
 
-    private int countOutgoingEdges(Vertex typeVertex, String edgeLabel) {
+    private int countOutgoingEdges(AtlasVertex typeVertex, String edgeLabel) {
 
-        Iterator<Edge> outGoingEdgesByLabel = GraphHelper.getInstance().getOutGoingEdgesByLabel(typeVertex, edgeLabel);
+        Iterator<AtlasEdge> outGoingEdgesByLabel = GraphHelper.getInstance().getOutGoingEdgesByLabel(typeVertex, edgeLabel);
         int edgeCount = 0;
-        for (Iterator<Edge> iterator = outGoingEdgesByLabel; iterator.hasNext();) {
+        for (Iterator<AtlasEdge> iterator = outGoingEdgesByLabel; iterator.hasNext();) {
             iterator.next();
             edgeCount++;
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTest.java b/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTest.java
index b7cf7e9..6c6c959 100644
--- a/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTest.java
+++ b/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTest.java
@@ -17,14 +17,15 @@
  */
 package org.apache.atlas.repository.typestore;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.inject.Inject;
+
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
 import org.apache.atlas.TestUtils;
-import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.typesystem.types.AttributeInfo;
 import org.apache.atlas.typesystem.types.ClassType;
 import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
@@ -40,9 +41,8 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import javax.inject.Inject;
-import java.util.HashMap;
-import java.util.Map;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 
 /**
@@ -52,9 +52,6 @@ import java.util.Map;
 public class StoreBackedTypeCacheTest {
 
     @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
-    @Inject
     private ITypeStore typeStore;
 
     @Inject
@@ -64,8 +61,15 @@ public class StoreBackedTypeCacheTest {
 
     private Map<String, ClassType> classTypesToTest = new HashMap<>();
 
+    @Inject
+    public StoreBackedTypeCacheTest() {
+    }
+
     @BeforeClass
     public void setUp() throws Exception {
+        //force graph to be initialized up front
+        TestUtils.getGraph();
+
         ts = TypeSystem.getInstance();
         ts.reset();
         ts.setTypeCache(typeCache);
@@ -75,7 +79,7 @@ public class StoreBackedTypeCacheTest {
         TestUtils.createHiveTypes(ts);
         ImmutableList<String> typeNames = ts.getTypeNames();
         typeStore.store(ts, typeNames);
-
+        
         ClassType type = ts.getDataType(ClassType.class, "Manager");
         classTypesToTest.put("Manager", type);
         type = ts.getDataType(ClassType.class, TestUtils.TABLE_TYPE);
@@ -85,24 +89,12 @@ public class StoreBackedTypeCacheTest {
     @AfterClass
     public void tearDown() throws Exception {
         ts.reset();
-        try {
-            graphProvider.get().shutdown();
-        }
-        catch(Exception e) {
-            e.printStackTrace();
-        }
-
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        }
-        catch(Exception e) {
-            e.printStackTrace();
-        }
+        AtlasGraphProvider.cleanup();
     }
 
     @BeforeMethod
     public void setupTestMethod() throws Exception {
-        ts.reset();
+        typeCache.clear();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTestModule.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTestModule.java b/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTestModule.java
index 058ed4d..a1d7a74 100644
--- a/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTestModule.java
+++ b/repository/src/test/java/org/apache/atlas/repository/typestore/StoreBackedTypeCacheTestModule.java
@@ -20,6 +20,7 @@ package org.apache.atlas.repository.typestore;
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.RepositoryMetadataModule;
+import org.apache.atlas.util.AtlasRepositoryConfiguration;
 import org.apache.commons.configuration.Configuration;
 
 
@@ -32,7 +33,7 @@ public class StoreBackedTypeCacheTestModule extends RepositoryMetadataModule {
     protected Configuration getConfiguration() {
         try {
             Configuration configuration = ApplicationProperties.get();
-            configuration.setProperty(RepositoryMetadataModule.TYPE_CACHE_IMPLEMENTATION_PROPERTY,
+            configuration.setProperty(AtlasRepositoryConfiguration.TYPE_CACHE_IMPLEMENTATION_PROPERTY,
                     StoreBackedTypeCache.class.getName());
             return configuration;
         } catch (AtlasException e) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/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 e713ba5..96cefe2 100644
--- a/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/service/DefaultMetadataServiceTest.java
@@ -18,11 +18,30 @@
 
 package org.apache.atlas.service;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.inject.Inject;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
+import static org.apache.atlas.TestUtils.COLUMNS_ATTR_NAME;
+import static org.apache.atlas.TestUtils.COLUMN_TYPE;
+import static org.apache.atlas.TestUtils.PII;
+import static org.apache.atlas.TestUtils.TABLE_TYPE;
+import static org.apache.atlas.TestUtils.createColumnEntity;
+import static org.apache.atlas.TestUtils.createDBEntity;
+import static org.apache.atlas.TestUtils.createInstance;
+import static org.apache.atlas.TestUtils.createTableEntity;
+import static org.apache.atlas.TestUtils.randomString;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
+import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.EntityAuditEvent;
@@ -35,11 +54,7 @@ import org.apache.atlas.query.QueryParams;
 import org.apache.atlas.repository.audit.EntityAuditRepository;
 import org.apache.atlas.repository.audit.HBaseBasedAuditRepository;
 import org.apache.atlas.repository.audit.HBaseTestUtils;
-import org.apache.atlas.repository.graph.GraphProvider;
-import org.apache.atlas.services.AtlasTypeAttributePatch;
-import org.apache.atlas.services.AtlasTypePatch;
-import org.apache.atlas.services.AtlasTypePatch.PatchData;
-import org.apache.atlas.services.DefaultMetadataService;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.services.MetadataService;
 import org.apache.atlas.typesystem.IReferenceableInstance;
 import org.apache.atlas.typesystem.IStruct;
@@ -72,29 +87,9 @@ import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.atlas.TestUtils.COLUMNS_ATTR_NAME;
-import static org.apache.atlas.TestUtils.COLUMN_TYPE;
-import static org.apache.atlas.TestUtils.PII;
-import static org.apache.atlas.TestUtils.TABLE_TYPE;
-import static org.apache.atlas.TestUtils.createColumnEntity;
-import static org.apache.atlas.TestUtils.createDBEntity;
-import static org.apache.atlas.TestUtils.createInstance;
-import static org.apache.atlas.TestUtils.createTableEntity;
-import static org.apache.atlas.TestUtils.randomString;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
-import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNotNull;
-import static org.testng.Assert.assertNull;
-import static org.testng.Assert.assertTrue;
-import static org.testng.Assert.fail;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.inject.Inject;
 
 @Guice(modules = RepositoryMetadataModule.class)
 public class DefaultMetadataServiceTest {
@@ -102,9 +97,6 @@ public class DefaultMetadataServiceTest {
     private MetadataService metadataService;
 
     @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
-    @Inject
     private EntityAuditRepository auditRepository;
 
     @Inject
@@ -115,6 +107,7 @@ public class DefaultMetadataServiceTest {
     private Referenceable table;
 
     private Id tableId;
+    
     private final String NAME = "name";
 
 
@@ -124,7 +117,7 @@ public class DefaultMetadataServiceTest {
             HBaseTestUtils.startCluster();
             ((HBaseBasedAuditRepository) auditRepository).start();
         }
-        RequestContext.createContext();
+        TestUtils.resetRequestContext();
         RequestContext.get().setUser("testuser");
 
         TypesDef typesDef = TestUtils.defineHiveTypes();
@@ -145,25 +138,18 @@ public class DefaultMetadataServiceTest {
 
     @AfterTest
     public void shutdown() throws Exception {
-        TypeSystem.getInstance().reset();
         try {
-            //TODO - Fix failure during shutdown while using BDB
-            graphProvider.get().shutdown();
-        } catch(Exception e) {
-            e.printStackTrace();
-        }
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        } catch(Exception e) {
-            e.printStackTrace();
-        }
+            TypeSystem.getInstance().reset();
 
-        if (auditRepository instanceof HBaseBasedAuditRepository) {
-            ((HBaseBasedAuditRepository) auditRepository).stop();
-            HBaseTestUtils.stopCluster();
+            if (auditRepository instanceof HBaseBasedAuditRepository) {
+                ((HBaseBasedAuditRepository) auditRepository).stop();
+                HBaseTestUtils.stopCluster();
+            }
+        }
+        finally {
+            AtlasGraphProvider.cleanup();
         }
     }
-
     private AtlasClient.EntityResult updateInstance(Referenceable entity) throws Exception {
         RequestContext.createContext();
         ParamChecker.notNull(entity, "Entity");
@@ -773,7 +759,7 @@ public class DefaultMetadataServiceTest {
     @Test
     public void testArrayOfStructs() throws Exception {
         //Add array of structs
-        TestUtils.dumpGraph(graphProvider.get());
+        TestUtils.dumpGraph(TestUtils.getGraph());
 
         final Struct partition1 = new Struct(TestUtils.PARTITION_STRUCT_TYPE);
         partition1.set(NAME, "part1");
@@ -1095,43 +1081,6 @@ public class DefaultMetadataServiceTest {
     }
 
     @Test
-    public void testPatchFrameworkForTypeUpdate() throws AtlasException, JSONException {
-        String typeName = "test_type_" + RandomStringUtils.randomAlphanumeric(10);
-        HierarchicalTypeDefinition<ClassType> typeDef = TypesUtil.createClassTypeDef(typeName, ImmutableSet.<String>of(),
-                TypesUtil.createUniqueRequiredAttrDef("type_attr1", DataTypes.STRING_TYPE));
-
-        TypesDef typesDef = new TypesDef(typeDef, false);
-        metadataService.createType(TypesSerialization.toJson(typesDef));
-
-        AtlasTypeAttributePatch patch = new AtlasTypeAttributePatch((DefaultMetadataService) metadataService, TypeSystem.getInstance());
-        AttributeDefinition[] attrDefs = new AttributeDefinition[]{
-                new AttributeDefinition("type_attr2", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null),
-                new AttributeDefinition("type_attr3", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null)};
-
-        // Testing add attribute patch
-        AtlasTypePatch.PatchData addAttributePatch = new PatchData("ADD_ATTRIBUTE", typeName, "1.0", "2.0", null, attrDefs);
-        TypesDef newAttrTypesDef = patch.updateTypesDef(typesDef, addAttributePatch);
-        metadataService.updateType(TypesSerialization.toJson(newAttrTypesDef));
-        TypesDef addedTypesDef = TypesSerialization.fromJson(metadataService.getTypeDefinition(typeName));
-
-        // test added attributes and update version to 2.0
-        assertEquals(addedTypesDef.classTypes().head().attributeDefinitions.length, 3);
-        assertEquals(addedTypesDef.classTypes().head().typeVersion, "2.0");
-
-        // Testing update attribute patch
-        AttributeDefinition[] updateAttrDef = new AttributeDefinition[]{
-                new AttributeDefinition("type_attr1", DataTypes.STRING_TYPE.getName(), Multiplicity.OPTIONAL, false, null)};
-        AtlasTypePatch.PatchData updateAttributePatch = new PatchData("UPDATE_ATTRIBUTE", typeName, "2.0", "3.0", null, updateAttrDef);
-        TypesDef updateAttrTypesDef = patch.updateTypesDef(addedTypesDef, updateAttributePatch);
-        metadataService.updateType(TypesSerialization.toJson(updateAttrTypesDef));
-        TypesDef updatedTypesDef = TypesSerialization.fromJson(metadataService.getTypeDefinition(typeName));
-
-        // test update attribute to optional and update version to 3.0
-        assertEquals(updatedTypesDef.classTypes().head().attributeDefinitions[0].multiplicity, Multiplicity.OPTIONAL);
-        assertEquals(updatedTypesDef.classTypes().head().typeVersion, "3.0");
-    }
-
-    @Test
     public void testAuditEventsInvalidParams() throws Exception {
         //entity id can't be null
         try {
@@ -1218,7 +1167,7 @@ public class DefaultMetadataServiceTest {
                 deletedEntities.add(entity.getId()._getId());
             }
         }
-
+        
         public List<String> getDeletedEntities() {
             return deletedEntities;
         }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/service/StoreBackedTypeCacheMetadataServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/service/StoreBackedTypeCacheMetadataServiceTest.java b/repository/src/test/java/org/apache/atlas/service/StoreBackedTypeCacheMetadataServiceTest.java
index 8fb59c5..b73cc86 100644
--- a/repository/src/test/java/org/apache/atlas/service/StoreBackedTypeCacheMetadataServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/service/StoreBackedTypeCacheMetadataServiceTest.java
@@ -18,7 +18,7 @@
 package org.apache.atlas.service;
 
 import org.apache.atlas.TestUtils;
-import org.apache.atlas.repository.graph.GraphProvider;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.repository.typestore.ITypeStore;
 import org.apache.atlas.repository.typestore.StoreBackedTypeCache;
 import org.apache.atlas.repository.typestore.StoreBackedTypeCacheTestModule;
@@ -44,8 +44,6 @@ import org.testng.annotations.Test;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.inject.Inject;
-import com.thinkaurelius.titan.core.TitanGraph;
-import com.thinkaurelius.titan.core.util.TitanCleanup;
 
 
 /**
@@ -67,9 +65,6 @@ public class StoreBackedTypeCacheMetadataServiceTest
 
     private StoreBackedTypeCache storeBackedTypeCache;
 
-    @Inject
-    private GraphProvider<TitanGraph> graphProvider;
-
     private TypeSystem ts;
 
     @BeforeClass
@@ -94,22 +89,10 @@ public class StoreBackedTypeCacheMetadataServiceTest
 
     @AfterClass
     public void tearDown() throws Exception {
-        ts.reset();
-        try {
-            graphProvider.get().shutdown();
-        }
-        catch(Exception e) {
-            e.printStackTrace();
-        }
-
-        try {
-            TitanCleanup.clear(graphProvider.get());
-        }
-        catch(Exception e) {
-            e.printStackTrace();
-        }
+        TypeSystem.getInstance().reset();
+        AtlasGraphProvider.cleanup();
     }
-
+    
     @Test
     public void testGetTypeDefinition() throws Exception {
         // Cache should be empty

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/java/org/apache/atlas/utils/HiveModel.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/utils/HiveModel.java b/repository/src/test/java/org/apache/atlas/utils/HiveModel.java
new file mode 100644
index 0000000..dbd4f42
--- /dev/null
+++ b/repository/src/test/java/org/apache/atlas/utils/HiveModel.java
@@ -0,0 +1,303 @@
+/**
+ * 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.utils;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.atlas.TestUtils;
+import org.apache.atlas.typesystem.ITypedReferenceableInstance;
+import org.apache.atlas.typesystem.Referenceable;
+import org.apache.atlas.typesystem.Struct;
+import org.apache.atlas.typesystem.persistence.Id;
+import org.apache.atlas.typesystem.types.ClassType;
+import org.apache.atlas.typesystem.types.Multiplicity;
+import org.apache.atlas.typesystem.types.TypeSystem;
+
+/**
+ * Allows easy creation of entities for classes in the hive test model.
+ *
+ */
+public class HiveModel {
+
+    public static class StructInstance {
+
+        public String getTypeName() {
+            return getClass().getSimpleName();
+        }
+
+        public Struct toStruct() throws Exception {
+
+            Struct entity = new Struct(getTypeName());
+            addDeclaredFields(getClass(), entity);
+            return entity;
+        }
+
+        protected void addDeclaredFields(Class clazz, Struct r) throws Exception {
+            
+            for (Field f : clazz.getDeclaredFields()) {
+                
+                if (Modifier.isTransient(f.getModifiers())) {
+                    continue;
+                }
+                String fieldName = f.getName();
+
+                f.setAccessible(true);
+                Object value = f.get(this);
+                
+                if (value instanceof List) {
+                    
+                    List listValue = (List) value;
+                    List toSet = new ArrayList(listValue.size());
+                    for (Object listItem : listValue) {
+                        Object toAdd = null;
+                        toAdd = convertValue(listItem);
+                        toSet.add(toAdd);
+                    }
+                    r.set(fieldName, toSet);
+                } else {
+                    
+                    Object converted = convertValue(value);
+                    r.set(fieldName, converted);
+                }
+            }
+            
+            if (clazz != StructInstance.class) {
+                addDeclaredFields(clazz.getSuperclass(), r);
+            }
+        }
+
+        private Object convertValue(Object toConvert) throws Exception {
+
+            if (toConvert instanceof ClassInstance) {
+                return ((ClassInstance) toConvert).toReferenceable();
+            }
+            if (toConvert instanceof StructInstance) {
+                return ((StructInstance) toConvert).toStruct();
+            } else {
+                return toConvert;
+            }
+        }
+    }
+
+    public static class ClassInstance<T> extends StructInstance {
+
+        private transient final Id guid;
+        private transient List<String> traits = new ArrayList();
+
+        public T withTrait(String name) {
+            traits.add(name);
+            return getInstance();
+        }
+
+        public T withTraits(List<String> names) {
+            traits.addAll(names);
+            return getInstance();
+        }
+
+        public T getInstance() {
+            return (T) this;
+        }
+
+        public ClassInstance() {
+            guid = new Id(getTypeName());
+        }
+
+        public Referenceable toReferenceable() throws Exception {
+
+            String[] traitArray = new String[traits.size()];
+            traitArray = traits.toArray(traitArray);
+            Referenceable entity = new Referenceable(getTypeName(), traitArray);
+            entity.replaceWithNewId(guid);
+            addDeclaredFields(getClass(), entity);
+
+            return entity;
+        }
+
+        public List<ITypedReferenceableInstance> getTypedReferencebles() throws Exception {
+
+            List<ITypedReferenceableInstance> result = new ArrayList();
+            for (ClassInstance containedInstance : getAllInstances()) {
+                Referenceable entity = containedInstance.toReferenceable();
+                ClassType type = TypeSystem.getInstance().getDataType(ClassType.class, entity.getTypeName());
+                ITypedReferenceableInstance converted = type.convert(entity, Multiplicity.REQUIRED);
+                result.add(converted);
+            }
+            return result;
+        }
+
+        protected List<ClassInstance> getAllInstances() {
+
+            return (List) Collections.singletonList(this);
+        }
+
+        public Id getId() {
+            return guid;
+        }
+    }
+
+    public static class NamedInstance<T> extends ClassInstance<T> {
+        
+        private final String name;
+
+        public NamedInstance(String name) {
+            super();
+            this.name = name;
+        }
+    }
+
+    public static class HiveOrder extends StructInstance {
+        
+        private String col;
+        private int order;
+
+        public HiveOrder(String col, int order) {
+            super();
+            this.col = col;
+            this.order = order;
+        }
+
+    }
+
+    public static class DB extends NamedInstance<DB> {
+        
+        private String owner;
+        private int createTime;
+        private String clusterName;
+
+        public DB(String name, String owner, int createTime, String clusterName) {
+            super(name);
+            this.owner = owner;
+            this.createTime = createTime;
+            this.clusterName = clusterName;
+        }
+    }
+
+    public static class StorageDescriptor extends ClassInstance<StorageDescriptor> {
+        
+        private String inputFormat;
+        private String outputFormat;
+        private List<HiveOrder> sortCols;
+
+        public StorageDescriptor(String inputFormat, String outputFormat, List<HiveOrder> sortCols) {
+            super();
+            this.inputFormat = inputFormat;
+            this.outputFormat = outputFormat;
+            this.sortCols = sortCols;
+        }
+    }
+
+    public static class Column extends NamedInstance<Column> {
+
+        private String type;
+        private StorageDescriptor sd;
+
+        public Column(String name, String type) {
+            super(name);
+            this.type = type;
+        }
+
+        public void setStorageDescriptor(StorageDescriptor sd) {
+            this.sd = sd;
+        }
+    }
+
+    public static class Table extends NamedInstance<Table> {
+        
+        private DB db;
+        private Date created;
+        private StorageDescriptor sd;
+        private transient List<Column> colDefs;
+
+        public Table(String name, DB db, StorageDescriptor sd, List<Column> colDefs) {
+            this(name, db, sd, new Date(TestUtils.TEST_DATE_IN_LONG), colDefs);
+        }
+
+        public Table(String name, DB db, StorageDescriptor sd, Date created, List<Column> colDefs) {
+            
+            super(name);
+            this.colDefs = colDefs;
+            this.db = db;
+            this.sd = sd;
+            this.created = created;
+            for (Column col : colDefs) {
+                col.setStorageDescriptor(sd);
+            }
+        }
+
+        public List<Column> getColumns() {
+            return colDefs;
+        }
+
+        @Override
+        protected List<ClassInstance> getAllInstances() {
+            
+            List<ClassInstance> result = new ArrayList(colDefs.size() + 2);
+            result.add(sd);
+            result.addAll(colDefs);
+            result.add(this);
+            return result;
+        }
+    }
+
+    public static class Partition extends ClassInstance<Partition> {
+        
+        private List<String> values;
+        private Table table;
+
+        public Partition(List<String> values, Table table) {
+            
+            super();
+            this.values = values;
+            this.table = table;
+        }
+
+    }
+
+    public static class LoadProcess extends NamedInstance<LoadProcess> {
+
+        private List<Table> inputTables;
+        private Table outputTable;
+
+        public LoadProcess(String name, List<Table> inputTables, Table outputTable) {
+            super(name);
+            this.inputTables = inputTables;
+            this.outputTable = outputTable;
+        }
+
+    }
+
+    public static class View extends NamedInstance<View> {
+
+        private DB db;
+        private List<Table> inputTables;
+
+        public View(String name, DB db, List<Table> inputTables) {
+            super(name);
+            this.db = db;
+            this.inputTables = inputTables;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/d2d6ff7d/repository/src/test/scala/org/apache/atlas/query/GremlinTest.scala
----------------------------------------------------------------------
diff --git a/repository/src/test/scala/org/apache/atlas/query/GremlinTest.scala b/repository/src/test/scala/org/apache/atlas/query/GremlinTest.scala
index fa48c0e..22717ff 100755
--- a/repository/src/test/scala/org/apache/atlas/query/GremlinTest.scala
+++ b/repository/src/test/scala/org/apache/atlas/query/GremlinTest.scala
@@ -18,45 +18,50 @@
 
 package org.apache.atlas.query
 
-import com.thinkaurelius.titan.core.TitanGraph
-import com.thinkaurelius.titan.core.util.TitanCleanup
+import org.apache.atlas.repository.graphdb.AtlasGraph
 import org.apache.atlas.discovery.graph.DefaultGraphPersistenceStrategy
 import org.apache.atlas.query.Expressions._
-import org.apache.atlas.repository.graph.{TitanGraphProvider, GraphBackedMetadataRepository}
+import org.apache.atlas.repository.graph.{AtlasGraphProvider, GraphBackedMetadataRepository}
 import org.apache.atlas.typesystem.types.TypeSystem
 import org.testng.annotations.{Test,BeforeClass,AfterClass}
+import org.apache.atlas.repository.graph.AtlasGraphProvider
+import org.testng.annotations.BeforeMethod
+import org.apache.atlas.TestUtils
 
 class GremlinTest extends BaseGremlinTest {
 
-  var g: TitanGraph = null
+  var g: AtlasGraph[_,_] = null
   var gp: GraphPersistenceStrategies = null;
-  var gProvider: TitanGraphProvider = null;
+
+  @BeforeMethod
+  def resetRequestContext() {
+    TestUtils.resetRequestContext()
+  }
 
   @BeforeClass
   def beforeAll() {
-    TypeSystem.getInstance().reset()
-    QueryTestsUtils.setupTypes
-    gProvider = new TitanGraphProvider()
-    gp = new DefaultGraphPersistenceStrategy(new GraphBackedMetadataRepository(gProvider, null))
-    g = QueryTestsUtils.setupTestGraph(gProvider)
-  }
+     TypeSystem.getInstance().reset()
+     var repo = new GraphBackedMetadataRepository(null);
+     TestUtils.setupGraphProvider(repo);
+    //force graph to be initialized first
+    AtlasGraphProvider.getGraphInstance();
+    
+    //create types and indices up front.  Without this, some of the property keys (particularly __traitNames and __superTypes)
+    //get ended up created implicitly with some graph backends with the wrong multiplicity.  This also makes the queries
+    //we execute perform better :-)    
+    QueryTestsUtils.setupTypesAndIndices()    
+
+    gp = new DefaultGraphPersistenceStrategy(repo)
+    g = QueryTestsUtils.setupTestGraph(repo)    
+    g
+  }  
 
   @AfterClass
   def afterAll() {
-    try {
-        g.shutdown()
-    } catch {
-      case ex: Exception =>
-        print("Could not shutdown the graph ", ex);
-    }
-    try {
-      TitanCleanup.clear(g);
-    } catch {
-      case ex: Exception =>
-        print("Could not clear the graph ", ex);
-    }
+    AtlasGraphProvider.cleanup();
   }
 
+
   @Test def testClass {
     val r = QueryProcessor.evaluate(_class("DB"), g, gp)
     validateJson(r, """{