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/07/31 21:21:49 UTC

atlas git commit: ATLAS-2008: basic-search gremlin query updated to use bind varibles, to avoid ClassCastException

Repository: atlas
Updated Branches:
  refs/heads/master 9b72de980 -> e75473a3b


ATLAS-2008: basic-search gremlin query updated to use bind varibles, to avoid ClassCastException

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


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

Branch: refs/heads/master
Commit: e75473a3babb61cf7530fd42ed4ff6fd133c574c
Parents: 9b72de9
Author: Madhan Neethiraj <ma...@apache.org>
Authored: Sun Jul 30 09:45:51 2017 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Sun Jul 30 11:46:01 2017 -0700

----------------------------------------------------------------------
 .../ClassificationSearchProcessor.java          | 43 +++++----
 .../atlas/discovery/EntitySearchProcessor.java  | 46 +++++-----
 .../apache/atlas/discovery/SearchProcessor.java | 91 +++++++++++---------
 3 files changed, 92 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/atlas/blob/e75473a3/repository/src/main/java/org/apache/atlas/discovery/ClassificationSearchProcessor.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/ClassificationSearchProcessor.java b/repository/src/main/java/org/apache/atlas/discovery/ClassificationSearchProcessor.java
index f94d24b..74197ca 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/ClassificationSearchProcessor.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/ClassificationSearchProcessor.java
@@ -18,7 +18,6 @@
 package org.apache.atlas.discovery;
 
 import org.apache.atlas.exception.AtlasBaseException;
-import org.apache.atlas.model.discovery.SearchParameters;
 import org.apache.atlas.model.discovery.SearchParameters.FilterCriteria;
 import org.apache.atlas.model.instance.AtlasEntity;
 import org.apache.atlas.repository.Constants;
@@ -33,7 +32,6 @@ import org.apache.atlas.type.AtlasClassificationType;
 import org.apache.atlas.util.AtlasGremlinQueryProvider;
 import org.apache.atlas.utils.AtlasPerfTracer;
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,30 +63,30 @@ public class ClassificationSearchProcessor extends SearchProcessor {
         final FilterCriteria          filterCriteria        = context.getSearchParameters().getTagFilters();
         final Set<String>             typeAndSubTypes       = classificationType.getTypeAndAllSubTypes();
         final String                  typeAndSubTypesQryStr = classificationType.getTypeAndAllSubTypesQryStr();
-        final Set<String>             solrAttributes        = new HashSet<>();
-        final Set<String>             gremlinAttributes     = new HashSet<>();
+        final Set<String>             indexAttributes       = new HashSet<>();
+        final Set<String>             graphAttributes       = new HashSet<>();
         final Set<String>             allAttributes         = new HashSet<>();
 
 
-        processSearchAttributes(classificationType, filterCriteria, solrAttributes, gremlinAttributes, allAttributes);
+        processSearchAttributes(classificationType, filterCriteria, indexAttributes, graphAttributes, allAttributes);
 
-        // for classification search, if any attribute can't be handled by Solr - switch to all Gremlin
-        boolean useSolrSearch = typeAndSubTypesQryStr.length() <= MAX_QUERY_STR_LENGTH_TAGS && CollectionUtils.isEmpty(gremlinAttributes) && canApplySolrFilter(classificationType, filterCriteria, false);
+        // for classification search, if any attribute can't be handled by index query - switch to all filter by Graph query
+        boolean useIndexSearch = typeAndSubTypesQryStr.length() <= MAX_QUERY_STR_LENGTH_TAGS && CollectionUtils.isEmpty(graphAttributes) && canApplyIndexFilter(classificationType, filterCriteria, false);
 
         AtlasGraph graph = context.getGraph();
 
-        if (useSolrSearch) {
-            StringBuilder solrQuery = new StringBuilder();
+        if (useIndexSearch) {
+            StringBuilder indexQuery = new StringBuilder();
 
-            constructTypeTestQuery(solrQuery, typeAndSubTypesQryStr);
-            constructFilterQuery(solrQuery, classificationType, filterCriteria, solrAttributes);
+            constructTypeTestQuery(indexQuery, typeAndSubTypesQryStr);
+            constructFilterQuery(indexQuery, classificationType, filterCriteria, indexAttributes);
 
-            String solrQueryString = STRAY_AND_PATTERN.matcher(solrQuery).replaceAll(")");
+            String indexQueryString = STRAY_AND_PATTERN.matcher(indexQuery).replaceAll(")");
 
-            solrQueryString = STRAY_OR_PATTERN.matcher(solrQueryString).replaceAll(")");
-            solrQueryString = STRAY_ELIPSIS_PATTERN.matcher(solrQueryString).replaceAll("");
+            indexQueryString = STRAY_OR_PATTERN.matcher(indexQueryString).replaceAll(")");
+            indexQueryString = STRAY_ELIPSIS_PATTERN.matcher(indexQueryString).replaceAll("");
 
-            indexQuery = graph.indexQuery(Constants.VERTEX_INDEX, solrQueryString);
+            this.indexQuery = graph.indexQuery(Constants.VERTEX_INDEX, indexQueryString);
         } else {
             indexQuery = null;
         }
@@ -101,28 +99,29 @@ public class ClassificationSearchProcessor extends SearchProcessor {
             // Now filter on the tag attributes
             AtlasGremlinQueryProvider queryProvider = AtlasGremlinQueryProvider.INSTANCE;
 
+            gremlinQueryBindings = new HashMap<>();
+
             StringBuilder gremlinQuery = new StringBuilder();
             gremlinQuery.append("g.V().has('__guid', T.in, guids)");
             gremlinQuery.append(queryProvider.getQuery(AtlasGremlinQueryProvider.AtlasGremlinQuery.BASIC_SEARCH_CLASSIFICATION_FILTER));
             gremlinQuery.append(".as('e').out()");
             gremlinQuery.append(queryProvider.getQuery(AtlasGremlinQueryProvider.AtlasGremlinQuery.BASIC_SEARCH_TYPE_FILTER));
 
-            constructGremlinFilterQuery(gremlinQuery, context.getClassificationType(), context.getSearchParameters().getTagFilters());
+            constructGremlinFilterQuery(gremlinQuery, gremlinQueryBindings, context.getClassificationType(), context.getSearchParameters().getTagFilters());
             // After filtering on tags go back to e and output the list of entity vertices
             gremlinQuery.append(".back('e').toList()");
 
-            gremlinTagFilterQuery = gremlinQuery.toString();
-
-            gremlinQueryBindings = new HashMap<>();
             gremlinQueryBindings.put("traitNames", typeAndSubTypes);
             gremlinQueryBindings.put("typeNames", typeAndSubTypes); // classification typeName
 
+            gremlinTagFilterQuery = gremlinQuery.toString();
+
             if (LOG.isDebugEnabled()) {
                 LOG.debug("gremlinTagFilterQuery={}", gremlinTagFilterQuery);
             }
         } else {
             gremlinTagFilterQuery = null;
-            gremlinQueryBindings = null;
+            gremlinQueryBindings  = null;
         }
     }
 
@@ -170,7 +169,7 @@ public class ClassificationSearchProcessor extends SearchProcessor {
                 if (indexQuery != null) {
                     Iterator<AtlasIndexQuery.Result> queryResult = indexQuery.vertices(qryOffset, limit);
 
-                    if (!queryResult.hasNext()) { // no more results from solr - end of search
+                    if (!queryResult.hasNext()) { // no more results from index query - end of search
                         break;
                     }
 
@@ -259,7 +258,7 @@ public class ClassificationSearchProcessor extends SearchProcessor {
                 }
 
             } catch (AtlasBaseException | ScriptException e) {
-                LOG.warn(e.getMessage());
+                LOG.warn(e.getMessage(), e);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/e75473a3/repository/src/main/java/org/apache/atlas/discovery/EntitySearchProcessor.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/EntitySearchProcessor.java b/repository/src/main/java/org/apache/atlas/discovery/EntitySearchProcessor.java
index 59fed17..9cd83fb 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/EntitySearchProcessor.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntitySearchProcessor.java
@@ -50,50 +50,50 @@ public class EntitySearchProcessor extends SearchProcessor {
         final FilterCriteria  filterCriteria        = context.getSearchParameters().getEntityFilters();
         final Set<String>     typeAndSubTypes       = entityType.getTypeAndAllSubTypes();
         final String          typeAndSubTypesQryStr = entityType.getTypeAndAllSubTypesQryStr();
-        final Set<String>     solrAttributes        = new HashSet<>();
-        final Set<String>     gremlinAttributes     = new HashSet<>();
+        final Set<String>     indexAttributes       = new HashSet<>();
+        final Set<String>     graphAttributes       = new HashSet<>();
         final Set<String>     allAttributes         = new HashSet<>();
 
         final AtlasClassificationType classificationType   = context.getClassificationType();
         final boolean                 filterClassification = classificationType != null && !context.needClassificationProcessor();
 
 
-        processSearchAttributes(entityType, filterCriteria, solrAttributes, gremlinAttributes, allAttributes);
+        processSearchAttributes(entityType, filterCriteria, indexAttributes, graphAttributes, allAttributes);
 
-        final boolean typeSearchBySolr = !filterClassification && typeAndSubTypesQryStr.length() <= MAX_QUERY_STR_LENGTH_TYPES;
-        final boolean attrSearchBySolr = !filterClassification && CollectionUtils.isNotEmpty(solrAttributes) && canApplySolrFilter(entityType, filterCriteria, false);
+        final boolean typeSearchByIndex = !filterClassification && typeAndSubTypesQryStr.length() <= MAX_QUERY_STR_LENGTH_TYPES;
+        final boolean attrSearchByIndex = !filterClassification && CollectionUtils.isNotEmpty(indexAttributes) && canApplyIndexFilter(entityType, filterCriteria, false);
 
-        StringBuilder solrQuery = new StringBuilder();
+        StringBuilder indexQuery = new StringBuilder();
 
-        if (typeSearchBySolr) {
-            constructTypeTestQuery(solrQuery, typeAndSubTypesQryStr);
+        if (typeSearchByIndex) {
+            constructTypeTestQuery(indexQuery, typeAndSubTypesQryStr);
         }
 
-        if (attrSearchBySolr) {
-            constructFilterQuery(solrQuery, entityType, filterCriteria, solrAttributes);
+        if (attrSearchByIndex) {
+            constructFilterQuery(indexQuery, entityType, filterCriteria, indexAttributes);
         } else {
-            gremlinAttributes.addAll(solrAttributes);
+            graphAttributes.addAll(indexAttributes);
         }
 
-        if (solrQuery.length() > 0) {
+        if (indexQuery.length() > 0) {
             if (context.getSearchParameters().getExcludeDeletedEntities()) {
-                constructStateTestQuery(solrQuery);
+                constructStateTestQuery(indexQuery);
             }
 
-            String solrQueryString = STRAY_AND_PATTERN.matcher(solrQuery).replaceAll(")");
+            String indexQueryString = STRAY_AND_PATTERN.matcher(indexQuery).replaceAll(")");
 
-            solrQueryString = STRAY_OR_PATTERN.matcher(solrQueryString).replaceAll(")");
-            solrQueryString = STRAY_ELIPSIS_PATTERN.matcher(solrQueryString).replaceAll("");
+            indexQueryString = STRAY_OR_PATTERN.matcher(indexQueryString).replaceAll(")");
+            indexQueryString = STRAY_ELIPSIS_PATTERN.matcher(indexQueryString).replaceAll("");
 
-            indexQuery = context.getGraph().indexQuery(Constants.VERTEX_INDEX, solrQueryString);
+            this.indexQuery = context.getGraph().indexQuery(Constants.VERTEX_INDEX, indexQueryString);
         } else {
-            indexQuery = null;
+            this.indexQuery = null;
         }
 
-        if (CollectionUtils.isNotEmpty(gremlinAttributes) || !typeSearchBySolr) {
+        if (CollectionUtils.isNotEmpty(graphAttributes) || !typeSearchByIndex) {
             AtlasGraphQuery query = context.getGraph().query();
 
-            if (!typeSearchBySolr) {
+            if (!typeSearchByIndex) {
                 query.in(Constants.TYPE_NAME_PROPERTY_KEY, typeAndSubTypes);
             }
 
@@ -101,9 +101,9 @@ public class EntitySearchProcessor extends SearchProcessor {
                 query.in(Constants.TRAIT_NAMES_PROPERTY_KEY, classificationType.getTypeAndAllSubTypes());
             }
 
-            graphQuery = toGraphFilterQuery(entityType, filterCriteria, gremlinAttributes, query);
+            graphQuery = toGraphFilterQuery(entityType, filterCriteria, graphAttributes, query);
 
-            if (context.getSearchParameters().getExcludeDeletedEntities() && indexQuery == null) {
+            if (context.getSearchParameters().getExcludeDeletedEntities() && this.indexQuery == null) {
                 graphQuery.has(Constants.STATE_PROPERTY_KEY, "ACTIVE");
             }
         } else {
@@ -161,7 +161,7 @@ public class EntitySearchProcessor extends SearchProcessor {
                 if (indexQuery != null) {
                     Iterator<AtlasIndexQuery.Result> idxQueryResult = indexQuery.vertices(qryOffset, limit);
 
-                    if (!idxQueryResult.hasNext()) { // no more results from solr - end of search
+                    if (!idxQueryResult.hasNext()) { // no more results from index query - end of search
                         break;
                     }
 

http://git-wip-us.apache.org/repos/asf/atlas/blob/e75473a3/repository/src/main/java/org/apache/atlas/discovery/SearchProcessor.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/SearchProcessor.java b/repository/src/main/java/org/apache/atlas/discovery/SearchProcessor.java
index 31d145c..b209ecb 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/SearchProcessor.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/SearchProcessor.java
@@ -96,7 +96,7 @@ public abstract class SearchProcessor {
     }
 
 
-    protected void processSearchAttributes(AtlasStructType structType, FilterCriteria filterCriteria, Set<String> solrFiltered, Set<String> gremlinFiltered, Set<String> allAttributes) {
+    protected void processSearchAttributes(AtlasStructType structType, FilterCriteria filterCriteria, Set<String> indexFiltered, Set<String> graphFiltered, Set<String> allAttributes) {
         if (structType == null || filterCriteria == null) {
             return;
         }
@@ -106,7 +106,7 @@ public abstract class SearchProcessor {
 
         if (filterCondition != null && CollectionUtils.isNotEmpty(criterion)) {
             for (SearchParameters.FilterCriteria criteria : criterion) {
-                processSearchAttributes(structType, criteria, solrFiltered, gremlinFiltered, allAttributes);
+                processSearchAttributes(structType, criteria, indexFiltered, graphFiltered, allAttributes);
             }
         } else if (StringUtils.isNotEmpty(filterCriteria.getAttributeName())) {
             try {
@@ -115,11 +115,11 @@ public abstract class SearchProcessor {
                 Set<String> indexedKeys   = context.getIndexedKeys();
 
                 if (indexedKeys != null && indexedKeys.contains(qualifiedName)) {
-                    solrFiltered.add(attributeName);
+                    indexFiltered.add(attributeName);
                 } else {
                     LOG.warn("search includes non-indexed attribute '{}'; might cause poor performance", qualifiedName);
 
-                    gremlinFiltered.add(attributeName);
+                    graphFiltered.add(attributeName);
                 }
 
                 if (structType instanceof AtlasEntityType) {
@@ -136,16 +136,16 @@ public abstract class SearchProcessor {
 
     //
     // If filterCriteria contains any non-indexed attribute inside OR condition:
-    //    Solr+Grelin can't be used. Need to use only Gremlin filter for all attributes. Examples:
+    //    Index+Graph can't be used. Need to use only Graph query filter for all attributes. Examples:
     //    (OR idx-att1=x non-idx-attr=z)
     //    (AND idx-att1=x (OR idx-attr2=y non-idx-attr=z))
     // Else
-    //    Solr can be used for indexed-attribute filtering and Gremlin for non-indexed attributes. Examples:
+    //    Index query can be used for indexed-attribute filtering and Graph query for non-indexed attributes. Examples:
     //      (AND idx-att1=x idx-attr2=y non-idx-attr=z)
     //      (AND (OR idx-att1=x idx-attr1=y) non-idx-attr=z)
     //      (AND (OR idx-att1=x idx-attr1=y) non-idx-attr=z (AND idx-attr2=xyz idx-attr2=abc))
     //
-    protected boolean canApplySolrFilter(AtlasStructType structType, FilterCriteria filterCriteria, boolean insideOrCondition) {
+    protected boolean canApplyIndexFilter(AtlasStructType structType, FilterCriteria filterCriteria, boolean insideOrCondition) {
         if (filterCriteria == null) {
             return true;
         }
@@ -161,7 +161,7 @@ public abstract class SearchProcessor {
 
             // If we have nested criterion let's find any nested ORs with non-indexed attr
             for (FilterCriteria criteria : criterion) {
-                ret = canApplySolrFilter(structType, criteria, insideOrCondition);
+                ret = canApplyIndexFilter(structType, criteria, insideOrCondition);
 
                 if (!ret) {
                     break;
@@ -182,33 +182,33 @@ public abstract class SearchProcessor {
         return ret;
     }
 
-    protected void constructTypeTestQuery(StringBuilder solrQuery, String typeAndAllSubTypesQryStr) {
+    protected void constructTypeTestQuery(StringBuilder indexQuery, String typeAndAllSubTypesQryStr) {
         if (StringUtils.isNotEmpty(typeAndAllSubTypesQryStr)) {
-            if (solrQuery.length() > 0) {
-                solrQuery.append(AND_STR);
+            if (indexQuery.length() > 0) {
+                indexQuery.append(AND_STR);
             }
 
-            solrQuery.append("v.\"").append(Constants.TYPE_NAME_PROPERTY_KEY).append("\":").append(typeAndAllSubTypesQryStr);
+            indexQuery.append("v.\"").append(Constants.TYPE_NAME_PROPERTY_KEY).append("\":").append(typeAndAllSubTypesQryStr);
         }
     }
 
-    protected void constructFilterQuery(StringBuilder solrQuery, AtlasStructType type, FilterCriteria filterCriteria, Set<String> solrAttributes) {
+    protected void constructFilterQuery(StringBuilder indexQuery, AtlasStructType type, FilterCriteria filterCriteria, Set<String> indexAttributes) {
         if (filterCriteria != null) {
             LOG.debug("Processing Filters");
 
-            String filterQuery = toSolrQuery(type, filterCriteria, solrAttributes, 0);
+            String filterQuery = toIndexQuery(type, filterCriteria, indexAttributes, 0);
 
             if (StringUtils.isNotEmpty(filterQuery)) {
-                if (solrQuery.length() > 0) {
-                    solrQuery.append(AND_STR);
+                if (indexQuery.length() > 0) {
+                    indexQuery.append(AND_STR);
                 }
 
-                solrQuery.append(filterQuery);
+                indexQuery.append(filterQuery);
             }
         }
     }
 
-    protected void constructGremlinFilterQuery(StringBuilder tagFilterQuery, AtlasStructType structType, FilterCriteria filterCriteria) {
+    protected void constructGremlinFilterQuery(StringBuilder gremlinQuery, Map<String, Object> queryBindings, AtlasStructType structType, FilterCriteria filterCriteria) {
         if (filterCriteria != null) {
             FilterCriteria.Condition condition = filterCriteria.getCondition();
 
@@ -223,16 +223,16 @@ public abstract class SearchProcessor {
                     if (condition == FilterCriteria.Condition.OR) {
                         StringBuilder nestedOrQuery = new StringBuilder("_()");
 
-                        constructGremlinFilterQuery(nestedOrQuery, structType, criteria);
+                        constructGremlinFilterQuery(nestedOrQuery, queryBindings, structType, criteria);
 
                         orQuery.append(i == 0 ? "" : ",").append(nestedOrQuery);
                     } else {
-                        constructGremlinFilterQuery(tagFilterQuery, structType, criteria);
+                        constructGremlinFilterQuery(gremlinQuery, queryBindings, structType, criteria);
                     }
                 }
 
                 if (condition == FilterCriteria.Condition.OR) {
-                    tagFilterQuery.append(".or(").append(orQuery).append(")");
+                    gremlinQuery.append(".or(").append(orQuery).append(")");
                 }
             } else {
                 String         attributeName = filterCriteria.getAttributeName();
@@ -242,7 +242,7 @@ public abstract class SearchProcessor {
                     SearchParameters.Operator operator       = filterCriteria.getOperator();
                     String                    attributeValue = filterCriteria.getAttributeValue();
 
-                    tagFilterQuery.append(toGremlinComparisonQuery(attribute, operator, attributeValue));
+                    gremlinQuery.append(toGremlinComparisonQuery(attribute, operator, attributeValue, queryBindings));
                 } else {
                     LOG.warn("Ignoring unknown attribute {}.{}", structType.getTypeName(), attributeName);
                 }
@@ -251,30 +251,30 @@ public abstract class SearchProcessor {
         }
     }
 
-    protected void constructStateTestQuery(StringBuilder solrQuery) {
-        if (solrQuery.length() > 0) {
-            solrQuery.append(AND_STR);
+    protected void constructStateTestQuery(StringBuilder indexQuery) {
+        if (indexQuery.length() > 0) {
+            indexQuery.append(AND_STR);
         }
 
-        solrQuery.append("v.\"").append(Constants.STATE_PROPERTY_KEY).append("\":ACTIVE");
+        indexQuery.append("v.\"").append(Constants.STATE_PROPERTY_KEY).append("\":ACTIVE");
     }
 
-    private String toSolrQuery(AtlasStructType type, FilterCriteria criteria, Set<String> solrAttributes, int level) {
-        return toSolrQuery(type, criteria, solrAttributes, new StringBuilder(), level);
+    private String toIndexQuery(AtlasStructType type, FilterCriteria criteria, Set<String> indexAttributes, int level) {
+        return toIndexQuery(type, criteria, indexAttributes, new StringBuilder(), level);
     }
 
-    private String toSolrQuery(AtlasStructType type, FilterCriteria criteria, Set<String> solrAttributes, StringBuilder sb, int level) {
+    private String toIndexQuery(AtlasStructType type, FilterCriteria criteria, Set<String> indexAttributes, StringBuilder sb, int level) {
         if (criteria.getCondition() != null && CollectionUtils.isNotEmpty(criteria.getCriterion())) {
             StringBuilder nestedExpression = new StringBuilder();
 
             for (FilterCriteria filterCriteria : criteria.getCriterion()) {
-                String nestedQuery = toSolrQuery(type, filterCriteria, solrAttributes, level + 1);
+                String nestedQuery = toIndexQuery(type, filterCriteria, indexAttributes, level + 1);
 
                 if (StringUtils.isNotEmpty(nestedQuery)) {
                     if (nestedExpression.length() > 0) {
                         nestedExpression.append(SPACE_STRING).append(criteria.getCondition()).append(SPACE_STRING);
                     }
-                    // todo: when a neq operation is nested and occurs in the beginning of the query, solr has issues
+                    // todo: when a neq operation is nested and occurs in the beginning of the query, index query has issues
                     nestedExpression.append(nestedQuery);
                 }
             }
@@ -284,14 +284,14 @@ public abstract class SearchProcessor {
             } else {
                 return nestedExpression.length() > 0 ? sb.append(BRACE_OPEN_STR).append(nestedExpression).append(BRACE_CLOSE_STR).toString() : EMPTY_STRING;
             }
-        } else if (solrAttributes.contains(criteria.getAttributeName())){
-            return toSolrExpression(type, criteria.getAttributeName(), criteria.getOperator(), criteria.getAttributeValue());
+        } else if (indexAttributes.contains(criteria.getAttributeName())){
+            return toIndexExpression(type, criteria.getAttributeName(), criteria.getOperator(), criteria.getAttributeValue());
         } else {
             return EMPTY_STRING;
         }
     }
 
-    private String toSolrExpression(AtlasStructType type, String attrName, SearchParameters.Operator op, String attrVal) {
+    private String toIndexExpression(AtlasStructType type, String attrName, SearchParameters.Operator op, String attrVal) {
         String ret = EMPTY_STRING;
 
         try {
@@ -307,12 +307,12 @@ public abstract class SearchProcessor {
         return ret;
     }
 
-    protected AtlasGraphQuery toGraphFilterQuery(AtlasStructType type, FilterCriteria criteria, Set<String> gremlinAttributes, AtlasGraphQuery query) {
+    protected AtlasGraphQuery toGraphFilterQuery(AtlasStructType type, FilterCriteria criteria, Set<String> graphAttributes, AtlasGraphQuery query) {
         if (criteria != null) {
             if (criteria.getCondition() != null) {
                 if (criteria.getCondition() == Condition.AND) {
                     for (FilterCriteria filterCriteria : criteria.getCriterion()) {
-                        AtlasGraphQuery nestedQuery = toGraphFilterQuery(type, filterCriteria, gremlinAttributes, context.getGraph().query());
+                        AtlasGraphQuery nestedQuery = toGraphFilterQuery(type, filterCriteria, graphAttributes, context.getGraph().query());
 
                         query.addConditionsFrom(nestedQuery);
                     }
@@ -320,7 +320,7 @@ public abstract class SearchProcessor {
                     List<AtlasGraphQuery> orConditions = new LinkedList<>();
 
                     for (FilterCriteria filterCriteria : criteria.getCriterion()) {
-                        AtlasGraphQuery nestedQuery = toGraphFilterQuery(type, filterCriteria, gremlinAttributes, context.getGraph().query());
+                        AtlasGraphQuery nestedQuery = toGraphFilterQuery(type, filterCriteria, graphAttributes, context.getGraph().query());
 
                         orConditions.add(context.getGraph().query().createChildQuery().addConditionsFrom(nestedQuery));
                     }
@@ -329,7 +329,7 @@ public abstract class SearchProcessor {
                         query.or(orConditions);
                     }
                 }
-            } else if (gremlinAttributes.contains(criteria.getAttributeName())) {
+            } else if (graphAttributes.contains(criteria.getAttributeName())) {
                 String                    attrName  = criteria.getAttributeName();
                 String                    attrValue = criteria.getAttributeValue();
                 SearchParameters.Operator operator  = criteria.getOperator();
@@ -374,7 +374,7 @@ public abstract class SearchProcessor {
                             break;
                     }
                 } catch (AtlasBaseException e) {
-                    LOG.error("toGremlinFilterQuery(): failed for attrName=" + attrName + "; operator=" + operator + "; attrValue=" + attrValue, e);
+                    LOG.error("toGraphFilterQuery(): failed for attrName=" + attrName + "; operator=" + operator + "; attrValue=" + attrValue, e);
                 }
             }
         }
@@ -382,7 +382,10 @@ public abstract class SearchProcessor {
         return query;
     }
 
-    private String toGremlinComparisonQuery(AtlasAttribute attribute, SearchParameters.Operator operator, String attrValue) {
+    private String toGremlinComparisonQuery(AtlasAttribute attribute, SearchParameters.Operator operator, String attrValue, Map<String, Object> queryBindings) {
+        String bindName  = "__bind_" + queryBindings.size();
+        Object bindValue = attribute.getAttributeType().getNormalizedValue(attrValue);
+
         AtlasGremlinQueryProvider queryProvider = AtlasGremlinQueryProvider.INSTANCE;
         String queryTemplate = null;
         switch (operator) {
@@ -419,11 +422,13 @@ public abstract class SearchProcessor {
         }
 
         if (org.apache.commons.lang3.StringUtils.isNotEmpty(queryTemplate)) {
-            if (StringUtils.equalsIgnoreCase(attribute.getAttributeType().getTypeName(), AtlasBaseTypeDef.ATLAS_TYPE_STRING)) {
-                attrValue = "'" + attrValue + "'";
+            if (bindValue instanceof Date) {
+                bindValue = ((Date)bindValue).getTime();
             }
 
-            return String.format(queryTemplate, attribute.getQualifiedName(), attrValue);
+            queryBindings.put(bindName, bindValue);
+
+            return String.format(queryTemplate, attribute.getQualifiedName(), bindName);
         } else {
             return EMPTY_STRING;
         }