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/03/04 06:49:56 UTC

incubator-atlas git commit: ATLAS-1630: basic search implementation using fulltext with support for filter by type & classification

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 98f02ff79 -> 2bbbd1a5a


ATLAS-1630: basic search implementation using fulltext with support for filter by type & classification


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

Branch: refs/heads/master
Commit: 2bbbd1a5adc7e2856d16c5b53833480c7f950e8c
Parents: 98f02ff
Author: Sarath Subramanian <ss...@hortonworks.com>
Authored: Fri Mar 3 10:40:19 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Fri Mar 3 22:49:45 2017 -0800

----------------------------------------------------------------------
 .../model/discovery/AtlasSearchResult.java      | 32 +++++---
 .../atlas/type/AtlasClassificationType.java     | 25 +++---
 .../org/apache/atlas/type/AtlasEntityType.java  | 23 ++++--
 .../atlas/discovery/AtlasDiscoveryService.java  | 11 +++
 .../atlas/discovery/EntityDiscoveryService.java | 80 ++++++++++++++++++++
 .../atlas/util/AtlasGremlin2QueryProvider.java  |  9 +++
 .../atlas/util/AtlasGremlinQueryProvider.java   |  7 +-
 .../apache/atlas/web/rest/DiscoveryREST.java    | 28 +++++++
 8 files changed, 188 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/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
index c4c0063..8928bdf 100644
--- a/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java
+++ b/intg/src/main/java/org/apache/atlas/model/discovery/AtlasSearchResult.java
@@ -42,8 +42,10 @@ import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONL
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.PROPERTY)
 public class AtlasSearchResult implements Serializable {
-    private String                    queryText;
     private AtlasQueryType            queryType;
+    private String                    queryText;
+    private String                    type;
+    private String                    classification;
     private List<AtlasEntityHeader>   entities;
     private AttributeSearchResult     attributes;
     private List<AtlasFullTextResult> fullTextResult;
@@ -58,13 +60,21 @@ public class AtlasSearchResult implements Serializable {
         setFullTextResult(null);
     }
 
+    public AtlasQueryType getQueryType() { return queryType; }
+
+    public void setQueryType(AtlasQueryType queryType) { this.queryType = queryType; }
+
     public String getQueryText() { return queryText; }
 
     public void setQueryText(String queryText) { this.queryText = queryText; }
 
-    public AtlasQueryType getQueryType() { return queryType; }
+    public String getType() { return type; }
 
-    public void setQueryType(AtlasQueryType queryType) { this.queryType = queryType; }
+    public void setType(String type) { this.type = type; }
+
+    public String getClassification() { return classification; }
+
+    public void setClassification(String classification) { this.classification = classification; }
 
     public List<AtlasEntityHeader> getEntities() { return entities; }
 
@@ -83,21 +93,25 @@ public class AtlasSearchResult implements Serializable {
         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) &&
+        return Objects.equals(queryType, that.queryType) &&
+               Objects.equals(queryText, that.queryText) &&
+               Objects.equals(type, that.type) &&
+               Objects.equals(classification, that.classification) &&
                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); }
+    public int hashCode() { return Objects.hash(queryText, queryType, entities, attributes, fullTextResult, type, classification); }
 
     @Override
     public String toString() {
         return "AtlasSearchResult{" +
-                "queryText='" + queryText + '\'' +
-                ", queryType=" + queryType +
+                "queryType=" + queryType +
+                ", queryText='" + queryText + '\'' +
+                ", type=" + type +
+                ", classification=" + classification +
                 ", entities=" + entities +
                 ", attributes=" + attributes +
                 ", fullTextResult=" + fullTextResult +
@@ -131,7 +145,7 @@ public class AtlasSearchResult implements Serializable {
         }
     }
 
-    public enum AtlasQueryType { DSL, FULL_TEXT, GREMLIN }
+    public enum AtlasQueryType { DSL, FULL_TEXT, GREMLIN, BASIC }
 
     @JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
     @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java b/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
index 82b0310..c1c3add 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasClassificationType.java
@@ -45,9 +45,10 @@ public class AtlasClassificationType extends AtlasStructType {
 
     private final AtlasClassificationDef classificationDef;
 
-    private List<AtlasClassificationType>  superTypes    = Collections.emptyList();
-    private Set<String>                    allSuperTypes = Collections.emptySet();
-    private Set<String>                    allSubTypes   = Collections.emptySet();
+    private List<AtlasClassificationType> superTypes         = Collections.emptyList();
+    private Set<String>                   allSuperTypes      = Collections.emptySet();
+    private Set<String>                   allSubTypes        = Collections.emptySet();
+    private Set<String>                   typeAndAllSubTypes = Collections.emptySet();
 
     public AtlasClassificationType(AtlasClassificationDef classificationDef) {
         super(classificationDef);
@@ -87,11 +88,14 @@ public class AtlasClassificationType extends AtlasStructType {
             }
         }
 
-        this.superTypes     = Collections.unmodifiableList(s);
-        this.allSuperTypes  = Collections.unmodifiableSet(allS);
-        this.allAttributes  = Collections.unmodifiableMap(allA);
-        this.uniqAttributes = getUniqueAttributes(this.allAttributes);
-        this.allSubTypes    = new HashSet<>(); // this will be populated in resolveReferencesPhase2()
+        this.superTypes         = Collections.unmodifiableList(s);
+        this.allSuperTypes      = Collections.unmodifiableSet(allS);
+        this.allAttributes      = Collections.unmodifiableMap(allA);
+        this.uniqAttributes     = getUniqueAttributes(this.allAttributes);
+        this.allSubTypes        = new HashSet<>(); // this will be populated in resolveReferencesPhase2()
+        this.typeAndAllSubTypes = new HashSet<>(); // this will be populated in resolveReferencesPhase2()
+
+        this.typeAndAllSubTypes.add(this.getTypeName());
     }
 
     @Override
@@ -106,6 +110,7 @@ public class AtlasClassificationType extends AtlasStructType {
 
     private void addSubType(AtlasClassificationType subType) {
         allSubTypes.add(subType.getTypeName());
+        typeAndAllSubTypes.add(subType.getTypeName());
     }
 
     public Set<String> getSuperTypes() {
@@ -115,9 +120,11 @@ public class AtlasClassificationType extends AtlasStructType {
     public Set<String> getAllSuperTypes() { return allSuperTypes; }
 
     public Set<String> getAllSubTypes() {
-        return allSubTypes;
+        return Collections.unmodifiableSet(allSubTypes);
     }
 
+        public Set<String> getTypeAndAllSubTypes() { return Collections.unmodifiableSet(typeAndAllSubTypes); }
+
     public boolean isSuperTypeOf(AtlasClassificationType classificationType) {
         return classificationType != null && allSubTypes.contains(classificationType.getTypeName());
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java b/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
index c6be8f5..6516d48 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasEntityType.java
@@ -47,9 +47,10 @@ public class AtlasEntityType extends AtlasStructType {
 
     private final AtlasEntityDef entityDef;
 
-    private List<AtlasEntityType>       superTypes              = Collections.emptyList();
-    private Set<String>                 allSuperTypes           = Collections.emptySet();
-    private Set<String>                 allSubTypes             = Collections.emptySet();
+    private List<AtlasEntityType> superTypes         = Collections.emptyList();
+    private Set<String>           allSuperTypes      = Collections.emptySet();
+    private Set<String>           allSubTypes        = Collections.emptySet();
+    private Set<String>           typeAndAllSubTypes = Collections.emptySet();
 
     public AtlasEntityType(AtlasEntityDef entityDef) {
         super(entityDef);
@@ -87,11 +88,14 @@ public class AtlasEntityType extends AtlasStructType {
             }
         }
 
-        this.superTypes     = Collections.unmodifiableList(s);
-        this.allSuperTypes  = Collections.unmodifiableSet(allS);
-        this.allAttributes  = Collections.unmodifiableMap(allA);
-        this.uniqAttributes = getUniqueAttributes(this.allAttributes);
-        this.allSubTypes    = new HashSet<>();   // this will be populated in resolveReferencesPhase2()
+        this.superTypes         = Collections.unmodifiableList(s);
+        this.allSuperTypes      = Collections.unmodifiableSet(allS);
+        this.allAttributes      = Collections.unmodifiableMap(allA);
+        this.uniqAttributes     = getUniqueAttributes(this.allAttributes);
+        this.allSubTypes        = new HashSet<>();   // this will be populated in resolveReferencesPhase2()
+        this.typeAndAllSubTypes = new HashSet<>();   // this will be populated in resolveReferencesPhase2()
+
+        this.typeAndAllSubTypes.add(this.getTypeName());
     }
 
     @Override
@@ -114,6 +118,8 @@ public class AtlasEntityType extends AtlasStructType {
 
     public Set<String> getAllSubTypes() { return Collections.unmodifiableSet(allSubTypes); }
 
+    public Set<String> getTypeAndAllSubTypes() { return Collections.unmodifiableSet(typeAndAllSubTypes); }
+
     public boolean isSuperTypeOf(AtlasEntityType entityType) {
         return entityType != null && allSubTypes.contains(entityType.getTypeName());
     }
@@ -316,6 +322,7 @@ public class AtlasEntityType extends AtlasStructType {
 
     private void addSubType(AtlasEntityType subType) {
         allSubTypes.add(subType.getTypeName());
+        typeAndAllSubTypes.add(subType.getTypeName());
     }
 
     private void getTypeHierarchyInfo(AtlasTypeRegistry              typeRegistry,

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/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
index 21ce14d..1044aaa 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
@@ -40,4 +40,15 @@ public interface AtlasDiscoveryService {
      * @return AtlasSearchResult
      */
     AtlasSearchResult searchUsingFullTextQuery(String query, int limit, int offset) throws AtlasBaseException;
+
+    /**
+     *
+     * @param query search query.
+     * @param type entity type.
+     * @param classification classification name.
+     * @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 searchUsingBasicQuery(String query, String type, String classification, int limit, int offset) throws AtlasBaseException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/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
index 5628ed8..90f084d 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
@@ -42,9 +42,14 @@ 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.atlas.repository.store.graph.v1.EntityGraphRetriever;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
 import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.util.AtlasGremlinQueryProvider;
+import org.apache.atlas.util.AtlasGremlinQueryProvider.AtlasGremlinQuery;
 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;
@@ -59,6 +64,8 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.atlas.AtlasErrorCode.DISCOVERY_QUERY_FAILED;
+import static org.apache.atlas.AtlasErrorCode.UNKNOWN_TYPENAME;
+import static org.apache.atlas.AtlasErrorCode.CLASSIFICATION_NOT_FOUND;
 
 public class EntityDiscoveryService implements AtlasDiscoveryService {
     private static final Logger LOG = LoggerFactory.getLogger(EntityDiscoveryService.class);
@@ -66,12 +73,16 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
     private final AtlasGraph                      graph;
     private final DefaultGraphPersistenceStrategy graphPersistenceStrategy;
     private final EntityGraphRetriever            entityRetriever;
+    private final AtlasGremlinQueryProvider       gremlinQueryProvider;
+    private final AtlasTypeRegistry               typeRegistry;
 
     @Inject
     EntityDiscoveryService(MetadataRepository metadataRepository, AtlasTypeRegistry typeRegistry) {
         this.graph                    = AtlasGraphProvider.getGraphInstance();
         this.graphPersistenceStrategy = new DefaultGraphPersistenceStrategy(metadataRepository);
         this.entityRetriever          = new EntityGraphRetriever(typeRegistry);
+        this.gremlinQueryProvider     = AtlasGremlinQueryProvider.INSTANCE;
+        this.typeRegistry             = typeRegistry;
     }
 
     @Override
@@ -145,6 +156,75 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
         return ret;
     }
 
+    @Override
+    public AtlasSearchResult searchUsingBasicQuery(String query, String typeName, String classification, int limit, int offset) throws AtlasBaseException {
+        AtlasSearchResult ret = new AtlasSearchResult(query, AtlasQueryType.BASIC);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing basic search query: {} with type: {} and classification: {}", query, typeName, classification);
+        }
+
+        QueryParams params     = validateSearchParams(limit, offset);
+        String      basicQuery = "g.V()";
+
+        if (StringUtils.isNotEmpty(typeName)) {
+            AtlasEntityType entityType = typeRegistry.getEntityTypeByName(typeName);
+
+            if (entityType == null) {
+                throw new AtlasBaseException(UNKNOWN_TYPENAME, typeName);
+            }
+
+            String typeFilterExpr = gremlinQueryProvider.getQuery(AtlasGremlinQuery.BASIC_SEARCH_TYPE_FILTER);
+
+            basicQuery += String.format(typeFilterExpr,
+                                        StringUtils.join(entityType.getTypeAndAllSubTypes(), "','"));
+
+            ret.setType(typeName);
+        }
+
+        if (StringUtils.isNotEmpty(classification)) {
+            AtlasClassificationType classificationType = typeRegistry.getClassificationTypeByName(classification);
+
+            if (classificationType == null) {
+                throw new AtlasBaseException(CLASSIFICATION_NOT_FOUND, classification);
+            }
+
+            String classificationFilterExpr = gremlinQueryProvider.getQuery(AtlasGremlinQuery.BASIC_SEARCH_CLASSIFICATION_FILTER);
+
+            basicQuery += String.format(classificationFilterExpr,
+                                        StringUtils.join(classificationType.getTypeAndAllSubTypes(), "','"));
+
+            ret.setClassification(classification);
+        }
+
+        basicQuery += String.format(gremlinQueryProvider.getQuery(AtlasGremlinQuery.BASIC_SEARCH_QUERY_FILTER), query);
+        basicQuery += String.format(gremlinQueryProvider.getQuery(AtlasGremlinQuery.TO_RANGE_LIST), params.offset(), params.limit());
+
+        try {
+            Object result = graph.executeGremlinScript(basicQuery, false);
+
+            if (result instanceof List && CollectionUtils.isNotEmpty((List) result)) {
+                List   queryResult  = (List) result;
+                Object firstElement = queryResult.get(0);
+
+                if (firstElement instanceof AtlasVertex) {
+                    for (Object element : queryResult) {
+                        if (element instanceof AtlasVertex) {
+                            ret.addEntity(entityRetriever.toAtlasEntityHeader((AtlasVertex) element));
+
+                        } else {
+                            LOG.warn("searchUsingBasicQuery({}): expected an AtlasVertex; found unexpected entry in result {}", basicQuery, element);
+                        }
+                    }
+                }
+            }
+        } catch (ScriptException e) {
+            throw new AtlasBaseException(DISCOVERY_QUERY_FAILED, basicQuery);
+        }
+
+        return ret;
+    }
+
     private List<AtlasFullTextResult> getIndexQueryResults(AtlasIndexQuery query, QueryParams params) throws AtlasBaseException {
         List<AtlasFullTextResult> ret  = new ArrayList<>();
         Iterator<Result>          iter = query.vertices();

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java b/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
index e4777be..201db6e 100644
--- a/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
+++ b/repository/src/main/java/org/apache/atlas/util/AtlasGremlin2QueryProvider.java
@@ -64,6 +64,15 @@ public class AtlasGremlin2QueryProvider extends AtlasGremlinQueryProvider {
                         "loop('src', {it.loops <= %s}, {((it.object.'__superTypeNames') ? " +
                         "(it.object.'__superTypeNames'.contains('DataSet')) : false)})." +
                         "path().toList()";
+
+            case BASIC_SEARCH_QUERY_FILTER:
+                return ".has('entityText', com.thinkaurelius.titan.core.attribute.Text.CONTAINS, '%s')";
+            case BASIC_SEARCH_TYPE_FILTER:
+                return ".has('__typeName', T.in, ['%s'])";
+            case BASIC_SEARCH_CLASSIFICATION_FILTER:
+                return ".has('__traitNames', T.in, ['%s'])";
+            case TO_RANGE_LIST:
+                return " [%s..<%s].toList()";
         }
         // Should never reach this point
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java b/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
index f076750..ad8f073 100644
--- a/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
+++ b/repository/src/main/java/org/apache/atlas/util/AtlasGremlinQueryProvider.java
@@ -53,7 +53,12 @@ public abstract class AtlasGremlinQueryProvider {
 
         // Lineage Queries
         FULL_LINEAGE,
-        PARTIAL_LINEAGE;
+        PARTIAL_LINEAGE,
+
         // Discovery Queries
+        BASIC_SEARCH_QUERY_FILTER,
+        BASIC_SEARCH_TYPE_FILTER,
+        BASIC_SEARCH_CLASSIFICATION_FILTER,
+        TO_RANGE_LIST
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/2bbbd1a5/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
index 7f75a27..4dd8bf5 100644
--- a/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java
+++ b/webapp/src/main/java/org/apache/atlas/web/rest/DiscoveryREST.java
@@ -90,4 +90,32 @@ public class DiscoveryREST {
 
         return ret;
     }
+
+    /**
+     * Retrieve data for the specified fulltext query
+     * @param query Fulltext query
+     * @param type limit the result to only entities of specified type or its sub-types
+     * @param classification limit the result to only entities tagged with the given classification or or its sub-types
+     * @param limit limit the result set to only include the specified number of entries
+     * @param offset start offset of the result set (useful for pagination)
+     * @return Search results
+     * @throws AtlasBaseException
+     * @HTTP 200 On successful FullText lookup with some results, might return an empty list if execution succeeded
+     * without any results
+     * @HTTP 400 Invalid fulltext or query parameters
+     */
+    @GET
+    @Path("/basic")
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasSearchResult searchUsingBasic(@QueryParam("query") String query,
+                                              @QueryParam("type") String type,
+                                              @QueryParam("classification") String classification,
+                                              @QueryParam("limit") int limit,
+                                              @QueryParam("offset") int offset) throws AtlasBaseException {
+
+        AtlasSearchResult ret = atlasDiscoveryService.searchUsingBasicQuery(query, type, classification, limit, offset);
+
+        return ret;
+    }
 }
\ No newline at end of file