You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by sa...@apache.org on 2017/10/27 21:16:28 UTC

[4/4] atlas git commit: ATLAS-1757: Introduce JanusGraph 0.1.1 graph store in atlas

ATLAS-1757: Introduce JanusGraph 0.1.1 graph store in atlas

Signed-off-by: Sarath Subramanian <ss...@hortonworks.com>


Project: http://git-wip-us.apache.org/repos/asf/atlas/repo
Commit: http://git-wip-us.apache.org/repos/asf/atlas/commit/503dddb8
Tree: http://git-wip-us.apache.org/repos/asf/atlas/tree/503dddb8
Diff: http://git-wip-us.apache.org/repos/asf/atlas/diff/503dddb8

Branch: refs/heads/master
Commit: 503dddb8c820ae0dba62d200052262144409d3cf
Parents: 5d1868b
Author: Graham Wallis <gr...@uk.ibm.com>
Authored: Fri Oct 27 14:15:44 2017 -0700
Committer: Sarath Subramanian <ss...@hortonworks.com>
Committed: Fri Oct 27 14:15:44 2017 -0700

----------------------------------------------------------------------
 distro/pom.xml                                  |  39 +-
 distro/src/conf/atlas-application.properties    |   8 +-
 docs/src/site/twiki/Configuration.twiki         |   2 +-
 graphdb/api/pom.xml                             |   2 +-
 .../query/NativeTinkerpopGraphQuery.java        |  81 +++
 .../query/NativeTinkerpopQueryFactory.java      |  34 ++
 .../tinkerpop/query/TinkerpopGraphQuery.java    | 249 +++++++++
 .../tinkerpop/query/expr/AndCondition.java      |  96 ++++
 .../tinkerpop/query/expr/HasPredicate.java      |  48 ++
 .../tinkerpop/query/expr/InPredicate.java       |  49 ++
 .../tinkerpop/query/expr/OrCondition.java       | 136 +++++
 .../tinkerpop/query/expr/QueryPredicate.java    |  33 ++
 .../titan/query/NativeTitanGraphQuery.java      |  81 ---
 .../titan/query/NativeTitanQueryFactory.java    |  34 --
 .../graphdb/titan/query/TitanGraphQuery.java    | 251 ---------
 .../graphdb/titan/query/expr/AndCondition.java  |  96 ----
 .../graphdb/titan/query/expr/HasPredicate.java  |  48 --
 .../graphdb/titan/query/expr/InPredicate.java   |  49 --
 .../graphdb/titan/query/expr/OrCondition.java   | 136 -----
 .../titan/query/expr/QueryPredicate.java        |  33 --
 graphdb/graphdb-impls/pom.xml                   |  23 +-
 graphdb/janus/pom.xml                           | 240 +++++++++
 graphdb/janus/readme.txt                        |  52 ++
 .../graphdb/janus/AtlasJanusEdge.java           |  55 ++
 .../graphdb/janus/AtlasJanusEdgeLabel.java      |  59 +++
 .../graphdb/janus/AtlasJanusElement.java        | 256 +++++++++
 .../graphdb/janus/AtlasJanusGraph.java          | 430 ++++++++++++++++
 .../graphdb/janus/AtlasJanusGraphDatabase.java  | 176 +++++++
 .../graphdb/janus/AtlasJanusGraphIndex.java     | 100 ++++
 .../janus/AtlasJanusGraphManagement.java        | 205 ++++++++
 .../graphdb/janus/AtlasJanusIndexQuery.java     | 101 ++++
 .../graphdb/janus/AtlasJanusObjectFactory.java  |  85 +++
 .../graphdb/janus/AtlasJanusPropertyKey.java    |  77 +++
 .../graphdb/janus/AtlasJanusVertex.java         | 102 ++++
 .../graphdb/janus/AtlasJanusVertexQuery.java    |  80 +++
 .../graphdb/janus/GraphDbObjectFactory.java     | 130 +++++
 .../graphson/AtlasElementPropertyConfig.java    | 136 +++++
 .../janus/graphson/AtlasGraphSONMode.java       |  43 ++
 .../janus/graphson/AtlasGraphSONTokens.java     |  51 ++
 .../janus/graphson/AtlasGraphSONUtility.java    | 513 +++++++++++++++++++
 .../janus/query/AtlasJanusGraphQuery.java       |  56 ++
 .../janus/query/NativeJanusGraphQuery.java      | 141 +++++
 .../janus/serializer/BigDecimalSerializer.java  |  49 ++
 .../janus/serializer/BigIntegerSerializer.java  |  46 ++
 .../janus/serializer/StringListSerializer.java  |  54 ++
 .../serializer/TypeCategorySerializer.java      |  31 ++
 .../services/javax.script.ScriptEngineFactory   |   1 +
 .../janus/AbstractGraphDatabaseTest.java        | 189 +++++++
 .../graphdb/janus/AtlasJanusDatabaseTest.java   | 431 ++++++++++++++++
 .../graphdb/janus/GraphQueryTest.java           | 451 ++++++++++++++++
 .../graphdb/janus/JanusGraphProviderTest.java   |  80 +++
 .../test/resources/atlas-application.properties |  98 ++++
 graphdb/pom.xml                                 |   6 +-
 graphdb/readme.txt                              |  31 ++
 graphdb/titan0/pom.xml                          | 140 +++--
 .../titan0/query/NativeTitan0GraphQuery.java    |   4 +-
 .../graphdb/titan0/query/Titan0GraphQuery.java  |  14 +-
 .../test/resources/atlas-application.properties |   8 +-
 graphdb/titan1/pom.xml                          | 190 +++----
 .../titan1/query/NativeTitan1GraphQuery.java    |   6 +-
 .../graphdb/titan1/query/Titan1GraphQuery.java  |  14 +-
 .../test/resources/atlas-application.properties |   8 +-
 pom.xml                                         | 206 ++++----
 repository/pom.xml                              |  27 +-
 .../atlas/discovery/EntityDiscoveryService.java |  23 +-
 .../gremlin/Gremlin3ExpressionFactory.java      |   8 +-
 .../atlas/repository/graph/GraphHelper.java     |   8 +-
 .../graph/GraphToTypedInstanceMapper.java       |   7 +-
 .../atlas/repository/impexp/ExportService.java  |  20 +-
 .../graph/v1/AtlasRelationshipStoreV1.java      |   6 +-
 .../store/graph/v1/EntityGraphMapper.java       |   5 +-
 .../store/graph/v1/EntityGraphRetriever.java    |   4 +-
 .../atlas/util/AtlasGremlin2QueryProvider.java  |   4 +
 .../atlas/util/AtlasGremlin3QueryProvider.java  |  28 +
 .../atlas/util/AtlasGremlinQueryProvider.java   |   4 +
 .../GraphBackedDiscoveryServiceTest.java        |  38 +-
 .../GraphBackedMetadataRepositoryTest.java      |   5 +-
 .../impexp/ZipFileResourceTestUtils.java        |   2 +-
 .../graph/v1/AtlasRelationshipStoreV1Test.java  |   3 +-
 .../test/resources/atlas-application.properties |   6 +-
 webapp/pom.xml                                  |  11 +-
 .../NotificationHookConsumerTest.java           |   1 +
 82 files changed, 5715 insertions(+), 1168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/distro/pom.xml
----------------------------------------------------------------------
diff --git a/distro/pom.xml b/distro/pom.xml
index 9bea008..9db742d 100644
--- a/distro/pom.xml
+++ b/distro/pom.xml
@@ -32,16 +32,16 @@
     <!-- by default configure hbase and solr with the distribution -->
     <properties>
 
-        <titan.storage.backend>hbase</titan.storage.backend>
-        <titan.storage.properties>#Hbase
+        <graph.storage.backend>hbase</graph.storage.backend>
+        <graph.storage.properties>#Hbase
 #For standalone mode , specify localhost
 #for distributed mode, specify zookeeper quorum here - For more information refer http://s3.thinkaurelius.com/docs/titan/current/hbase.html#_remote_server_mode_2
 atlas.graph.storage.hostname=
 atlas.graph.storage.hbase.regions-per-server=1
 atlas.graph.storage.lock.wait-time=10000
-        </titan.storage.properties>
-        <titan.index.backend>solr5</titan.index.backend>
-        <titan.index.properties>#Solr
+        </graph.storage.properties>
+        <graph.index.backend>solr5</graph.index.backend>
+        <graph.index.properties>#Solr
 #Solr cloud mode properties
 atlas.graph.index.search.solr.mode=cloud
 atlas.graph.index.search.solr.zookeeper-url=
@@ -51,7 +51,7 @@ atlas.graph.index.search.solr.zookeeper-session-timeout=60000
 #Solr http mode properties
 #atlas.graph.index.search.solr.mode=http
 #atlas.graph.index.search.solr.http-urls=http://localhost:8983/solr
-        </titan.index.properties>
+        </graph.index.properties>
         <hbase.embedded>false</hbase.embedded>
         <solr.embedded>false</solr.embedded>
 	<entity.repository.properties>atlas.EntityAuditRepository.impl=org.apache.atlas.repository.audit.HBaseBasedAuditRepository</entity.repository.properties>
@@ -116,6 +116,7 @@ atlas.graph.index.search.solr.zookeeper-session-timeout=60000
                 </plugins>
             </build>
         </profile>
+
         <!-- profile to configure berkeley and elasticsearch with the distribution -->
         <profile>
             <id>berkeley-elasticsearch</id>
@@ -123,25 +124,25 @@ atlas.graph.index.search.solr.zookeeper-session-timeout=60000
                 <activeByDefault>false</activeByDefault>
             </activation>
             <properties>
-                <titan.storage.backend>berkeleyje</titan.storage.backend>
-                <titan.storage.properties>#Berkeley
-atlas.graph.storage.directory=${sys:atlas.home}/data/berkley
+                <graph.storage.backend>berkeleyje</graph.storage.backend>
+                <graph.storage.properties>#Berkeley
+atlas.graph.storage.directory=${sys:atlas.home}/data/berkeley
 atlas.graph.storage.lock.clean-expired=true
 atlas.graph.storage.lock.expiry-time=500
 atlas.graph.storage.lock.wait-time=300
-                </titan.storage.properties>
-                <titan.index.backend>elasticsearch</titan.index.backend>
-                <titan.index.properties>#ElasticSearch
+                </graph.storage.properties>
+                <graph.index.backend>elasticsearch</graph.index.backend>
+                <graph.index.properties>#ElasticSearch
 atlas.graph.index.search.directory=${sys:atlas.home}/data/es
 atlas.graph.index.search.elasticsearch.client-only=false
 atlas.graph.index.search.elasticsearch.local-mode=true
 atlas.graph.index.search.elasticsearch.create.sleep=2000
-                </titan.index.properties>
-		<entity.repository.properties>atlas.EntityAuditRepository.impl=org.apache.atlas.repository.audit.NoopEntityAuditRepository</entity.repository.properties>
+                </graph.index.properties>
+		        <entity.repository.properties>atlas.EntityAuditRepository.impl=org.apache.atlas.repository.audit.NoopEntityAuditRepository</entity.repository.properties>
             </properties>
         </profile>
 
-        <!-- profile to configure berkeley and elasticsearch with the distribution -->
+        <!-- profile to configure external hbase and solr  with the distribution -->
         <profile>
             <id>external-hbase-solr</id>
             <activation>
@@ -156,14 +157,14 @@ atlas.graph.index.search.elasticsearch.create.sleep=2000
                 <activeByDefault>false</activeByDefault>
             </activation>
             <properties>
-                <titan.storage.properties>#Hbase
+                <graph.storage.properties>#Hbase
 #For standalone mode , specify localhost
 #for distributed mode, specify zookeeper quorum here - For more information refer http://s3.thinkaurelius.com/docs/titan/current/hbase.html#_remote_server_mode_2
 atlas.graph.storage.hostname=localhost
 atlas.graph.storage.hbase.regions-per-server=1
 atlas.graph.storage.lock.wait-time=10000
-                </titan.storage.properties>
-                <titan.index.properties>#Solr
+                </graph.storage.properties>
+                <graph.index.properties>#Solr
 #Solr cloud mode properties
 atlas.graph.index.search.solr.mode=cloud
 atlas.graph.index.search.solr.zookeeper-url=localhost:2181
@@ -173,7 +174,7 @@ atlas.graph.index.search.solr.zookeeper-session-timeout=60000
 #Solr http mode properties
 #atlas.graph.index.search.solr.mode=http
 #atlas.graph.index.search.solr.http-urls=http://localhost:8983/solr
-                </titan.index.properties>
+                </graph.index.properties>
 
                 <hbase.embedded>true</hbase.embedded>
                 <solr.embedded>true</solr.embedded>

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/distro/src/conf/atlas-application.properties
----------------------------------------------------------------------
diff --git a/distro/src/conf/atlas-application.properties b/distro/src/conf/atlas-application.properties
index 9e8adca..b61c2bc 100755
--- a/distro/src/conf/atlas-application.properties
+++ b/distro/src/conf/atlas-application.properties
@@ -24,10 +24,10 @@
 #atlas.graphdb.backend=org.apache.atlas.repository.graphdb.titan0.Titan0GraphDatabase
 
 # Graph Storage
-atlas.graph.storage.backend=${titan.storage.backend}
+atlas.graph.storage.backend=${graph.storage.backend}
 atlas.graph.storage.hbase.table=apache_atlas_titan
 
-${titan.storage.properties}
+${graph.storage.properties}
 
 # Gremlin Query Optimizer
 #
@@ -58,9 +58,9 @@ ${entity.repository.properties}
 
 
 # Graph Search Index
-atlas.graph.index.search.backend=${titan.index.backend}
+atlas.graph.index.search.backend=${graph.index.backend}
 
-${titan.index.properties}
+${graph.index.properties}
 
 # Solr-specific configuration property
 atlas.graph.index.search.max-result-set-size=150

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/docs/src/site/twiki/Configuration.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/Configuration.twiki b/docs/src/site/twiki/Configuration.twiki
index a5069ab..19c39b0 100644
--- a/docs/src/site/twiki/Configuration.twiki
+++ b/docs/src/site/twiki/Configuration.twiki
@@ -13,7 +13,7 @@ details. The example below uses BerkeleyDBJE.
 
 <verbatim>
 atlas.graph.storage.backend=berkeleyje
-atlas.graph.storage.directory=data/berkley
+atlas.graph.storage.directory=data/berkeley
 </verbatim>
 
 ---++++ Graph persistence engine - Hbase

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/api/pom.xml
----------------------------------------------------------------------
diff --git a/graphdb/api/pom.xml b/graphdb/api/pom.xml
index 186e745..c949a98 100644
--- a/graphdb/api/pom.xml
+++ b/graphdb/api/pom.xml
@@ -33,7 +33,7 @@
 
     <dependencies>
 
-         <dependency>
+        <dependency>
             <groupId>org.apache.atlas</groupId>
             <artifactId>atlas-typesystem</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopGraphQuery.java
new file mode 100644
index 0000000..5c57d6f
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopGraphQuery.java
@@ -0,0 +1,81 @@
+/**
+ * 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.graphdb.tinkerpop.query;
+
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+
+import java.util.Collection;
+
+/**
+ * Interfaces that provides a thin wrapper around GraphQuery (used by Titan0) and
+ * TitanGraphQuery (used by Titan 1).
+ *
+ * This abstraction  allows TitanGraphQuery to work on any version of Titan.
+ *
+ * @param <V>
+ * @param <E>
+ */
+public interface NativeTinkerpopGraphQuery<V, E> {
+
+    /**
+     * Executes the graph query.
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices();
+
+    /**
+     * Executes the graph query.
+     * @return
+     */
+    Iterable<AtlasEdge<V, E>> edges();
+
+    /**
+     * Executes graph query
+     * @param limit Max vertices to return
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int limit);
+
+    /**
+     * Executes graph query
+     * @param offset Starting offset
+     * @param limit Max vertices to return
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int offset, int limit);
+
+
+    /**
+     * Adds an in condition to the query.
+     *
+     * @param propertyName
+     * @param values
+     */
+    void in(String propertyName, Collection<?> values);
+
+    /**
+     * Adds a has condition to the query.
+     *
+     * @param propertyName
+     * @param op
+     * @param value
+     */
+    void has(String propertyName, QueryOperator op, Object value);
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopQueryFactory.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopQueryFactory.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopQueryFactory.java
new file mode 100644
index 0000000..faa4f36
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/NativeTinkerpopQueryFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.graphdb.tinkerpop.query;
+
+/**
+ * Interface that indicates that something can create instances of
+ * NativeTitanGraphQuery.
+ *
+ * @param <V>
+ * @param <E>
+ */
+public interface NativeTinkerpopQueryFactory<V, E> {
+
+    /**
+     * Creates a NativeTinkerpopGraphQuery.
+     * @return
+     */
+    NativeTinkerpopGraphQuery<V, E> createNativeTinkerpopQuery();
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/TinkerpopGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/TinkerpopGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/TinkerpopGraphQuery.java
new file mode 100644
index 0000000..a59a06a
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/TinkerpopGraphQuery.java
@@ -0,0 +1,249 @@
+/**
+ * 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.graphdb.tinkerpop.query;
+
+import com.google.common.base.Preconditions;
+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.repository.graphdb.tinkerpop.query.expr.AndCondition;
+import org.apache.atlas.repository.graphdb.tinkerpop.query.expr.HasPredicate;
+import org.apache.atlas.repository.graphdb.tinkerpop.query.expr.InPredicate;
+import org.apache.atlas.repository.graphdb.tinkerpop.query.expr.OrCondition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * TODO!! - update comment
+ * Abstract implementation of AtlasGraphQuery that is used by both Titan 0.5.4
+ * and Titan 1.0.0.
+ * <p>
+ * Represents a graph query as an OrConditions which consists of
+ * 1 or more AndConditions.  The query is executed by converting
+ * the AndConditions to native GraphQuery instances that can be executed
+ * directly against Titan.  The overall result is obtained by unioning together
+ * the results from those individual GraphQueries.
+ * <p>
+ * Here is a pictoral view of what is going on here.  Conceptually,
+ * the query being executed can be though of as the where clause
+ * in a query
+ *
+ * <pre>
+ * where (a =1 and b=2) or (a=2 and b=3)
+ *
+ *                ||
+ *               \||/
+ *                \/
+ *
+ *           OrCondition
+ *                 |
+ *       +---------+--------+
+ *       |                  |
+ *   AndCondition     AndCondition
+ *   (a=1 and b=2)     (a=2 and b=3)
+ *
+ *       ||                 ||
+ *      \||/               \||/
+ *       \/                 \/
+ *
+ *   GraphQuery          GraphQuery
+ *
+ *       ||                 ||
+ *      \||/               \||/
+ *       \/                 \/
+ *
+ *     vertices          vertices
+ *           \            /
+ *           _\/        \/_
+ *               (UNION)
+ *
+ *                 ||
+ *                \||/
+ *                 \/
+ *
+ *               result
+ * </pre>
+ *
+ *
+ */
+public abstract class TinkerpopGraphQuery<V, E> implements AtlasGraphQuery<V, E> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TinkerpopGraphQuery.class);
+    protected final AtlasGraph<V, E> graph;
+    private final OrCondition queryCondition = new OrCondition();
+    private final boolean isChildQuery;
+    protected abstract NativeTinkerpopQueryFactory<V, E> getQueryFactory();
+
+    /**
+     * Creates a TinkerpopGraphQuery.
+     *
+     * @param graph
+     */
+    public TinkerpopGraphQuery(AtlasGraph<V, E> graph) {
+        this.graph = graph;
+        this.isChildQuery = false;
+    }
+
+    /**
+     * Creates a TinkerpopGraphQuery.
+     *
+     * @param graph
+     * @param isChildQuery
+     */
+    public TinkerpopGraphQuery(AtlasGraph<V, E> graph, boolean isChildQuery) {
+        this.graph = graph;
+        this.isChildQuery = isChildQuery;
+    }
+
+    @Override
+    public AtlasGraphQuery<V, E> has(String propertyKey, Object value) {
+        queryCondition.andWith(new HasPredicate(propertyKey, ComparisionOperator.EQUAL, value));
+        return this;
+    }
+
+    @Override
+    public Iterable<AtlasVertex<V, E>> vertices() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing: " + queryCondition);
+        }
+
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
+        Set<AtlasVertex<V, E>> result = new HashSet<>();
+        for(AndCondition andExpr : queryCondition.getAndTerms()) {
+            NativeTinkerpopGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
+            for(AtlasVertex<V, E> vertex : andQuery.vertices()) {
+                result.add(vertex);
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public Iterable<AtlasEdge<V, E>> edges() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing: " + queryCondition);
+        }
+
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
+        Set<AtlasEdge<V, E>> result = new HashSet<>();
+        for(AndCondition andExpr : queryCondition.getAndTerms()) {
+            NativeTinkerpopGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
+            for(AtlasEdge<V, E> edge : andQuery.edges()) {
+                result.add(edge);
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public Iterable<AtlasVertex<V, E>> vertices(int limit) {
+        return vertices(0, limit);
+    }
+
+    @Override
+    public Iterable<AtlasVertex<V, E>> vertices(int offset, int limit) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing: " + queryCondition);
+        }
+
+        Preconditions.checkArgument(offset >= 0, "Offset must be non-negative");
+        Preconditions.checkArgument(limit >= 0, "Limit must be non-negative");
+
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
+        Set<AtlasVertex<V, E>> result = new HashSet<>();
+        long resultIdx = 0;
+        for(AndCondition andExpr : queryCondition.getAndTerms()) {
+            if (result.size() == limit) {
+                break;
+            }
+
+            NativeTinkerpopGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
+            for(AtlasVertex<V, E> vertex : andQuery.vertices(offset + limit)) {
+                if (resultIdx >= offset) {
+                    result.add(vertex);
+
+                    if (result.size() == limit) {
+                        break;
+                    }
+                }
+
+                resultIdx++;
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public AtlasGraphQuery<V, E> has(String propertyKey, QueryOperator operator,
+            Object value) {
+        queryCondition.andWith(new HasPredicate(propertyKey, operator, value));
+        return this;
+    }
+
+
+    @Override
+    public AtlasGraphQuery<V, E> in(String propertyKey, Collection<?> values) {
+        queryCondition.andWith(new InPredicate(propertyKey, values));
+        return this;
+    }
+
+    @Override
+    public AtlasGraphQuery<V, E> or(List<AtlasGraphQuery<V, E>> childQueries) {
+
+        //Construct an overall OrCondition by combining all of the children for
+        //the OrConditions in all of the childQueries that we passed in.  Then, "and" the current
+        //query condition with this overall OrCondition.
+
+        OrCondition overallChildQuery = new OrCondition(false);
+
+        for(AtlasGraphQuery<V, E> atlasChildQuery : childQueries) {
+            if (!atlasChildQuery.isChildQuery()) {
+                throw new IllegalArgumentException(atlasChildQuery + " is not a child query");
+            }
+            TinkerpopGraphQuery<V, E> childQuery = (TinkerpopGraphQuery<V, E>)atlasChildQuery;
+            overallChildQuery.orWith(childQuery.getOrCondition());
+        }
+
+        queryCondition.andWith(overallChildQuery);
+        return this;
+    }
+
+    private OrCondition getOrCondition() {
+        return queryCondition;
+    }
+
+    @Override
+    public AtlasGraphQuery<V, E> addConditionsFrom(AtlasGraphQuery<V, E> otherQuery) {
+
+        TinkerpopGraphQuery<V, E> childQuery = (TinkerpopGraphQuery<V, E>)otherQuery;
+        queryCondition.andWith(childQuery.getOrCondition());
+        return this;
+    }
+
+    @Override
+    public boolean isChildQuery() {
+        return isChildQuery;
+    }
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/AndCondition.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/AndCondition.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/AndCondition.java
new file mode 100644
index 0000000..8140e09
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/AndCondition.java
@@ -0,0 +1,96 @@
+/**
+ * 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.graphdb.tinkerpop.query.expr;
+
+import org.apache.atlas.repository.graphdb.tinkerpop.query.NativeTinkerpopQueryFactory;
+import org.apache.atlas.repository.graphdb.tinkerpop.query.NativeTinkerpopGraphQuery;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Represents an AndCondition in a graph query.  Only vertices that
+ * satisfy the conditions in all of the query predicates will be returned
+ *
+ * Represents a query with predicates that are 'AND'ed together. These can be
+ * executed natively using Titan's GraphQuery mechanism.
+ */
+public class AndCondition {
+
+    private List<QueryPredicate> children = new ArrayList<>();
+
+    public AndCondition() {
+
+    }
+
+    /**
+     * Adds a query predicate that must be met by vertices.
+     * @param predicate
+     */
+    public void andWith(QueryPredicate predicate) {
+        children.add(predicate);
+    }
+
+    /**
+     * Adds multiple predicates that much be met by the vertices.
+     *
+     * @param predicates
+     */
+    public void andWith(List<QueryPredicate> predicates) {
+        children.addAll(predicates);
+    }
+
+    /**
+     * Makes a copy of this AndExpr.
+     *
+     * @return
+     */
+    public AndCondition copy() {
+        AndCondition builder = new AndCondition();
+        builder.children.addAll(children);
+        return builder;
+    }
+
+    /**
+     * Gets the query predicates.
+     *
+     * @return
+     */
+    public List<QueryPredicate> getTerms() {
+        return children;
+    }
+
+    /**
+     * Creates a NativeTinkerpopGraphQuery that can be used to evaluate this condition.
+     *
+     * @param factory
+     * @return
+     */
+    public <V, E> NativeTinkerpopGraphQuery<V, E> create(NativeTinkerpopQueryFactory<V, E> factory) {
+        NativeTinkerpopGraphQuery<V, E> query = factory.createNativeTinkerpopQuery();
+        for (QueryPredicate predicate : children) {
+            predicate.addTo(query);
+        }
+        return query;
+    }
+
+    @Override
+    public String toString() {
+        return "AndExpr [predicates=" + children + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/HasPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/HasPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/HasPredicate.java
new file mode 100644
index 0000000..d637cd8
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/HasPredicate.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas.repository.graphdb.tinkerpop.query.expr;
+
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
+import org.apache.atlas.repository.graphdb.tinkerpop.query.NativeTinkerpopGraphQuery;
+
+/**
+ * Query predicate that checks whether the given property has the specified
+ * relationship with the value specified.
+ */
+public class HasPredicate implements QueryPredicate {
+
+    private String propertyName;
+    private QueryOperator op;
+    private Object value;
+
+    public HasPredicate(String propertyName, QueryOperator op, Object value) {
+        this.propertyName = propertyName;
+        this.op = op;
+        this.value = value;
+    }
+
+    @Override
+    public void addTo(NativeTinkerpopGraphQuery query) {
+        query.has(propertyName, op, value);
+    }
+
+    @Override
+    public String toString() {
+        return "HasTerm [propertyName=" + propertyName + ", op=" + op + ", value=" + value + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/InPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/InPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/InPredicate.java
new file mode 100644
index 0000000..19969dd
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/InPredicate.java
@@ -0,0 +1,49 @@
+/**
+ * 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.graphdb.tinkerpop.query.expr;
+
+import java.util.Collection;
+
+import org.apache.atlas.repository.graphdb.tinkerpop.query.NativeTinkerpopGraphQuery;
+
+/**
+ * Query predicate that checks whether the value of a given property is within the
+ * provided set of allowed values.
+ */
+public class InPredicate implements QueryPredicate {
+
+    private String propertyName;
+    private Collection<?> values;
+
+    public InPredicate(String propertyName, Collection<?> values) {
+        super();
+        this.propertyName = propertyName;
+        this.values = values;
+    }
+
+    @Override
+    public void addTo(NativeTinkerpopGraphQuery query) {
+        query.in(propertyName, values);
+    }
+
+    @Override
+    public String toString() {
+        return "InPredicate [propertyName=" + propertyName + ", values=" + values + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/OrCondition.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/OrCondition.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/OrCondition.java
new file mode 100644
index 0000000..034201f
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/OrCondition.java
@@ -0,0 +1,136 @@
+/**
+ * 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.graphdb.tinkerpop.query.expr;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Represents an OrCondition that has one or more AndConditions as it children.  The OrCondition
+ * matches vertices that meet the criteria in at least one of its children.  The overall query result is
+ * computed by executing the TitanGraphQuery queries that correspond to each AndCondition individually and
+ * then unioning their results together. This is needed because the native Titan query mechanism does not
+ * natively support 'OR' conditions.  When we execute the query, we accomplish the 'OR' by executing all of the
+ * individual queries and unioning the results together.
+ *
+ */
+public class OrCondition {
+
+    private List<AndCondition> children;
+
+    public OrCondition() {
+        this(true);
+    }
+
+    private OrCondition(List<AndCondition> children) {
+        this.children = children;
+    }
+
+    public OrCondition(boolean addInitialTerm) {
+        this.children = new ArrayList<>();
+        if (addInitialTerm) {
+            children.add(new AndCondition());
+        }
+    }
+
+    /**
+    /**
+     * Updates this OrCondition in place so that it  matches vertices that satisfy the current
+     * OrCondition AND that match the specified predicate.
+     *
+     * @param other
+     */
+    public void andWith(QueryPredicate predicate) {
+
+        for (AndCondition child : children) {
+            child.andWith(predicate);
+        }
+    }
+
+    public List<AndCondition> getAndTerms() {
+        return children;
+    }
+
+    /**
+     * Updates this OrCondition in place so that it matches vertices that satisfy the current
+     * OrCondition AND that satisfy the provided OrCondition.
+     *
+     * @param other
+     */
+    public void andWith(OrCondition other) {
+
+        //Because Titan does not natively support Or conditions in Graph Queries,
+        //we need to expand out the condition so it is in the form of a single OrCondition
+        //that contains only AndConditions.  We do this by following the rules of boolean
+        //algebra.  As an example, suppose the current condition is ((a=1 and b=2) or (c=3 and d=4)).
+        //Suppose "other" is ((e=5 and f=6) or (g=7 or h=8)).  The overall condition, after applying the
+        //"and" is:
+        //
+        //((a=1 and b=2) or (c=3 and d=4)) and ((e=5 and f=6) or (g=7 and h=8))
+        //
+        //This needs to be expanded out to remove the nested or clauses.  The result of this expansion is:
+        //
+        //(a=1 and b=2 and e=5 and f=6) or
+        //(a=1 and b=2 and g=7 and h=8) or
+        //(c=3 and d=4 and e=5 and f=6) or
+        //(c=3 and d=4 and g=7 and h=8)
+
+        //The logic below does this expansion, in a generalized way.  It loops through the existing AndConditions
+        //and, in a nested loop, through the AndConditions in "other".  For each of those combinations,
+        //it creates a new AndCondition that combines the two AndConditions together.  These combined
+        //AndConditions become the new set of AndConditions in this OrCondition.
+
+        List<AndCondition> expandedExpressionChildren = new ArrayList<>();
+        for (AndCondition otherExprTerm : other.getAndTerms()) {
+            for (AndCondition currentExpr : children) {
+                AndCondition currentAndConditionCopy = currentExpr.copy();
+                currentAndConditionCopy.andWith(otherExprTerm.getTerms());
+                expandedExpressionChildren.add(currentAndConditionCopy);
+            }
+        }
+        children = expandedExpressionChildren;
+    }
+
+    /**
+     * Updates this OrCondition in place so that it matches vertices that satisfy the current
+     * OrCondition OR that satisfy the provided OrCondition.
+     *
+     * @param other
+     */
+    public void orWith(OrCondition other) {
+        children.addAll(other.getAndTerms());
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("OrCondition [andExprs=");
+        Iterator<AndCondition> it = children.iterator();
+        while (it.hasNext()) {
+            AndCondition andExpr = it.next();
+            builder.append(andExpr.toString());
+            if (it.hasNext()) {
+                builder.append(",");
+            }
+        }
+        builder.append("]");
+        return builder.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/QueryPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/QueryPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/QueryPredicate.java
new file mode 100644
index 0000000..d44fabd
--- /dev/null
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/tinkerpop/query/expr/QueryPredicate.java
@@ -0,0 +1,33 @@
+/**
+ * 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.graphdb.tinkerpop.query.expr;
+
+import org.apache.atlas.repository.graphdb.tinkerpop.query.NativeTinkerpopGraphQuery;
+
+/**
+ * Represents a predicate in an AndExpression.
+ */
+public interface QueryPredicate {
+
+    /**
+     * Adds the query term to a NativeTinkerpopGraphQuery that is being generated.
+     *
+     * @param query
+     */
+    void addTo(NativeTinkerpopGraphQuery query);
+}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
deleted file mode 100644
index 288b325..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * 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.graphdb.titan.query;
-
-import org.apache.atlas.repository.graphdb.AtlasEdge;
-import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
-import org.apache.atlas.repository.graphdb.AtlasVertex;
-
-import java.util.Collection;
-
-/**
- * Interfaces that provides a thin wrapper around GraphQuery (used by Titan0) and
- * TitanGraphQuery (used by Titan 1).
- *
- * This abstraction  allows TitanGraphQuery to work on any version of Titan.
- *
- * @param <V>
- * @param <E>
- */
-public interface NativeTitanGraphQuery<V, E> {
-
-    /**
-     * Executes the graph query.
-     * @return
-     */
-    Iterable<AtlasVertex<V, E>> vertices();
-
-    /**
-     * Executes the graph query.
-     * @return
-     */
-    Iterable<AtlasEdge<V, E>> edges();
-
-    /**
-     * Executes graph query
-     * @param limit Max vertices to return
-     * @return
-     */
-    Iterable<AtlasVertex<V, E>> vertices(int limit);
-
-    /**
-     * Executes graph query
-     * @param offset Starting offset
-     * @param limit Max vertices to return
-     * @return
-     */
-    Iterable<AtlasVertex<V, E>> vertices(int offset, int limit);
-
-
-    /**
-     * Adds an in condition to the query.
-     *
-     * @param propertyName
-     * @param values
-     */
-    void in(String propertyName, Collection<?> values);
-
-    /**
-     * Adds a has condition to the query.
-     *
-     * @param propertyName
-     * @param op
-     * @param value
-     */
-    void has(String propertyName, QueryOperator op, Object value);
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanQueryFactory.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanQueryFactory.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanQueryFactory.java
deleted file mode 100644
index ac7ff9e..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanQueryFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.graphdb.titan.query;
-
-/**
- * Interface that indicates that something can create instances of
- * NativeTitanGraphQuery.
- *
- * @param <V>
- * @param <E>
- */
-public interface NativeTitanQueryFactory<V, E> {
-
-    /**
-     * Creates a NativeTitanGraphQuery.
-     * @return
-     */
-    NativeTitanGraphQuery<V, E> createNativeTitanQuery();
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
deleted file mode 100644
index dfdb91b..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- * 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.graphdb.titan.query;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-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.repository.graphdb.titan.query.expr.AndCondition;
-import org.apache.atlas.repository.graphdb.titan.query.expr.HasPredicate;
-import org.apache.atlas.repository.graphdb.titan.query.expr.InPredicate;
-import org.apache.atlas.repository.graphdb.titan.query.expr.OrCondition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-/**
- * Abstract implementation of AtlasGraphQuery that is used by both Titan 0.5.4
- * and Titan 1.0.0.
- * <p>
- * Represents a graph query as an OrConditions which consists of
- * 1 or more AndConditions.  The query is executed by converting
- * the AndConditions to native GraphQuery instances that can be executed
- * directly against Titan.  The overall result is obtained by unioning together
- * the results from those individual GraphQueries.
- * <p>
- * Here is a pictoral view of what is going on here.  Conceptually,
- * the query being executed can be though of as the where clause
- * in a query
- *
- * <pre>
- * where (a =1 and b=2) or (a=2 and b=3)
- *
- *                ||
- *               \||/
- *                \/
- *
- *           OrCondition
- *                 |
- *       +---------+--------+
- *       |                  |
- *   AndCondition     AndCondition
- *   (a=1 and b=2)     (a=2 and b=3)
- *
- *       ||                 ||
- *      \||/               \||/
- *       \/                 \/
- *
- *   GraphQuery          GraphQuery
- *
- *       ||                 ||
- *      \||/               \||/
- *       \/                 \/
- *
- *     vertices          vertices
- *           \            /
- *           _\/        \/_
- *               (UNION)
- *
- *                 ||
- *                \||/
- *                 \/
- *
- *               result
- * </pre>
- *
- *
- */
-public abstract class TitanGraphQuery<V, E> implements AtlasGraphQuery<V, E> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TitanGraphQuery.class);
-    protected final AtlasGraph<V, E> graph;
-    private final OrCondition queryCondition = new OrCondition();
-    private final boolean isChildQuery;
-    protected abstract NativeTitanQueryFactory<V, E> getQueryFactory();
-
-    /**
-     * Creates a TitanGraphQuery.
-     *
-     * @param graph
-     */
-    public TitanGraphQuery(AtlasGraph<V, E> graph) {
-        this.graph = graph;
-        this.isChildQuery = false;
-    }
-
-    /**
-     * Creates a TitanGraphQuery.
-     *
-     * @param graph
-     * @param isChildQuery
-     */
-    public TitanGraphQuery(AtlasGraph<V, E> graph, boolean isChildQuery) {
-        this.graph = graph;
-        this.isChildQuery = isChildQuery;
-    }
-
-    @Override
-    public AtlasGraphQuery<V, E> has(String propertyKey, Object value) {
-        queryCondition.andWith(new HasPredicate(propertyKey, ComparisionOperator.EQUAL, value));
-        return this;
-    }
-
-    @Override
-    public Iterable<AtlasVertex<V, E>> vertices() {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Executing: " + queryCondition);
-        }
-
-        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
-        Set<AtlasVertex<V, E>> result = new HashSet<>();
-        for(AndCondition andExpr : queryCondition.getAndTerms()) {
-            NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
-            for(AtlasVertex<V, E> vertex : andQuery.vertices()) {
-                result.add(vertex);
-            }
-        }
-        return result;
-    }
-
-    @Override
-    public Iterable<AtlasEdge<V, E>> edges() {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Executing: " + queryCondition);
-        }
-
-        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
-        Set<AtlasEdge<V, E>> result = new HashSet<>();
-        for(AndCondition andExpr : queryCondition.getAndTerms()) {
-            NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
-            for(AtlasEdge<V, E> edge : andQuery.edges()) {
-                result.add(edge);
-            }
-        }
-        return result;
-    }
-
-    @Override
-    public Iterable<AtlasVertex<V, E>> vertices(int limit) {
-        return vertices(0, limit);
-    }
-
-    @Override
-    public Iterable<AtlasVertex<V, E>> vertices(int offset, int limit) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Executing: " + queryCondition);
-        }
-
-        Preconditions.checkArgument(offset >= 0, "Offset must be non-negative");
-        Preconditions.checkArgument(limit >= 0, "Limit must be non-negative");
-
-        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
-        Set<AtlasVertex<V, E>> result = new HashSet<>();
-        long resultIdx = 0;
-        for(AndCondition andExpr : queryCondition.getAndTerms()) {
-            if (result.size() == limit) {
-                break;
-            }
-
-            NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
-            for(AtlasVertex<V, E> vertex : andQuery.vertices(offset + limit)) {
-                if (resultIdx >= offset) {
-                    result.add(vertex);
-
-                    if (result.size() == limit) {
-                        break;
-                    }
-                }
-
-                resultIdx++;
-            }
-        }
-
-        return result;
-    }
-
-    @Override
-    public AtlasGraphQuery<V, E> has(String propertyKey, QueryOperator operator,
-            Object value) {
-        queryCondition.andWith(new HasPredicate(propertyKey, operator, value));
-        return this;
-    }
-
-
-    @Override
-    public AtlasGraphQuery<V, E> in(String propertyKey, Collection<?> values) {
-        queryCondition.andWith(new InPredicate(propertyKey, values));
-        return this;
-    }
-
-    @Override
-    public AtlasGraphQuery<V, E> or(List<AtlasGraphQuery<V, E>> childQueries) {
-
-        //Construct an overall OrCondition by combining all of the children for
-        //the OrConditions in all of the childQueries that we passed in.  Then, "and" the current
-        //query condition with this overall OrCondition.
-
-        OrCondition overallChildQuery = new OrCondition(false);
-
-        for(AtlasGraphQuery<V, E> atlasChildQuery : childQueries) {
-            if (!atlasChildQuery.isChildQuery()) {
-                throw new IllegalArgumentException(atlasChildQuery + " is not a child query");
-            }
-            TitanGraphQuery<V, E> childQuery = (TitanGraphQuery<V, E>)atlasChildQuery;
-            overallChildQuery.orWith(childQuery.getOrCondition());
-        }
-
-        queryCondition.andWith(overallChildQuery);
-        return this;
-    }
-
-    private OrCondition getOrCondition() {
-        return queryCondition;
-    }
-
-    @Override
-    public AtlasGraphQuery<V, E> addConditionsFrom(AtlasGraphQuery<V, E> otherQuery) {
-
-        TitanGraphQuery<V, E> childQuery = (TitanGraphQuery<V, E>)otherQuery;
-        queryCondition.andWith(childQuery.getOrCondition());
-        return this;
-    }
-
-    @Override
-    public boolean isChildQuery() {
-        return isChildQuery;
-    }
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
deleted file mode 100644
index db5093f..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * 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.graphdb.titan.query.expr;
-
-import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
-import org.apache.atlas.repository.graphdb.titan.query.NativeTitanQueryFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Represents an AndCondition in a graph query.  Only vertices that
- * satisfy the conditions in all of the query predicates will be returned
- *
- * Represents a query with predicates that are 'AND'ed together. These can be
- * executed natively using Titan's GraphQuery mechanism.
- */
-public class AndCondition {
-
-    private List<QueryPredicate> children = new ArrayList<>();
-
-    public AndCondition() {
-
-    }
-
-    /**
-     * Adds a query predicate that must be met by vertices.
-     * @param predicate
-     */
-    public void andWith(QueryPredicate predicate) {
-        children.add(predicate);
-    }
-
-    /**
-     * Adds multiple predicates that much be met by the vertices.
-     *
-     * @param predicates
-     */
-    public void andWith(List<QueryPredicate> predicates) {
-        children.addAll(predicates);
-    }
-
-    /**
-     * Makes a copy of this AndExpr.
-     *
-     * @return
-     */
-    public AndCondition copy() {
-        AndCondition builder = new AndCondition();
-        builder.children.addAll(children);
-        return builder;
-    }
-
-    /**
-     * Gets the query predicates.
-     *
-     * @return
-     */
-    public List<QueryPredicate> getTerms() {
-        return children;
-    }
-
-    /**
-     * Creates a NativeTitanGraphQuery that can be used to evaluate this condition.
-     *
-     * @param factory
-     * @return
-     */
-    public <V, E> NativeTitanGraphQuery<V, E> create(NativeTitanQueryFactory<V, E> factory) {
-        NativeTitanGraphQuery<V, E> query = factory.createNativeTitanQuery();
-        for (QueryPredicate predicate : children) {
-            predicate.addTo(query);
-        }
-        return query;
-    }
-
-    @Override
-    public String toString() {
-        return "AndExpr [predicates=" + children + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
deleted file mode 100644
index 0652c41..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * 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.graphdb.titan.query.expr;
-
-import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
-import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
-
-/**
- * Query predicate that checks whether the given property has the specified
- * relationship with the value specified.
- */
-public class HasPredicate implements QueryPredicate {
-
-    private String propertyName;
-    private QueryOperator op;
-    private Object value;
-
-    public HasPredicate(String propertyName, QueryOperator op, Object value) {
-        this.propertyName = propertyName;
-        this.op = op;
-        this.value = value;
-    }
-
-    @Override
-    public void addTo(NativeTitanGraphQuery query) {
-        query.has(propertyName, op, value);
-    }
-
-    @Override
-    public String toString() {
-        return "HasTerm [propertyName=" + propertyName + ", op=" + op + ", value=" + value + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/InPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/InPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/InPredicate.java
deleted file mode 100644
index ca0e8ab..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/InPredicate.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.graphdb.titan.query.expr;
-
-import java.util.Collection;
-
-import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
-
-/**
- * Query predicate that checks whether the value of a given property is within the
- * provided set of allowed values.
- */
-public class InPredicate implements QueryPredicate {
-
-    private String propertyName;
-    private Collection<?> values;
-
-    public InPredicate(String propertyName, Collection<?> values) {
-        super();
-        this.propertyName = propertyName;
-        this.values = values;
-    }
-
-    @Override
-    public void addTo(NativeTitanGraphQuery query) {
-        query.in(propertyName, values);
-    }
-
-    @Override
-    public String toString() {
-        return "InPredicate [propertyName=" + propertyName + ", values=" + values + "]";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/OrCondition.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/OrCondition.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/OrCondition.java
deleted file mode 100644
index e7a8a75..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/OrCondition.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * 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.graphdb.titan.query.expr;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Represents an OrCondition that has one or more AndConditions as it children.  The OrCondition
- * matches vertices that meet the criteria in at least one of its children.  The overall query result is
- * computed by executing the TitanGraphQuery queries that correspond to each AndCondition individually and
- * then unioning their results together. This is needed because the native Titan query mechanism does not
- * natively support 'OR' conditions.  When we execute the query, we accomplish the 'OR' by executing all of the
- * individual queries and unioning the results together.
- *
- */
-public class OrCondition {
-
-    private List<AndCondition> children;
-
-    public OrCondition() {
-        this(true);
-    }
-
-    private OrCondition(List<AndCondition> children) {
-        this.children = children;
-    }
-
-    public OrCondition(boolean addInitialTerm) {
-        this.children = new ArrayList<>();
-        if (addInitialTerm) {
-            children.add(new AndCondition());
-        }
-    }
-
-    /**
-    /**
-     * Updates this OrCondition in place so that it  matches vertices that satisfy the current
-     * OrCondition AND that match the specified predicate.
-     *
-     * @param other
-     */
-    public void andWith(QueryPredicate predicate) {
-
-        for (AndCondition child : children) {
-            child.andWith(predicate);
-        }
-    }
-
-    public List<AndCondition> getAndTerms() {
-        return children;
-    }
-
-    /**
-     * Updates this OrCondition in place so that it matches vertices that satisfy the current
-     * OrCondition AND that satisfy the provided OrCondition.
-     *
-     * @param other
-     */
-    public void andWith(OrCondition other) {
-
-        //Because Titan does not natively support Or conditions in Graph Queries,
-        //we need to expand out the condition so it is in the form of a single OrCondition
-        //that contains only AndConditions.  We do this by following the rules of boolean
-        //algebra.  As an example, suppose the current condition is ((a=1 and b=2) or (c=3 and d=4)).
-        //Suppose "other" is ((e=5 and f=6) or (g=7 or h=8)).  The overall condition, after applying the
-        //"and" is:
-        //
-        //((a=1 and b=2) or (c=3 and d=4)) and ((e=5 and f=6) or (g=7 and h=8))
-        //
-        //This needs to be expanded out to remove the nested or clauses.  The result of this expansion is:
-        //
-        //(a=1 and b=2 and e=5 and f=6) or
-        //(a=1 and b=2 and g=7 and h=8) or
-        //(c=3 and d=4 and e=5 and f=6) or
-        //(c=3 and d=4 and g=7 and h=8)
-
-        //The logic below does this expansion, in a generalized way.  It loops through the existing AndConditions
-        //and, in a nested loop, through the AndConditions in "other".  For each of those combinations,
-        //it creates a new AndCondition that combines the two AndConditions together.  These combined
-        //AndConditions become the new set of AndConditions in this OrCondition.
-
-        List<AndCondition> expandedExpressionChildren = new ArrayList<>();
-        for (AndCondition otherExprTerm : other.getAndTerms()) {
-            for (AndCondition currentExpr : children) {
-                AndCondition currentAndConditionCopy = currentExpr.copy();
-                currentAndConditionCopy.andWith(otherExprTerm.getTerms());
-                expandedExpressionChildren.add(currentAndConditionCopy);
-            }
-        }
-        children = expandedExpressionChildren;
-    }
-
-    /**
-     * Updates this OrCondition in place so that it matches vertices that satisfy the current
-     * OrCondition OR that satisfy the provided OrCondition.
-     *
-     * @param other
-     */
-    public void orWith(OrCondition other) {
-        children.addAll(other.getAndTerms());
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder builder = new StringBuilder();
-        builder.append("OrCondition [andExprs=");
-        Iterator<AndCondition> it = children.iterator();
-        while (it.hasNext()) {
-            AndCondition andExpr = it.next();
-            builder.append(andExpr.toString());
-            if (it.hasNext()) {
-                builder.append(",");
-            }
-        }
-        builder.append("]");
-        return builder.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/QueryPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/QueryPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/QueryPredicate.java
deleted file mode 100644
index a80522b..0000000
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/QueryPredicate.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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.graphdb.titan.query.expr;
-
-import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
-
-/**
- * Represents a predicate in an AndExpression.
- */
-public interface QueryPredicate {
-
-    /**
-     * Adds the query term to a NativeTitanGraphQuery that is being generated.
-     *
-     * @param query
-     */
-    void addTo(NativeTitanGraphQuery query);
-}

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/graphdb-impls/pom.xml
----------------------------------------------------------------------
diff --git a/graphdb/graphdb-impls/pom.xml b/graphdb/graphdb-impls/pom.xml
index 62a0994..fd87ea4 100644
--- a/graphdb/graphdb-impls/pom.xml
+++ b/graphdb/graphdb-impls/pom.xml
@@ -26,27 +26,22 @@
         <relativePath>../pom.xml</relativePath>
     </parent>
     <artifactId>atlas-graphdb-impls</artifactId>
-    <!-- Convenience dependency project that allows
-    the dependency on the correct implementation class
-    to be configured through profiles.  In implementation
-    specific profiles, the dependency on this project
-    should be configured to exclude all but the
-    proper dependency
-    -->
+
 
     <description>Apache Atlas Graph Database Implementation Dependencies</description>
     <name>Apache Atlas Graph Database Implementation Dependencies</name>
     <packaging>pom</packaging>
+
+
     <dependencies>
+        <!-- Single graph provider configured in root pom using
+             graphGroup, graphModule, graphVersion properties -->
         <dependency>
-            <groupId>org.apache.atlas</groupId>
-            <artifactId>atlas-graphdb-titan1</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.atlas</groupId>
-            <artifactId>atlas-graphdb-titan0</artifactId>
+            <groupId>${graphGroup}</groupId>
+            <artifactId>${graphArtifact}</artifactId>
             <version>${project.version}</version>
         </dependency>
+
     </dependencies>
+
 </project>

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/janus/pom.xml
----------------------------------------------------------------------
diff --git a/graphdb/janus/pom.xml b/graphdb/janus/pom.xml
new file mode 100644
index 0000000..e8a0a61
--- /dev/null
+++ b/graphdb/janus/pom.xml
@@ -0,0 +1,240 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+~ 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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>atlas-graphdb</artifactId>
+        <groupId>org.apache.atlas</groupId>
+        <version>1.0.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>atlas-graphdb-janus</artifactId>
+    <description>Apache Atlas JanusGraph DB Impl</description>
+    <name>Apache Atlas JanusGraph DB Impl</name>
+    <packaging>jar</packaging>
+
+    <!-- This POM file produces a jar with everything that Atlas needs to use JanusGraph.
+         Some dependencies, like slf4j are excluded from the jar because they are included in Atlas -->
+
+    <properties>
+        <tinkerpop.version>3.2.6</tinkerpop.version>
+        <janus.version>0.1.1</janus.version>
+        <checkstyle.failOnViolation>false</checkstyle.failOnViolation>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-graphdb-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-graphdb-api</artifactId>
+            <!-- set scope to provided to prevent the this from being included in the shared jar -->
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-core</artifactId>
+            <version>${janus.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-berkeleyje</artifactId>
+            <version>${janus.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-es</artifactId>
+            <version>${janus.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-hbase</artifactId>
+            <version>${janus.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-solr</artifactId>
+            <version>${janus.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.solr</groupId>
+                    <artifactId>solr-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.janusgraph</groupId>
+            <artifactId>janusgraph-lucene</artifactId>
+            <version>${janus.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.solr</groupId>
+            <artifactId>solr-solrj</artifactId>
+            <version>6.1.0</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.9.2</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.tinkerpop</groupId>
+            <artifactId>gremlin-core</artifactId>
+            <version>${tinkerpop.version}</version>
+        </dependency>
+
+        <!-- Test dependencies -->
+
+        <dependency>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.atlas</groupId>
+            <artifactId>atlas-graphdb-common</artifactId>
+            <version>${project.version}</version>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+
+
+    </dependencies>
+
+    <build>
+        <plugins>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-enforcer-plugin</artifactId>
+                <version>1.4.1</version>
+                <executions>
+                    <execution>
+                        <id>enforce-java-8</id>
+                        <goals>
+                            <goal>enforce</goal>
+                        </goals>
+                        <configuration>
+                            <rules>
+                                <requireJavaVersion>
+                                    <version>1.8.0</version>
+                                </requireJavaVersion>
+                                <requireMavenVersion>
+                                    <version>3.1.0</version>
+                                </requireMavenVersion>
+                            </rules>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.18.1</version>
+                <configuration>
+                    <skip>${skipUTs}</skip>
+                </configuration>
+            </plugin>
+
+            <!--
+                 Create 'uber' jar that contains all of the dependencies (except those whose scope is provided)
+                 janus and its dependencies are included.  The other dependencies are bundled in the war file.
+            -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.4.3</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <shadedArtifactAttached>false</shadedArtifactAttached>
+                            <artifactSet>
+                                <excludes>
+                                    <!-- these are bundled with Atlas -->
+                                    <exclude>org.slf4j:*</exclude>
+                                </excludes>
+                            </artifactSet>
+                            <transformers>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                            </transformers>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/atlas/blob/503dddb8/graphdb/janus/readme.txt
----------------------------------------------------------------------
diff --git a/graphdb/janus/readme.txt b/graphdb/janus/readme.txt
new file mode 100644
index 0000000..fb935be
--- /dev/null
+++ b/graphdb/janus/readme.txt
@@ -0,0 +1,52 @@
+==================
+JanusGraph README
+==================
+
+IMPORTANT: Support for JanusGraph in Atlas is currently a work in progress.
+
+
+ARCHITECTURE NOTES
+------------------
+
+To build Atlas with JanusGraph, you must set GRAPH-PROVIDER as follows:
+
+mvn install [-P dist] -DGRAPH-PROVIDER=janus
+
+
+JanusGraph support Gremlin3 only (and NOT Gremlin2), so the gremlin used by Atlas is translated into Gremlin3
+by the GremlinExpressionFactory.
+
+
+
+REQUIREMENTS
+--------------
+
+JanusGraph requires Java 8 to be used both when building and running Atlas.
+Unless Java 8 is used, the janus module will not be built - this is checked by the maven-enforcer-plugin.
+
+
+USING ATLAS WITH JANUS GRAPH
+----------------------------
+
+1) Build Atlas with the janus graph-provider maven profile enabled:
+
+mvn install [-P dist] -DGRAPH-PROVIDER=janus
+
+Some tests in the repository and webapp projects are skipped when running with the janus provider, due to hard
+dependencies on Gremlin2. These components need to be updated. Please refer to "known issues" section below.
+
+This will build Atlas and run all of the tests against Janus.  Such a build MUST be used with JanusGraph and
+CANNOT be used with any other graph provider, e.g. Titan 0.5.4 or Titan 1.0.0.
+
+2) Configure the Atlas runtime to use JanusGraph by setting the atlas.graphdb.backend property in
+ATLAS_HOME/conf/atlas-application.properties, as follows:
+
+atlas.graphdb.backend=org.apache.atlas.repository.graphdb.janus.AtlasJanusGraphDatabase
+
+3) Attempt to start the Atlas server.
+
+
+KNOWN ISSUES
+------------
+
+None yet...
\ No newline at end of file