You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/01/06 07:37:39 UTC

incubator-atlas git commit: ATLAS-1308: Discovery/Search REST API - v2

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 6c3b981ad -> 46f539e15


ATLAS-1308: Discovery/Search REST API - v2

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 46f539e1535986b0e3992da7fe6e1c7f6f01a69c
Parents: 6c3b981
Author: Sarath Subramanian <ss...@hortonworks.com>
Authored: Thu Jan 5 22:34:44 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Jan 5 23:37:19 2017 -0800

----------------------------------------------------------------------
 .../apache/atlas/AtlasDiscoveryClientV2.java    |  96 +++++++
 .../java/org/apache/atlas/AtlasErrorCode.java   |   7 +-
 .../model/discovery/AtlasSearchResult.java      | 225 +++++++++++++++
 .../atlas/model/instance/AtlasEntityHeader.java |   1 +
 .../model/lineage/AtlasLineageService.java      |  34 ---
 release-log.txt                                 |   1 +
 .../apache/atlas/RepositoryMetadataModule.java  |   5 +-
 .../atlas/discovery/AtlasDiscoveryService.java  |  43 +++
 .../atlas/discovery/AtlasLineageService.java    |  35 +++
 .../atlas/discovery/EntityDiscoveryService.java | 279 +++++++++++++++++++
 .../atlas/discovery/EntityLineageService.java   |   2 -
 .../org/apache/atlas/query/Expressions.scala    |   3 +
 .../atlas/web/resources/LineageResource.java    |   3 +-
 .../apache/atlas/web/rest/DiscoveryREST.java    |  68 +++++
 .../org/apache/atlas/web/rest/LineageREST.java  |   2 +-
 .../atlas/web/resources/BaseResourceIT.java     |   4 +
 .../EntityDiscoveryJerseyResourceIT.java        | 211 ++++++++++++++
 17 files changed, 977 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/client/src/main/java/org/apache/atlas/AtlasDiscoveryClientV2.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/atlas/AtlasDiscoveryClientV2.java b/client/src/main/java/org/apache/atlas/AtlasDiscoveryClientV2.java
new file mode 100644
index 0000000..f653111
--- /dev/null
+++ b/client/src/main/java/org/apache/atlas/AtlasDiscoveryClientV2.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.commons.configuration.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+
+import static org.apache.atlas.AtlasClient.LIMIT;
+import static org.apache.atlas.AtlasClient.OFFSET;
+import static org.apache.atlas.AtlasClient.QUERY;
+
+public class AtlasDiscoveryClientV2 extends AtlasBaseClient {
+
+    private static final String DISCOVERY_URI = BASE_URI + "v2/search";
+    private static final String DSL_URI       = DISCOVERY_URI + "/dsl";
+    private static final String FULL_TEXT_URI = DISCOVERY_URI + "/fulltext";
+
+    private static final APIInfo DSL_SEARCH       = new APIInfo(DSL_URI, HttpMethod.GET, Response.Status.OK);
+    private static final APIInfo FULL_TEXT_SEARCH = new APIInfo(FULL_TEXT_URI, HttpMethod.GET, Response.Status.OK);
+
+    public AtlasDiscoveryClientV2(String[] baseUrl, String[] basicAuthUserNamePassword) {
+        super(baseUrl, basicAuthUserNamePassword);
+    }
+
+    public AtlasDiscoveryClientV2(String... baseUrls) throws AtlasException {
+        super(baseUrls);
+    }
+
+    public AtlasDiscoveryClientV2(UserGroupInformation ugi, String doAsUser, String... baseUrls) {
+        super(ugi, doAsUser, baseUrls);
+    }
+
+    protected AtlasDiscoveryClientV2() {
+        super();
+    }
+
+    @VisibleForTesting
+    AtlasDiscoveryClientV2(WebResource service, Configuration configuration) {
+        super(service, configuration);
+    }
+
+    public AtlasSearchResult dslSearch(final String query) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add(QUERY, query);
+
+        return callAPI(DSL_SEARCH, AtlasSearchResult.class, queryParams);
+    }
+
+    public AtlasSearchResult dslSearchWithParams(final String query, final int limit, final int offset) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add(QUERY, query);
+        queryParams.add(LIMIT, String.valueOf(limit));
+        queryParams.add(OFFSET, String.valueOf(offset));
+
+        return callAPI(DSL_SEARCH, AtlasSearchResult.class, queryParams);
+    }
+
+    public AtlasSearchResult fullTextSearch(final String query) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add(QUERY, query);
+
+        return callAPI(FULL_TEXT_SEARCH, AtlasSearchResult.class, queryParams);
+    }
+
+    public AtlasSearchResult fullTextSearchWithParams(final String query, final int limit, final int offset) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add(QUERY, query);
+        queryParams.add(LIMIT, String.valueOf(limit));
+        queryParams.add(OFFSET, String.valueOf(offset));
+
+        return callAPI(FULL_TEXT_SEARCH, AtlasSearchResult.class, queryParams);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
index f026ec6..6770c41 100644
--- a/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
+++ b/intg/src/main/java/org/apache/atlas/AtlasErrorCode.java
@@ -20,13 +20,14 @@ package org.apache.atlas;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.Response;
 import java.text.MessageFormat;
 import java.util.Arrays;
+
+import javax.ws.rs.core.Response;
+
 public enum AtlasErrorCode {
     NO_SEARCH_RESULTS(204, "ATLAS2041E", "Given search filter {0} did not yield any results"),
 
-    // All Bad request enums go here
     UNKNOWN_TYPE(400, "ATLAS4001E", "Unknown type {0} for {1}.{2}"),
     CIRCULAR_REFERENCE(400, "ATLAS4002E", "{0}: invalid supertypes - circular reference back to self {1}"),
     INCOMPATIBLE_SUPERTYPE(400, "ATLAS4003E", "{0}: incompatible supertype {1}"),
@@ -60,6 +61,8 @@ public enum AtlasErrorCode {
     INSTANCE_GUID_NOT_FOUND(404, "ATLAS4045E", "Given instance guid {0} is invalid"),
     INSTANCE_LINEAGE_INVALID_PARAMS(404, "ATLAS4046E", "Invalid lineage query parameters passed {0}: {1}"),
     INSTANCE_LINEAGE_QUERY_FAILED(404, "ATLAS4047E", "Instance lineage query failed {0}"),
+    DISCOVERY_QUERY_FAILED(404, "ATLAS4048E", "Discovery query failed {0}"),
+
 
     // All data conflict errors go here
     TYPE_ALREADY_EXISTS(409, "ATLAS4091E", "Given type {0} already exists"),

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java b/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java
new file mode 100644
index 0000000..c4c0063
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas.model.discovery;
+
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.NONE;
+import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONLY;
+
+@JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown = true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.PROPERTY)
+public class AtlasSearchResult implements Serializable {
+    private String                    queryText;
+    private AtlasQueryType            queryType;
+    private List<AtlasEntityHeader>   entities;
+    private AttributeSearchResult     attributes;
+    private List<AtlasFullTextResult> fullTextResult;
+
+    public AtlasSearchResult() {}
+
+    public AtlasSearchResult(String queryText, AtlasQueryType queryType) {
+        setQueryText(queryText);
+        setQueryType(queryType);
+        setEntities(null);
+        setAttributes(null);
+        setFullTextResult(null);
+    }
+
+    public String getQueryText() { return queryText; }
+
+    public void setQueryText(String queryText) { this.queryText = queryText; }
+
+    public AtlasQueryType getQueryType() { return queryType; }
+
+    public void setQueryType(AtlasQueryType queryType) { this.queryType = queryType; }
+
+    public List<AtlasEntityHeader> getEntities() { return entities; }
+
+    public void setEntities(List<AtlasEntityHeader> entities) { this.entities = entities; }
+
+    public AttributeSearchResult getAttributes() { return attributes; }
+
+    public void setAttributes(AttributeSearchResult attributes) { this.attributes = attributes; }
+
+    public List<AtlasFullTextResult> getFullTextResult() { return fullTextResult; }
+
+    public void setFullTextResult(List<AtlasFullTextResult> fullTextResult) { this.fullTextResult = fullTextResult; }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        AtlasSearchResult that = (AtlasSearchResult) o;
+        return Objects.equals(queryText, that.queryText) &&
+               Objects.equals(queryType, that.queryType) &&
+               Objects.equals(entities, that.entities) &&
+               Objects.equals(attributes, that.attributes) &&
+               Objects.equals(fullTextResult, that.fullTextResult);
+    }
+
+    @Override
+    public int hashCode() { return Objects.hash(queryText, queryType, entities, attributes, fullTextResult); }
+
+    @Override
+    public String toString() {
+        return "AtlasSearchResult{" +
+                "queryText='" + queryText + '\'' +
+                ", queryType=" + queryType +
+                ", entities=" + entities +
+                ", attributes=" + attributes +
+                ", fullTextResult=" + fullTextResult +
+                '}';
+    }
+
+    public void addEntity(AtlasEntityHeader newEntity) {
+        if (entities == null) {
+            entities = new ArrayList<>();
+        }
+
+        if (entities.isEmpty()) {
+            entities.add(newEntity);
+        } else {
+            removeEntity(newEntity);
+            entities.add(newEntity);
+        }
+    }
+
+    public void removeEntity(AtlasEntityHeader entity) {
+        List<AtlasEntityHeader> entities = this.entities;
+
+        if (CollectionUtils.isNotEmpty(entities)) {
+            Iterator<AtlasEntityHeader> iter = entities.iterator();
+            while (iter.hasNext()) {
+                AtlasEntityHeader currEntity = iter.next();
+                if (StringUtils.equals(currEntity.getGuid(), entity.getGuid())) {
+                    iter.remove();
+                }
+            }
+        }
+    }
+
+    public enum AtlasQueryType { DSL, FULL_TEXT, GREMLIN }
+
+    @JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+    @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+    @JsonIgnoreProperties(ignoreUnknown = true)
+    @XmlRootElement
+    @XmlAccessorType(XmlAccessType.PROPERTY)
+    public static class AttributeSearchResult {
+        private List<String>       name;
+        private List<List<Object>> values;
+
+        public AttributeSearchResult() { }
+
+        public AttributeSearchResult(List<String> name, List<List<Object>> values) {
+            this.name = name;
+            this.values = values;
+        }
+
+        public List<String> getName() { return name; }
+
+        public void setName(List<String> name) { this.name = name; }
+
+        public List<List<Object>> getValues() { return values; }
+
+        public void setValues(List<List<Object>> values) { this.values = values; }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            AttributeSearchResult that = (AttributeSearchResult) o;
+            return Objects.equals(name, that.name) &&
+                    Objects.equals(values, that.values);
+        }
+
+        @Override
+        public int hashCode() { return Objects.hash(name, values); }
+
+        @Override
+        public String toString() {
+            return "AttributeSearchResult{" +
+                    "name="   + name + ", " +
+                    "values=" + values +
+                    '}';
+        }
+    }
+
+    @JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+    @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+    @JsonIgnoreProperties(ignoreUnknown = true)
+    @XmlRootElement
+    @XmlAccessorType(XmlAccessType.PROPERTY)
+    public static class AtlasFullTextResult {
+        AtlasEntityHeader entity;
+        Double            score;
+
+        public AtlasFullTextResult() {}
+
+        public AtlasFullTextResult(AtlasEntityHeader entity, Double score) {
+            this.entity = entity;
+            this.score  = score;
+        }
+
+        public AtlasEntityHeader getEntity() { return entity; }
+
+        public void setEntity(AtlasEntityHeader entity) { this.entity = entity; }
+
+        public Double getScore() { return score; }
+
+        public void setScore(Double score) { this.score = score; }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            AtlasFullTextResult that = (AtlasFullTextResult) o;
+            return Objects.equals(entity, that.entity) &&
+                   Objects.equals(score, that.score);
+        }
+
+        @Override
+        public int hashCode() { return Objects.hash(entity, score); }
+
+        @Override
+        public String toString() {
+            return "AtlasFullTextResult{" +
+                    "entity=" + entity +
+                    ", score=" + score +
+                    '}';
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntityHeader.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntityHeader.java b/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntityHeader.java
index 538534f..e7b70aa 100644
--- a/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntityHeader.java
+++ b/intg/src/main/java/org/apache/atlas/model/instance/AtlasEntityHeader.java
@@ -77,6 +77,7 @@ public class AtlasEntityHeader extends AtlasStruct implements Serializable {
         if (other != null) {
             setGuid(other.getGuid());
             setStatus(other.getStatus());
+            setDisplayText(other.getDisplayText());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/intg/src/main/java/org/apache/atlas/model/lineage/AtlasLineageService.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/lineage/AtlasLineageService.java b/intg/src/main/java/org/apache/atlas/model/lineage/AtlasLineageService.java
deleted file mode 100644
index fc58f58..0000000
--- a/intg/src/main/java/org/apache/atlas/model/lineage/AtlasLineageService.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.atlas.model.lineage;
-
-
-import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
-
-public interface AtlasLineageService {
-    /**
-     * @param entityGuid unique ID of the entity
-     * @param direction direction of lineage - INPUT, OUTPUT or BOTH
-     * @param depth number of hops in lineage
-     * @return AtlasLineageInfo
-     */
-    AtlasLineageInfo getAtlasLineageInfo(String entityGuid, LineageDirection direction, int depth) throws AtlasBaseException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 8887247..24319b8 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -9,6 +9,7 @@ ATLAS-1060 Add composite indexes for exact match performance improvements for al
 ATLAS-1127 Modify creation and modification timestamps to Date instead of Long(sumasai)
 
 ALL CHANGES:
+ATLAS-1308 Discovery/Search REST API - v2 (sarath.kum4r@gmail.com via mneethiraj)
 ATLAS-1408 added validation to prevent creating types with . (dot) in name (ashutoshm via mneethiraj)
 ATLAS-1277 Add feather use 'order by ' in the DSL search (zhangqiang2 via sumasai)
 ATLAS-1379 Avoid object query overhead when report query selects class type alias (guptaneeru via dkantor)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java b/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
index 8e086c9..54dda50 100755
--- a/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
+++ b/repository/src/main/java/org/apache/atlas/RepositoryMetadataModule.java
@@ -24,15 +24,17 @@ import com.google.inject.matcher.Matchers;
 import com.google.inject.multibindings.Multibinder;
 
 import org.aopalliance.intercept.MethodInterceptor;
+import org.apache.atlas.discovery.AtlasLineageService;
 import org.apache.atlas.discovery.DataSetLineageService;
 import org.apache.atlas.discovery.DiscoveryService;
+import org.apache.atlas.discovery.EntityDiscoveryService;
 import org.apache.atlas.discovery.EntityLineageService;
 import org.apache.atlas.discovery.LineageService;
 import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
 import org.apache.atlas.listener.EntityChangeListener;
 import org.apache.atlas.listener.TypeDefChangeListener;
 import org.apache.atlas.listener.TypesChangeListener;
-import org.apache.atlas.model.lineage.AtlasLineageService;
+import org.apache.atlas.discovery.AtlasDiscoveryService;
 import org.apache.atlas.repository.MetadataRepository;
 import org.apache.atlas.repository.audit.EntityAuditListener;
 import org.apache.atlas.repository.audit.EntityAuditRepository;
@@ -94,6 +96,7 @@ public class RepositoryMetadataModule extends com.google.inject.AbstractModule {
 
         // bind the DiscoveryService interface to an implementation
         bind(DiscoveryService.class).to(GraphBackedDiscoveryService.class).asEagerSingleton();
+        bind(AtlasDiscoveryService.class).to(EntityDiscoveryService.class).asEagerSingleton();
 
         bind(LineageService.class).to(DataSetLineageService.class).asEagerSingleton();
         bind(AtlasLineageService.class).to(EntityLineageService.class).asEagerSingleton();

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java b/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
new file mode 100644
index 0000000..e3eb52a
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.discovery;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+
+public interface AtlasDiscoveryService {
+    /**
+     *
+     * @param query search query in DSL format.
+     * @param limit number of resultant rows (for pagination). [ limit > 0 ] and [ limit < maxlimit ]. -1 maps to atlas.search.defaultlimit property.
+     * @param offset offset to the results returned (for pagination). [ offset >= 0 ]. -1 maps to offset 0.
+     * @return AtlasSearchResult
+     */
+    AtlasSearchResult searchUsingDslQuery(String query, int limit, int offset) throws AtlasBaseException;
+
+    /**
+     *
+     * @param query search query.
+     * @param limit number of resultant rows (for pagination). [ limit > 0 ] and [ limit < maxlimit ]. -1 maps to atlas.search.defaultlimit property.
+     * @param offset offset to the results returned (for pagination). [ offset >= 0 ]. -1 maps to offset 0.
+     * @return AtlasSearchResult
+     */
+    AtlasSearchResult searchUsingFullTextQuery(String query, int limit, int offset);
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/java/org/apache/atlas/discovery/AtlasLineageService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/AtlasLineageService.java b/repository/src/main/java/org/apache/atlas/discovery/AtlasLineageService.java
new file mode 100644
index 0000000..5f49625
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/AtlasLineageService.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.discovery;
+
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.lineage.AtlasLineageInfo;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
+
+public interface AtlasLineageService {
+    /**
+     * @param entityGuid unique ID of the entity
+     * @param direction direction of lineage - INPUT, OUTPUT or BOTH
+     * @param depth number of hops in lineage
+     * @return AtlasLineageInfo
+     */
+    AtlasLineageInfo getAtlasLineageInfo(String entityGuid, LineageDirection direction, int depth) throws AtlasBaseException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java b/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
new file mode 100644
index 0000000..be9fed2
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
@@ -0,0 +1,279 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas.discovery;
+
+import org.apache.atlas.AtlasConfiguration;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasFullTextResult;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasQueryType;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AttributeSearchResult;
+import org.apache.atlas.discovery.graph.DefaultGraphPersistenceStrategy;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.instance.AtlasEntity.Status;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.query.Expressions.AliasExpression;
+import org.apache.atlas.query.Expressions.Expression;
+import org.apache.atlas.query.Expressions.SelectExpression;
+import org.apache.atlas.query.GremlinQuery;
+import org.apache.atlas.query.GremlinTranslator;
+import org.apache.atlas.query.QueryParams;
+import org.apache.atlas.query.QueryParser;
+import org.apache.atlas.query.QueryProcessor;
+import org.apache.atlas.query.SelectExpressionHelper;
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.MetadataRepository;
+import org.apache.atlas.repository.graph.AtlasGraphProvider;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery.Result;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.util.Either;
+import scala.util.parsing.combinator.Parsers.NoSuccess;
+
+import javax.inject.Inject;
+import javax.script.ScriptException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.atlas.AtlasErrorCode.DISCOVERY_QUERY_FAILED;
+
+public class EntityDiscoveryService implements AtlasDiscoveryService {
+
+    private final AtlasGraph graph;
+    private final DefaultGraphPersistenceStrategy graphPersistenceStrategy;
+    private static final Logger LOG = LoggerFactory.getLogger(EntityDiscoveryService.class);
+
+    @Inject
+    EntityDiscoveryService(MetadataRepository metadataRepository) {
+        this.graph = AtlasGraphProvider.getGraphInstance();
+        this.graphPersistenceStrategy = new DefaultGraphPersistenceStrategy(metadataRepository);
+    }
+
+    @Override
+    public AtlasSearchResult searchUsingDslQuery(String dslQuery, int limit, int offset) throws AtlasBaseException {
+        AtlasSearchResult ret = new AtlasSearchResult(dslQuery, AtlasQueryType.DSL);
+        GremlinQuery gremlinQuery = toGremlinQuery(dslQuery, limit, offset);
+
+        try {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Executing DSL query: {}", dslQuery);
+            }
+
+            Object result = graph.executeGremlinScript(gremlinQuery.queryStr(), false);
+
+            if (result instanceof List) {
+                List queryResult = (List) result;
+
+                if (isAtlasVerticesList(queryResult)) {
+                    for (Object entity : queryResult) {
+                        ret.addEntity(toAtlasEntityHeader(entity));
+                    }
+                } else if (isTraitList(queryResult)) {
+                    ret.setEntities(toTraitResult(queryResult));
+
+                } else if (gremlinQuery.hasSelectList()) {
+                    ret.setAttributes(toAttributesResult(queryResult, gremlinQuery));
+                }
+            }
+
+        } catch (ScriptException e) {
+            throw new AtlasBaseException(DISCOVERY_QUERY_FAILED, gremlinQuery.queryStr());
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AtlasSearchResult searchUsingFullTextQuery(String fullTextQuery, int limit, int offset) {
+        AtlasSearchResult ret      = new AtlasSearchResult(fullTextQuery, AtlasQueryType.FULL_TEXT);
+        QueryParams       params   = validateSearchParams(limit, offset);
+        AtlasIndexQuery   idxQuery = toAtlasIndexQuery(fullTextQuery);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing Full text query: {}", fullTextQuery);
+        }
+        ret.setFullTextResult(getIndexQueryResults(idxQuery, params));
+
+        return ret;
+    }
+
+    private List<AtlasFullTextResult> getIndexQueryResults(AtlasIndexQuery query, QueryParams params) {
+        List<AtlasFullTextResult> ret  = new ArrayList<>();
+        Iterator<Result>          iter = query.vertices();
+
+        while (iter.hasNext() && ret.size() < params.limit()) {
+            Result idxQueryResult = iter.next();
+            AtlasVertex vertex = idxQueryResult.getVertex();
+            String guid = vertex.getProperty(Constants.GUID_PROPERTY_KEY, String.class);
+
+            if (guid != null) {
+                AtlasEntityHeader entity = toAtlasEntityHeader(idxQueryResult.getVertex());
+                Double score = idxQueryResult.getScore();
+                ret.add(new AtlasFullTextResult(entity, score));
+            }
+        }
+
+        return ret;
+    }
+
+    private GremlinQuery toGremlinQuery(String query, int limit, int offset) throws AtlasBaseException {
+        QueryParams params = validateSearchParams(limit, offset);
+        Either<NoSuccess, Expression> either = QueryParser.apply(query, params);
+
+        if (either.isLeft()) {
+            throw new AtlasBaseException(DISCOVERY_QUERY_FAILED, query);
+        }
+
+        Expression   expression      = either.right().get();
+        Expression   validExpression = QueryProcessor.validate(expression);
+        GremlinQuery gremlinQuery    = new GremlinTranslator(validExpression, graphPersistenceStrategy).translate();
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Translated Gremlin Query: {}", gremlinQuery.queryStr());
+        }
+
+        return gremlinQuery;
+    }
+
+    private QueryParams validateSearchParams(int limitParam, int offsetParam) {
+        int defaultLimit = AtlasConfiguration.SEARCH_DEFAULT_LIMIT.getInt();
+        int maxLimit     = AtlasConfiguration.SEARCH_MAX_LIMIT.getInt();
+
+        int limit = defaultLimit;
+        if (limitParam > 0 && limitParam <= maxLimit) {
+            limit = limitParam;
+        }
+
+        int offset = 0;
+        if (offsetParam > 0) {
+            offset = offsetParam;
+        }
+
+        return new QueryParams(limit, offset);
+    }
+
+    private AtlasEntityHeader toAtlasEntityHeader(Object vertexObj) {
+        AtlasEntityHeader ret = new AtlasEntityHeader();
+
+        if (vertexObj instanceof AtlasVertex) {
+            AtlasVertex vertex = (AtlasVertex) vertexObj;
+            ret.setTypeName(vertex.getProperty(Constants.TYPE_NAME_PROPERTY_KEY, String.class));
+            ret.setGuid(vertex.getProperty(Constants.GUID_PROPERTY_KEY, String.class));
+            ret.setDisplayText(vertex.getProperty(Constants.QUALIFIED_NAME, String.class));
+
+            String state = vertex.getProperty(Constants.STATE_PROPERTY_KEY, String.class);
+            if (state != null) {
+                Status status = (state.equalsIgnoreCase("ACTIVE") ? Status.STATUS_ACTIVE : Status.STATUS_DELETED);
+                ret.setStatus(status);
+            }
+
+        }
+
+        return ret;
+    }
+
+    private AtlasIndexQuery toAtlasIndexQuery(String fullTextQuery) {
+        String graphQuery = String.format("v.\"%s\":(%s)", Constants.ENTITY_TEXT_PROPERTY_KEY, fullTextQuery);
+        return graph.indexQuery(Constants.FULLTEXT_INDEX, graphQuery);
+    }
+
+    private boolean isAtlasVerticesList(List list) {
+        boolean ret = false;
+
+        if (CollectionUtils.isNotEmpty(list)) {
+            ret = list.get(0) instanceof AtlasVertex;
+        }
+
+        return ret;
+    }
+
+    private boolean isTraitList(List list) {
+        boolean ret = false;
+
+        if (CollectionUtils.isNotEmpty(list)) {
+            Object firstObj = list.get(0);
+
+            if (firstObj instanceof Map) {
+                Map map  = (Map) firstObj;
+                Set keys = map.keySet();
+                ret = (keys.contains("theInstance") || keys.contains("theTrait"));
+            }
+        }
+
+        return ret;
+    }
+
+    private List<AtlasEntityHeader> toTraitResult(List list) {
+        List<AtlasEntityHeader> ret = new ArrayList();
+
+        for (Object mapObj : list) {
+            Map map = (Map) mapObj;
+            if (MapUtils.isNotEmpty(map)) {
+                for (Object key : map.keySet()) {
+                    List values = (List) map.get(key);
+                    if (StringUtils.equals(key.toString(), "theInstance") && isAtlasVerticesList(values)) {
+                        ret.add(toAtlasEntityHeader(values.get(0)));
+                    }
+                }
+            }
+        }
+
+        return ret;
+    }
+
+    private AttributeSearchResult toAttributesResult(List list, GremlinQuery query) {
+        AttributeSearchResult ret = new AttributeSearchResult();
+        List<String> names = new ArrayList<>();
+        List<List<Object>> values = new ArrayList<>();
+
+        // extract select attributes from gremlin query
+        Option<SelectExpression> selectExpr = SelectExpressionHelper.extractSelectExpression(query.expr());
+        if (selectExpr.isDefined()) {
+            List<AliasExpression> aliases = selectExpr.get().toJavaList();
+
+            if (CollectionUtils.isNotEmpty(aliases)) {
+                for (AliasExpression alias : aliases) {
+                    names.add(alias.alias());
+                }
+                ret.setName(names);
+            }
+        }
+
+        for (Object mapObj : list) {
+            Map map = (mapObj instanceof Map ? (Map) mapObj : null);
+            if (MapUtils.isNotEmpty(map)) {
+                for (Object key : map.keySet()) {
+                    Object vals = map.get(key);
+                    values.add((List<Object>) vals);
+                }
+                ret.setValues(values);
+            }
+        }
+
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java b/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
index 14bf143..45e2dd2 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
@@ -27,7 +27,6 @@ import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.atlas.model.lineage.AtlasLineageInfo;
 import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageRelation;
 import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
-import org.apache.atlas.model.lineage.AtlasLineageService;
 import org.apache.atlas.repository.Constants;
 import org.apache.atlas.repository.graph.AtlasGraphProvider;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
@@ -36,7 +35,6 @@ import org.apache.commons.collections.CollectionUtils;
 
 import javax.inject.Inject;
 import javax.script.ScriptException;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/repository/src/main/scala/org/apache/atlas/query/Expressions.scala
----------------------------------------------------------------------
diff --git a/repository/src/main/scala/org/apache/atlas/query/Expressions.scala b/repository/src/main/scala/org/apache/atlas/query/Expressions.scala
index 81be7db..ce21a59 100755
--- a/repository/src/main/scala/org/apache/atlas/query/Expressions.scala
+++ b/repository/src/main/scala/org/apache/atlas/query/Expressions.scala
@@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableCollection
 import org.apache.atlas.AtlasException
 import org.apache.atlas.typesystem.types.DataTypes.{ArrayType, PrimitiveType, TypeCategory}
 import org.apache.atlas.typesystem.types._
+import scala.collection.JavaConverters._
 
 object Expressions {
 
@@ -732,6 +733,8 @@ object Expressions {
              var prefix = if(forGroupBy) { "" } else { s"""${child} select """ }
              s"""${prefix}${selectListWithAlias.mkString("", ", ", "")}"""
         }
+
+        def toJavaList = selectListWithAlias.asJava
     }
 
     case class LoopExpression(val input: Expression, val loopingExpression: Expression,

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/webapp/src/main/java/org/apache/atlas/web/resources/LineageResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/LineageResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/LineageResource.java
index 739c6c1..c7aa372 100644
--- a/webapp/src/main/java/org/apache/atlas/web/resources/LineageResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/LineageResource.java
@@ -20,16 +20,15 @@ package org.apache.atlas.web.resources;
 
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.aspect.Monitored;
+import org.apache.atlas.discovery.AtlasLineageService;
 import org.apache.atlas.discovery.DiscoveryException;
 import org.apache.atlas.discovery.LineageService;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.lineage.AtlasLineageInfo;
 import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
-import org.apache.atlas.model.lineage.AtlasLineageService;
 import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.exception.EntityNotFoundException;
 import org.apache.atlas.typesystem.exception.SchemaNotFoundException;
-import org.apache.atlas.utils.AtlasPerfTracer;
 import org.apache.atlas.web.util.LineageUtils;
 import org.apache.atlas.web.util.Servlets;
 import org.codehaus.jettison.json.JSONException;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java
new file mode 100644
index 0000000..48d8d49
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas.web.rest;
+
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.discovery.AtlasDiscoveryService;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.web.util.Servlets;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+
+@Path("v2/search")
+@Singleton
+public class DiscoveryREST {
+    private final AtlasDiscoveryService atlasDiscoveryService;
+
+    @Inject
+    public DiscoveryREST(AtlasDiscoveryService discoveryService) {
+        this.atlasDiscoveryService = discoveryService;
+    }
+
+    @GET
+    @Path("/dsl")
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasSearchResult searchUsingDSL(@QueryParam("query") String query,
+                                            @QueryParam("limit") int limit,
+                                            @QueryParam("offset") int offset) throws AtlasBaseException {
+
+        AtlasSearchResult ret = atlasDiscoveryService.searchUsingDslQuery(query, limit, offset);
+
+        return ret;
+    }
+
+    @GET
+    @Path("/fulltext")
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasSearchResult searchUsingFullText(@QueryParam("query") String query,
+                                                 @QueryParam("limit") int limit,
+                                                 @QueryParam("offset") int offset) throws AtlasBaseException {
+
+        AtlasSearchResult ret = atlasDiscoveryService.searchUsingFullTextQuery(query, limit, offset);
+
+        return ret;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/webapp/src/main/java/org/apache/atlas/web/rest/LineageREST.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/rest/LineageREST.java b/webapp/src/main/java/org/apache/atlas/web/rest/LineageREST.java
index effd29f..0904b69 100644
--- a/webapp/src/main/java/org/apache/atlas/web/rest/LineageREST.java
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/LineageREST.java
@@ -19,10 +19,10 @@
 package org.apache.atlas.web.rest;
 
 
+import org.apache.atlas.discovery.AtlasLineageService;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.lineage.AtlasLineageInfo;
 import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
-import org.apache.atlas.model.lineage.AtlasLineageService;
 import org.apache.atlas.web.util.Servlets;
 
 import javax.inject.Inject;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
index 9b56c92..dcb1264 100755
--- a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
@@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet;
 import kafka.consumer.ConsumerTimeoutException;
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasDiscoveryClientV2;
 import org.apache.atlas.AtlasEntitiesClientV2;
 import org.apache.atlas.AtlasServiceException;
 import org.apache.atlas.AtlasTypedefClientV2;
@@ -85,6 +86,7 @@ public abstract class BaseResourceIT {
     protected AtlasClient atlasClientV1;
     protected AtlasTypedefClientV2 typedefClientV2;
     protected AtlasEntitiesClientV2 entitiesClientV2;
+    protected AtlasDiscoveryClientV2 discoveryClientV2;
 
     public static final Logger LOG = LoggerFactory.getLogger(BaseResourceIT.class);
     protected static final int MAX_WAIT_TIME = 60000;
@@ -104,10 +106,12 @@ public abstract class BaseResourceIT {
             atlasClientV1 = new AtlasClient(atlasUrls, new String[]{"admin", "admin"});
             typedefClientV2 = new AtlasTypedefClientV2(atlasUrls, new String[]{"admin", "admin"});
             entitiesClientV2 = new AtlasEntitiesClientV2(atlasUrls, new String[]{"admin", "admin"});
+            discoveryClientV2 = new AtlasDiscoveryClientV2(atlasUrls, new String[]{"admin", "admin"});
         } else {
             atlasClientV1 = new AtlasClient(atlasUrls);
             typedefClientV2 = new AtlasTypedefClientV2(atlasUrls);
             entitiesClientV2 = new AtlasEntitiesClientV2(atlasUrls);
+            discoveryClientV2 = new AtlasDiscoveryClientV2(atlasUrls);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/46f539e1/webapp/src/test/java/org/apache/atlas/web/resources/EntityDiscoveryJerseyResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/EntityDiscoveryJerseyResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/EntityDiscoveryJerseyResourceIT.java
new file mode 100755
index 0000000..a576dd0
--- /dev/null
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/EntityDiscoveryJerseyResourceIT.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.web.resources;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import org.apache.atlas.AtlasServiceException;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasFullTextResult;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasQueryType;
+import org.apache.atlas.model.instance.AtlasEntity.Status;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.typesystem.TypesDef;
+import org.apache.atlas.typesystem.types.ClassType;
+import org.apache.atlas.typesystem.types.DataTypes;
+import org.apache.atlas.typesystem.types.EnumTypeDefinition;
+import org.apache.atlas.typesystem.types.HierarchicalTypeDefinition;
+import org.apache.atlas.typesystem.types.StructTypeDefinition;
+import org.apache.atlas.typesystem.types.TraitType;
+import org.apache.atlas.typesystem.types.utils.TypesUtil;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import javax.ws.rs.core.MultivaluedMap;
+import java.util.List;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+
+/**
+ * Search V2 Integration Tests.
+ */
+public class EntityDiscoveryJerseyResourceIT extends BaseResourceIT {
+    private String dbName;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        super.setUp();
+        dbName = "db" + randomString();
+        createTypes();
+        createInstance(createHiveDBInstanceV1(dbName));
+    }
+
+    @Test
+    public void testSearchByDSL() throws Exception {
+        String dslQuery = "from "+ DATABASE_TYPE + " " + QUALIFIED_NAME + "=\"" + dbName + "\"";
+
+        AtlasSearchResult searchResult = discoveryClientV2.dslSearch(dslQuery);
+        assertNotNull(searchResult);
+        assertEquals(searchResult.getQueryText(), dslQuery);
+        assertEquals(searchResult.getQueryType(), AtlasQueryType.DSL);
+
+        List<AtlasEntityHeader> entities = searchResult.getEntities();
+        assertNotNull(entities);
+        assertEquals(entities.size(), 1);
+
+        AtlasEntityHeader dbEntity = entities.get(0);
+        assertEquals(dbEntity.getTypeName(), DATABASE_TYPE);
+        assertEquals(dbEntity.getDisplayText(), dbName);
+        assertEquals(dbEntity.getStatus(), Status.STATUS_ACTIVE);
+        assertNotNull(dbEntity.getGuid());
+        assertNull(searchResult.getAttributes());
+        assertNull(searchResult.getFullTextResult());
+    }
+
+    @Test
+    public void testSearchDSLLimits() throws Exception {
+        String dslQuery = "from "+ DATABASE_TYPE + " " + QUALIFIED_NAME + "=\"" + dbName + "\"";
+        AtlasSearchResult searchResult = discoveryClientV2.dslSearch(dslQuery);
+        assertNotNull(searchResult);
+
+        //higher limit, all results returned
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, 10, 0);
+        assertEquals(searchResult.getEntities().size(), 1);
+
+        //default limit and offset -1, all results returned
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, -1, -1);
+        assertEquals(searchResult.getEntities().size(), 1);
+
+        //uses the limit parameter passed
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, 1, 0);
+        assertEquals(searchResult.getEntities().size(), 1);
+
+        //uses the offset parameter passed
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, 10, 1);
+        assertNull(searchResult.getEntities());
+
+        //limit > 0
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, 0, 10);
+        assertNull(searchResult.getEntities());
+
+        //limit > maxlimit
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, Integer.MAX_VALUE, 10);
+        assertNull(searchResult.getEntities());
+
+        //offset >= 0
+        searchResult = discoveryClientV2.dslSearchWithParams(dslQuery, 10, -2);
+        assertEquals(searchResult.getEntities().size(), 1);
+    }
+
+    @Test(expectedExceptions = AtlasServiceException.class)
+    public void testSearchByDSLForUnknownType() throws Exception {
+        String dslQuery = "from blah";
+        discoveryClientV2.dslSearch(dslQuery);
+    }
+
+    @Test
+    public void testSearchUsingDSL() throws Exception {
+        String query = "from "+ DATABASE_TYPE + " " + QUALIFIED_NAME + "=\"" + dbName + "\"";
+        AtlasSearchResult searchResult = discoveryClientV2.dslSearch(query);
+        assertNotNull(searchResult);
+
+        assertEquals(searchResult.getQueryText(), query);
+        assertEquals(searchResult.getQueryType(), AtlasQueryType.DSL);
+        List<AtlasEntityHeader> entities = searchResult.getEntities();
+        assertNotNull(entities);
+        assertEquals(entities.size(), 1);
+
+        AtlasEntityHeader dbEntity = entities.get(0);
+        assertEquals(dbEntity.getTypeName(), DATABASE_TYPE);
+        assertEquals(dbEntity.getDisplayText(), dbName);
+        assertEquals(dbEntity.getStatus(), Status.STATUS_ACTIVE);
+
+        assertNotNull(dbEntity.getGuid());
+        assertNull(searchResult.getAttributes());
+        assertNull(searchResult.getFullTextResult());
+    }
+
+    @Test
+    public void testSearchFullTextOnDSLFailure() throws Exception {
+        String query = "*";
+        AtlasSearchResult searchResult = discoveryClientV2.fullTextSearch(query);
+        assertNotNull(searchResult);
+        assertEquals(searchResult.getQueryText(), query);
+        assertEquals(searchResult.getQueryType(), AtlasQueryType.FULL_TEXT);
+    }
+
+    @Test(dependsOnMethods = "testSearchDSLLimits")
+    public void testSearchUsingFullText() throws Exception {
+        AtlasSearchResult searchResult = discoveryClientV2.fullTextSearchWithParams(dbName, 10, 0);
+        assertNotNull(searchResult);
+
+        assertEquals(searchResult.getQueryText(), dbName);
+        assertEquals(searchResult.getQueryType(), AtlasQueryType.FULL_TEXT);
+
+        List<AtlasFullTextResult> fullTextResults = searchResult.getFullTextResult();
+        assertEquals(fullTextResults.size(), 1);
+
+        AtlasFullTextResult result = fullTextResults.get(0);
+        assertNotNull(result.getEntity());
+        assertEquals(result.getEntity().getTypeName(), DATABASE_TYPE);
+        assertNotNull(result.getScore());
+
+        //API works without limit and offset
+        String query = dbName;
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add("query", query);
+        searchResult = discoveryClientV2.fullTextSearch(query);
+        assertNotNull(searchResult);
+        assertEquals(searchResult.getFullTextResult().size(), 1);
+
+        //verify passed in limits and offsets are used
+        //higher limit and 0 offset returns all results
+        searchResult = discoveryClientV2.fullTextSearchWithParams(query, 10, 0);
+        assertEquals(searchResult.getFullTextResult().size(), 1);
+
+        //offset is used
+        searchResult = discoveryClientV2.fullTextSearchWithParams(query, 10, 1);
+        assertEquals(searchResult.getFullTextResult().size(), 1);
+
+        //limit is used
+        searchResult = discoveryClientV2.fullTextSearchWithParams(query, 1, 0);
+        assertEquals(searchResult.getFullTextResult().size(), 1);
+
+        //higher offset returns 0 results
+        searchResult = discoveryClientV2.fullTextSearchWithParams(query, 1, 2);
+        assertEquals(searchResult.getFullTextResult().size(), 1);
+    }
+
+    private void createTypes() throws Exception {
+        HierarchicalTypeDefinition<ClassType> dslTestTypeDefinition = TypesUtil
+                .createClassTypeDef("dsl_test_type", ImmutableSet.<String>of(),
+                        TypesUtil.createUniqueRequiredAttrDef("name", DataTypes.STRING_TYPE),
+                        TypesUtil.createRequiredAttrDef("description", DataTypes.STRING_TYPE));
+
+        HierarchicalTypeDefinition<TraitType> classificationTraitDefinition = TypesUtil
+                .createTraitTypeDef("Classification", ImmutableSet.<String>of(),
+                        TypesUtil.createRequiredAttrDef("tag", DataTypes.STRING_TYPE));
+        TypesDef typesDef = TypesUtil.getTypesDef(ImmutableList.<EnumTypeDefinition>of(), ImmutableList.<StructTypeDefinition>of(),
+                ImmutableList.of(classificationTraitDefinition), ImmutableList.of(dslTestTypeDefinition));
+        createType(typesDef);
+    }
+}