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/06/30 09:00:00 UTC

[1/5] incubator-atlas git commit: ATLAS-1880: search API with support for entity/tag attribute filters

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 8101883cc -> 9a4ed469c


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/discovery/GremlinStep.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/GremlinStep.java b/repository/src/main/java/org/apache/atlas/discovery/GremlinStep.java
new file mode 100644
index 0000000..5565781
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/GremlinStep.java
@@ -0,0 +1,388 @@
+/**
+ * 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.SearchParameters;
+import org.apache.atlas.model.discovery.SearchParameters.FilterCriteria;
+import org.apache.atlas.model.discovery.SearchParameters.FilterCriteria.Condition;
+import org.apache.atlas.model.discovery.SearchParameters.Operator;
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasEdge;
+import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
+import org.apache.atlas.repository.graphdb.AtlasVertex;
+import org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.utils.AtlasPerfTracer;
+import org.apache.commons.collections.CollectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.inject.Inject;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.atlas.discovery.SearchPipeline.IndexResultType;
+import static org.apache.atlas.discovery.SearchPipeline.PipelineContext;
+import static org.apache.atlas.discovery.SearchPipeline.PipelineStep;
+import static org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import static org.apache.atlas.repository.graphdb.AtlasGraphQuery.MatchingOperator;
+
+@Component
+public class GremlinStep implements PipelineStep {
+    private static final Logger LOG      = LoggerFactory.getLogger(GremlinStep.class);
+    private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger("GremlinSearchStep");
+
+    private final AtlasGraph        graph;
+    private final AtlasTypeRegistry typeRegistry;
+
+    enum GremlinFilterQueryType { TAG, ENTITY }
+
+    @Inject
+    public GremlinStep(AtlasGraph graph, AtlasTypeRegistry typeRegistry) {
+        this.graph        = graph;
+        this.typeRegistry = typeRegistry;
+    }
+
+    @Override
+    public void execute(PipelineContext context) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> GremlinStep.execute({})", context);
+        }
+
+        if (context == null) {
+            throw new AtlasBaseException("Can't start search without any context");
+        }
+
+        AtlasPerfTracer perf = null;
+
+        if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) {
+            perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "GremlinSearchStep.execute(" + context +  ")");
+        }
+
+        final Iterator<AtlasVertex> result;
+
+        if (context.hasIndexResults()) {
+            // We have some results from the indexed step, let's proceed accordingly
+            if (context.getIndexResultType() == IndexResultType.TAG) {
+                // Index search was done on tag and filters
+                if (context.isTagProcessingComplete()) {
+                    LOG.debug("GremlinStep.execute(): index has completely processed tag, further TAG filtering not needed");
+
+                    Set<String> taggedVertexGUIDs = new HashSet<>();
+
+                    Iterator<AtlasIndexQuery.Result> tagVertexIterator = context.getIndexResultsIterator();
+
+                    while (tagVertexIterator.hasNext()) {
+                        // Find out which Vertex has this outgoing edge
+                        AtlasVertex         vertex = tagVertexIterator.next().getVertex();
+                        Iterable<AtlasEdge> edges  = vertex.getEdges(AtlasEdgeDirection.IN);
+
+                        for (AtlasEdge edge : edges) {
+                            String guid = AtlasGraphUtilsV1.getIdFromVertex(edge.getOutVertex());
+
+                            taggedVertexGUIDs.add(guid);
+                        }
+                    }
+
+                    // No entities are tagged  (actually this check is already done)
+                    if (!taggedVertexGUIDs.isEmpty()) {
+                        result = processEntity(taggedVertexGUIDs, context);
+                    } else {
+                        result = null;
+                    }
+                } else {
+                    result = processTagAndEntity(Collections.<String>emptySet(), context);
+                }
+            } else if (context.getIndexResultType() == IndexResultType.TEXT) {
+                // Index step processed full-text;
+                Set<String> entityIDs = getVertexIDs(context.getIndexResultsIterator());
+
+                result = processTagAndEntity(entityIDs, context);
+            } else if (context.getIndexResultType() == IndexResultType.ENTITY) {
+                // Index step processed entity and it's filters; tag filter wouldn't be set
+                Set<String> entityIDs = getVertexIDs(context.getIndexResultsIterator());
+
+                result = processEntity(entityIDs, context);
+            } else {
+                result = null;
+            }
+        } else {
+            // No index results, need full processing in Gremlin
+            if (context.getClassificationType() != null) {
+                // Process tag and filters first, then entity filters
+                result = processTagAndEntity(Collections.<String>emptySet(), context);
+            } else {
+                result = processEntity(Collections.<String>emptySet(), context);
+            }
+        }
+
+        context.setGremlinResultIterator(result);
+
+        AtlasPerfTracer.log(perf);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== GremlinStep.execute({})", context);
+        }
+    }
+
+    private Iterator<AtlasVertex> processEntity(Set<String> entityGUIDs, PipelineContext context) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> GremlinStep.processEntity(entityGUIDs={})", entityGUIDs);
+        }
+
+        final Iterator<AtlasVertex> ret;
+
+        SearchParameters searchParameters = context.getSearchParameters();
+        AtlasEntityType  entityType       = context.getEntityType();
+
+        if (entityType != null) {
+            AtlasGraphQuery entityFilterQuery = context.getGraphQuery("ENTITY_FILTER");
+
+            if (entityFilterQuery == null) {
+                entityFilterQuery = graph.query().in(Constants.TYPE_NAME_PROPERTY_KEY, entityType.getTypeAndAllSubTypes());
+
+                if (searchParameters.getEntityFilters() != null) {
+                    toGremlinFilterQuery(GremlinFilterQueryType.ENTITY, entityType, searchParameters.getEntityFilters(), entityFilterQuery, context);
+                }
+
+                if (searchParameters.getExcludeDeletedEntities()) {
+                    entityFilterQuery.has(Constants.STATE_PROPERTY_KEY, "ACTIVE");
+                }
+
+                context.cacheGraphQuery("ENTITY_FILTER", entityFilterQuery);
+            }
+
+            // Now get all vertices
+            if (CollectionUtils.isEmpty(entityGUIDs)) {
+                ret = entityFilterQuery.vertices(context.getCurrentOffset(), context.getMaxLimit()).iterator();
+            } else {
+                AtlasGraphQuery guidQuery = graph.query().in(Constants.GUID_PROPERTY_KEY, entityGUIDs);
+
+                if (entityFilterQuery != null) {
+                    guidQuery.addConditionsFrom(entityFilterQuery);
+                } else if (searchParameters.getExcludeDeletedEntities()) {
+                    guidQuery.has(Constants.STATE_PROPERTY_KEY, "ACTIVE");
+                }
+
+                ret = guidQuery.vertices(context.getMaxLimit()).iterator();
+            }
+        } else if (CollectionUtils.isNotEmpty(entityGUIDs)) {
+            AtlasGraphQuery guidQuery = graph.query().in(Constants.GUID_PROPERTY_KEY, entityGUIDs);
+
+            if (searchParameters.getExcludeDeletedEntities()) {
+                guidQuery.has(Constants.STATE_PROPERTY_KEY, "ACTIVE");
+            }
+
+            Iterable<AtlasVertex> vertices = guidQuery.vertices(context.getMaxLimit());
+
+            ret = vertices.iterator();
+        } else {
+            ret = null;
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== GremlinStep.processEntity(entityGUIDs={})", entityGUIDs);
+        }
+
+        return ret;
+    }
+
+    private Iterator<AtlasVertex> processTagAndEntity(Set<String> entityGUIDs, PipelineContext context) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> GremlinStep.processTagAndEntity(entityGUIDs={})", entityGUIDs);
+        }
+
+        final Iterator<AtlasVertex> ret;
+
+        AtlasClassificationType classificationType = context.getClassificationType();
+
+        if (classificationType != null) {
+            AtlasGraphQuery  tagVertexQuery = context.getGraphQuery("TAG_VERTEX");
+
+            if (tagVertexQuery == null) {
+                tagVertexQuery = graph.query().in(Constants.TYPE_NAME_PROPERTY_KEY, classificationType.getTypeAndAllSubTypes());
+
+                SearchParameters searchParameters = context.getSearchParameters();
+
+                // Do tag filtering first as it'll return a smaller subset of vertices
+                if (searchParameters.getTagFilters() != null) {
+                    toGremlinFilterQuery(GremlinFilterQueryType.TAG, classificationType, searchParameters.getTagFilters(), tagVertexQuery, context);
+                }
+
+                context.cacheGraphQuery("TAG_VERTEX", tagVertexQuery);
+            }
+
+            if (tagVertexQuery != null) {
+                Set<String> taggedVertexGuids = new HashSet<>();
+                // Now get all vertices after adjusting offset for each iteration
+                LOG.debug("Firing TAG query");
+
+                Iterator<AtlasVertex> tagVertexIterator = tagVertexQuery.vertices(context.getCurrentOffset(), context.getMaxLimit()).iterator();
+
+                while (tagVertexIterator.hasNext()) {
+                    // Find out which Vertex has this outgoing edge
+                    Iterable<AtlasEdge> edges = tagVertexIterator.next().getEdges(AtlasEdgeDirection.IN);
+                    for (AtlasEdge edge : edges) {
+                        String guid = AtlasGraphUtilsV1.getIdFromVertex(edge.getOutVertex());
+                        taggedVertexGuids.add(guid);
+                    }
+                }
+
+                entityGUIDs = taggedVertexGuids;
+            }
+        }
+
+        if (!entityGUIDs.isEmpty()) {
+            ret = processEntity(entityGUIDs, context);
+        } else {
+            ret = null;
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== GremlinStep.processTagAndEntity(entityGUIDs={})", entityGUIDs);
+        }
+
+        return ret;
+    }
+
+    private Set<String> getVertexIDs(Iterator<AtlasIndexQuery.Result> idxResultsIterator) {
+        Set<String> guids = new HashSet<>();
+        while (idxResultsIterator.hasNext()) {
+            AtlasVertex vertex = idxResultsIterator.next().getVertex();
+            String guid = AtlasGraphUtilsV1.getIdFromVertex(vertex);
+            guids.add(guid);
+        }
+        return guids;
+    }
+
+    private Set<String> getVertexIDs(Iterable<AtlasVertex> vertices) {
+        Set<String> guids = new HashSet<>();
+        for (AtlasVertex vertex : vertices) {
+            String guid = AtlasGraphUtilsV1.getIdFromVertex(vertex);
+            guids.add(guid);
+        }
+        return guids;
+    }
+
+    private AtlasGraphQuery toGremlinFilterQuery(GremlinFilterQueryType queryType, AtlasStructType type, FilterCriteria criteria,
+                                                 AtlasGraphQuery query, PipelineContext context) {
+        if (criteria.getCondition() != null) {
+            if (criteria.getCondition() == Condition.AND) {
+                for (FilterCriteria filterCriteria : criteria.getCriterion()) {
+                    AtlasGraphQuery nestedQuery = toGremlinFilterQuery(queryType, type, filterCriteria, graph.query(), context);
+                    query.addConditionsFrom(nestedQuery);
+                }
+            } else {
+                List<AtlasGraphQuery> orConditions = new LinkedList<>();
+
+                for (FilterCriteria filterCriteria : criteria.getCriterion()) {
+                    AtlasGraphQuery nestedQuery = toGremlinFilterQuery(queryType, type, filterCriteria, graph.query(), context);
+                    // FIXME: Something might not be right here as the queries are getting overwritten sometimes
+                    orConditions.add(graph.query().createChildQuery().addConditionsFrom(nestedQuery));
+                }
+
+                if (!orConditions.isEmpty()) {
+                    query.or(orConditions);
+                }
+            }
+        } else {
+            String   attrName  = criteria.getAttributeName();
+            String   attrValue = criteria.getAttributeValue();
+            Operator operator  = criteria.getOperator();
+
+            try {
+                // If attribute belongs to supertype then adjust the name accordingly
+                final String  qualifiedAttributeName;
+                final boolean attrProcessed;
+
+                if (queryType == GremlinFilterQueryType.TAG) {
+                    qualifiedAttributeName = type.getQualifiedAttributeName(attrName);
+                    attrProcessed          = context.hasProcessedTagAttribute(qualifiedAttributeName);
+                } else {
+                    qualifiedAttributeName = type.getQualifiedAttributeName(attrName);
+                    attrProcessed          = context.hasProcessedEntityAttribute(qualifiedAttributeName);
+                }
+
+                // Check if the qualifiedAttribute has been processed
+                if (!attrProcessed) {
+                    switch (operator) {
+                        case LT:
+                            query.has(qualifiedAttributeName, ComparisionOperator.LESS_THAN, attrValue);
+                            break;
+                        case LTE:
+                            query.has(qualifiedAttributeName, ComparisionOperator.LESS_THAN_EQUAL, attrValue);
+                            break;
+                        case GT:
+                            query.has(qualifiedAttributeName, ComparisionOperator.GREATER_THAN, attrValue);
+                            break;
+                        case GTE:
+                            query.has(qualifiedAttributeName, ComparisionOperator.GREATER_THAN_EQUAL, attrValue);
+                            break;
+                        case EQ:
+                            query.has(qualifiedAttributeName, ComparisionOperator.EQUAL, attrValue);
+                            break;
+                        case NEQ:
+                            query.has(qualifiedAttributeName, ComparisionOperator.NOT_EQUAL, attrValue);
+                            break;
+                        case LIKE:
+                            // TODO: Maybe we need to validate pattern
+                            query.has(qualifiedAttributeName, MatchingOperator.REGEX, getLikeRegex(attrValue));
+                            break;
+                        case CONTAINS:
+                            query.has(qualifiedAttributeName, MatchingOperator.REGEX, getContainsRegex(attrValue));
+                            break;
+                        case STARTS_WITH:
+                            query.has(qualifiedAttributeName, MatchingOperator.PREFIX, attrValue);
+                            break;
+                        case ENDS_WITH:
+                            query.has(qualifiedAttributeName, MatchingOperator.REGEX, getSuffixRegex(attrValue));
+                        case IN:
+                            LOG.warn("{}: unsupported operator. Ignored", operator);
+                            break;
+                    }
+                }
+            } catch (AtlasBaseException e) {
+                LOG.error("toGremlinFilterQuery(): failed for attrName=" + attrName + "; operator=" + operator + "; attrValue=" + attrValue, e);
+            }
+        }
+
+        return query;
+    }
+
+    private String getContainsRegex(String attributeValue) {
+        return ".*" + attributeValue + ".*";
+    }
+
+    private String getSuffixRegex(String attributeValue) {
+        return ".*" + attributeValue;
+    }
+
+    private String getLikeRegex(String attributeValue) { return ".*" + attributeValue + ".*"; }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/discovery/SearchPipeline.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/SearchPipeline.java b/repository/src/main/java/org/apache/atlas/discovery/SearchPipeline.java
new file mode 100644
index 0000000..0f91b2d
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/SearchPipeline.java
@@ -0,0 +1,611 @@
+/**
+ * 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.SearchParameters;
+import org.apache.atlas.model.discovery.SearchParameters.FilterCriteria;
+import org.apache.atlas.model.discovery.SearchParameters.FilterCriteria.Condition;
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
+import org.apache.atlas.repository.graphdb.*;
+import org.apache.atlas.type.AtlasClassificationType;
+import org.apache.atlas.type.AtlasEntityType;
+import org.apache.atlas.type.AtlasStructType;
+import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.util.SearchTracker;
+import org.apache.atlas.utils.AtlasPerfTracer;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Component
+public class SearchPipeline {
+    private static final Logger LOG      = LoggerFactory.getLogger(SearchPipeline.class);
+    private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger("SearchPipeline");
+
+    enum ExecutionMode { SOLR, GREMLIN, MIXED }
+
+    enum IndexResultType { TAG, ENTITY, TEXT }
+
+    private final SolrStep                 solrStep;
+    private final GremlinStep              gremlinStep;
+    private final SearchTracker            searchTracker;
+    private final AtlasTypeRegistry        typeRegistry;
+    private final Configuration            atlasConfiguration;
+    private final GraphBackedSearchIndexer indexer;
+
+    @Inject
+    public SearchPipeline(SolrStep solrStep, GremlinStep gremlinStep, SearchTracker searchTracker, AtlasTypeRegistry typeRegistry, Configuration atlasConfiguration, GraphBackedSearchIndexer indexer) {
+        this.solrStep           = solrStep;
+        this.gremlinStep        = gremlinStep;
+        this.searchTracker      = searchTracker;
+        this.typeRegistry       = typeRegistry;
+        this.atlasConfiguration = atlasConfiguration;
+        this.indexer            = indexer;
+    }
+
+    public List<AtlasVertex> run(SearchParameters searchParameters) throws AtlasBaseException {
+        final List<AtlasVertex> ret;
+
+        AtlasPerfTracer perf = null;
+
+        if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) {
+            perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "SearchPipeline.run("+ searchParameters +")");
+        }
+
+        AtlasEntityType         entityType = typeRegistry.getEntityTypeByName(searchParameters.getTypeName());
+        AtlasClassificationType classiType = typeRegistry.getClassificationTypeByName(searchParameters.getClassification());
+        PipelineContext         context    = new PipelineContext(searchParameters, entityType, classiType, indexer.getVertexIndexKeys());
+        String                  searchId   = searchTracker.add(context); // For future cancellation
+
+        try {
+            ExecutionMode mode = determineExecutionMode(context);
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Execution mode {}", mode);
+            }
+
+            switch (mode) {
+                case SOLR:
+                    ret = runOnlySolr(context);
+                    break;
+
+                case GREMLIN:
+                    ret = runOnlyGremlin(context);
+                    break;
+
+                case MIXED:
+                    ret = runMixed(context);
+                    break;
+
+                default:
+                    ret = Collections.emptyList();
+            }
+        } finally {
+            searchTracker.remove(searchId);
+
+            AtlasPerfTracer.log(perf);
+        }
+
+        return ret;
+    }
+
+    private List<AtlasVertex> runOnlySolr(PipelineContext context) throws AtlasBaseException {
+        // Only when there's no tag and query
+        List<AtlasVertex> results = new ArrayList<>();
+
+        while (results.size() < context.getSearchParameters().getLimit()) {
+            if (context.getForceTerminate()) {
+                LOG.debug("search has been terminated");
+
+                break;
+            }
+
+            // Execute solr search only
+            solrStep.execute(context);
+
+            List<AtlasVertex> stepResults = getIndexResults(context);
+
+            context.incrementSearchRound();
+
+            addToResult(results, stepResults, context.getSearchParameters().getLimit());
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Pipeline iteration {}: stepResults={}; totalResult={}", context.getIterationCount(), stepResults.size(), results.size());
+            }
+
+            if (CollectionUtils.isEmpty(stepResults)) {
+                // If no result is found any subsequent iteration then just stop querying the index
+                break;
+            }
+        }
+
+        if (context.getIndexResultType() == IndexResultType.TAG) {
+            List<AtlasVertex> entityVertices = new ArrayList<>(results.size());
+
+            for (AtlasVertex tagVertex : results) {
+                Iterable<AtlasEdge> edges = tagVertex.getEdges(AtlasEdgeDirection.IN);
+
+                for (AtlasEdge edge : edges) {
+                    AtlasVertex entityVertex = edge.getOutVertex();
+
+                    entityVertices.add(entityVertex);
+                }
+            }
+
+            results = entityVertices;
+        }
+
+        return results;
+    }
+
+    private List<AtlasVertex> runOnlyGremlin(PipelineContext context) throws AtlasBaseException {
+        List<AtlasVertex> results = new ArrayList<>();
+
+        while (results.size() < context.getSearchParameters().getLimit()) {
+            if (context.getForceTerminate()) {
+                LOG.debug("search has been terminated");
+
+                break;
+            }
+
+            gremlinStep.execute(context);
+
+            List<AtlasVertex> stepResults = getGremlinResults(context);
+
+            context.incrementSearchRound();
+
+            addToResult(results, stepResults, context.getSearchParameters().getLimit());
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Pipeline iteration {}: stepResults={}; totalResult={}", context.getIterationCount(), stepResults.size(), results.size());
+            }
+
+            if (CollectionUtils.isEmpty(stepResults)) {
+                // If no result is found any subsequent iteration then just stop querying the index
+                break;
+            }
+        }
+
+        return results;
+    }
+
+    /*
+        1. Index processes few attributes and then gremlin processes rest
+            1.1 Iterate for gremlin till the index results are non null
+        2. Index processes all attributes, gremlin has nothing to do
+
+        Sometimes the result set might be less than the max limit and we need to iterate until the result set is full
+        or the iteration doesn't return any results
+
+     */
+    private List<AtlasVertex> runMixed(PipelineContext context) throws AtlasBaseException {
+        List<AtlasVertex> results = new ArrayList<>();
+
+        while (results.size() < context.getSearchParameters().getLimit()) {
+            if (context.getForceTerminate()) {
+                LOG.debug("search has been terminated");
+
+                break;
+            }
+
+            // Execute Solr search and then pass it to the Gremlin step (if needed)
+            solrStep.execute(context);
+
+            if (!context.hasIndexResults()) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("No index results in iteration {}", context.getIterationCount());
+                }
+
+                // If no result is found any subsequent iteration then just stop querying the index
+                break;
+            }
+
+            // Attributes partially processed by Solr, use gremlin to process remaining attribute(s)
+            gremlinStep.execute(context);
+
+            context.incrementSearchRound();
+
+            List<AtlasVertex> stepResults = getGremlinResults(context);
+
+            addToResult(results, stepResults, context.getSearchParameters().getLimit());
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Pipeline iteration {}: stepResults={}; totalResult={}", context.getIterationCount(), stepResults.size(), results.size());
+            }
+        }
+
+        return results;
+    }
+
+    private void addToResult(List<AtlasVertex> result, List<AtlasVertex> stepResult, int maxLimit) {
+        if (result != null && stepResult != null && result.size() < maxLimit) {
+            for (AtlasVertex vertex : stepResult) {
+                result.add(vertex);
+
+                if (result.size() >= maxLimit) {
+                    break;
+                }
+            }
+        }
+    }
+
+    private List<AtlasVertex> getIndexResults(PipelineContext pipelineContext) {
+        List<AtlasVertex> ret = new ArrayList<>();
+
+        if (pipelineContext.hasIndexResults()) {
+            Iterator<AtlasIndexQuery.Result> iter = pipelineContext.getIndexResultsIterator();
+
+            while(iter.hasNext()) {
+                ret.add(iter.next().getVertex());
+            }
+        }
+
+        return ret;
+    }
+
+    private List<AtlasVertex> getGremlinResults(PipelineContext pipelineContext) {
+        List<AtlasVertex> ret = new ArrayList<>();
+
+        if (pipelineContext.hasGremlinResults()) {
+            Iterator<AtlasVertex> iter = pipelineContext.getGremlinResultIterator();
+
+            while (iter.hasNext()) {
+                ret.add(iter.next());
+            }
+        }
+
+        return ret;
+    }
+
+    private ExecutionMode determineExecutionMode(PipelineContext context) {
+        SearchParameters        searchParameters   = context.getSearchParameters();
+        AtlasClassificationType classificationType = context.getClassificationType();
+        AtlasEntityType         entityType         = context.getEntityType();
+        int                     solrCount          = 0;
+        int                     gremlinCount       = 0;
+
+        if (StringUtils.isNotEmpty(searchParameters.getQuery())) {
+            solrCount++;
+
+            // __state index only exists in vertex_index
+            if (searchParameters.getExcludeDeletedEntities()) {
+                gremlinCount++;
+            }
+        }
+
+        if (classificationType != null) {
+            Set<String> typeAndAllSubTypes = classificationType.getTypeAndAllSubTypes();
+
+            if (typeAndAllSubTypes.size() > atlasConfiguration.getInt(Constants.INDEX_SEARCH_MAX_TAGS_COUNT, 10)) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Classification type {} has too many subTypes ({}) to use in Solr search. Gremlin will be used to execute the search",
+                              classificationType.getTypeName(), typeAndAllSubTypes.size());
+                }
+
+                gremlinCount++;
+            } else {
+                if (hasNonIndexedAttrViolation(classificationType, context.getIndexedKeys(), searchParameters.getTagFilters())) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Tag filters not suitable for Solr search. Gremlin will be used to execute the search");
+                    }
+
+                    gremlinCount++;
+                } else {
+                    solrCount++;
+
+                    // __state index only exist in vertex_index
+                    if (searchParameters.getExcludeDeletedEntities()) {
+                        gremlinCount++;
+                    }
+                }
+            }
+        }
+
+        if (entityType != null) {
+            Set<String> typeAndAllSubTypes = entityType.getTypeAndAllSubTypes();
+
+            if (typeAndAllSubTypes.size() > atlasConfiguration.getInt(Constants.INDEX_SEARCH_MAX_TYPES_COUNT, 10)) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Entity type {} has too many subTypes ({}) to use in Solr search. Gremlin will be used to execute the search",
+                              entityType.getTypeName(), typeAndAllSubTypes.size());
+                }
+
+                gremlinCount++;
+            } else {
+                if (hasNonIndexedAttrViolation(entityType, context.getIndexedKeys(), searchParameters.getEntityFilters())) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Entity filters not suitable for Solr search. Gremlin will be used to execute the search");
+                    }
+
+                    gremlinCount++;
+                } else {
+                    solrCount++;
+                }
+            }
+        }
+
+        ExecutionMode mode = ExecutionMode.MIXED;
+
+        if (solrCount == 1 && gremlinCount == 0) {
+            mode = ExecutionMode.SOLR;
+        } else if (gremlinCount == 1 && solrCount == 0) {
+            mode = ExecutionMode.GREMLIN;
+        }
+
+        return mode;
+    }
+
+    // If Index can't process all attributes and any of the non-indexed attribute is present in OR nested within AND
+    // then the only way is Gremlin
+    // A violation (here) is defined as presence of non-indexed attribute within any OR clause nested under an AND clause
+    // the reason being that the index would not be able to process the nested OR attribute which might result in
+    // exclusion of valid result (vertex)
+    private boolean hasNonIndexedAttrViolation(AtlasStructType structType, Set<String> indexKeys, FilterCriteria filterCriteria) {
+        return hasNonIndexedAttrViolation(structType, indexKeys, filterCriteria, false);
+    }
+
+    private boolean hasNonIndexedAttrViolation(AtlasStructType structType, Set<String> indexKeys, FilterCriteria filterCriteria, boolean enclosedInOrCondition) {
+        if (filterCriteria == null) {
+            return false;
+        }
+
+        boolean              ret             = false;
+        Condition            filterCondition = filterCriteria.getCondition();
+        List<FilterCriteria> criterion       = filterCriteria.getCriterion();
+
+        if (filterCondition != null && CollectionUtils.isNotEmpty(criterion)) {
+            if (!enclosedInOrCondition) {
+                enclosedInOrCondition = filterCondition == Condition.OR;
+            }
+
+            // If we have nested criterion let's find any nested ORs with non-indexed attr
+            for (FilterCriteria criteria : criterion) {
+                ret |= hasNonIndexedAttrViolation(structType, indexKeys, criteria, enclosedInOrCondition);
+
+                if (ret) {
+                    break;
+                }
+            }
+        } else if (StringUtils.isNotEmpty(filterCriteria.getAttributeName())) {
+            // If attribute qualified name doesn't exist in the vertex index we potentially might have a problem
+            try {
+                String qualifiedAttributeName = structType.getQualifiedAttributeName(filterCriteria.getAttributeName());
+
+                ret = CollectionUtils.isEmpty(indexKeys) || !indexKeys.contains(qualifiedAttributeName);
+
+                if (ret) {
+                    LOG.warn("search includes non-indexed attribute '{}'; might cause poor performance", qualifiedAttributeName);
+                }
+            } catch (AtlasBaseException e) {
+                LOG.warn(e.getMessage());
+
+                ret = true;
+            }
+        }
+
+        // return ret && enclosedInOrCondition;
+
+        return ret;
+    }
+
+    public interface PipelineStep {
+        void execute(PipelineContext context) throws AtlasBaseException;
+    }
+
+    public static class PipelineContext {
+        // TODO: See if anything can be cached in the context
+
+        private final SearchParameters        searchParameters;
+        private final AtlasEntityType         entityType;
+        private final AtlasClassificationType classificationType;
+        private final Set<String>             indexedKeys;
+
+        private int     iterationCount;
+        private boolean forceTerminate;
+        private int     currentOffset;
+        private int     maxLimit;
+
+        // Continuous processing stuff
+        private Set<String> tagSearchAttributes       = new HashSet<>();
+        private Set<String> entitySearchAttributes    = new HashSet<>();
+        private Set<String> tagAttrProcessedBySolr    = new HashSet<>();
+        private Set<String> entityAttrProcessedBySolr = new HashSet<>();
+
+        // Results related stuff
+        private IndexResultType                  indexResultType;
+        private Iterator<AtlasIndexQuery.Result> indexResultsIterator;
+        private Iterator<AtlasVertex>            gremlinResultIterator;
+
+        private Map<String, AtlasIndexQuery> cachedIndexQueries = new HashMap<>();
+        private Map<String, AtlasGraphQuery> cachedGraphQueries = new HashMap<>();
+
+        public PipelineContext(SearchParameters searchParameters, AtlasEntityType entityType, AtlasClassificationType classificationType, Set<String> indexedKeys) {
+            this.searchParameters   = searchParameters;
+            this.entityType         = entityType;
+            this.classificationType = classificationType;
+            this.indexedKeys        = indexedKeys;
+
+            currentOffset = searchParameters.getOffset();
+            maxLimit      = searchParameters.getLimit();
+        }
+
+        public SearchParameters getSearchParameters() {
+            return searchParameters;
+        }
+
+        public AtlasEntityType getEntityType() {
+            return entityType;
+        }
+
+        public AtlasClassificationType getClassificationType() {
+            return classificationType;
+        }
+
+        public Set<String> getIndexedKeys() { return indexedKeys; }
+
+        public int getIterationCount() {
+            return iterationCount;
+        }
+
+        public boolean getForceTerminate() {
+            return forceTerminate;
+        }
+
+        public void setForceTerminate(boolean forceTerminate) {
+            this.forceTerminate = forceTerminate;
+        }
+
+        public boolean hasProcessedTagAttribute(String attributeName) {
+            return tagAttrProcessedBySolr.contains(attributeName);
+        }
+
+        public boolean hasProcessedEntityAttribute(String attributeName) {
+            return entityAttrProcessedBySolr.contains(attributeName);
+        }
+
+        public Iterator<AtlasIndexQuery.Result> getIndexResultsIterator() {
+            return indexResultsIterator;
+        }
+
+        public void setIndexResultsIterator(Iterator<AtlasIndexQuery.Result> indexResultsIterator) {
+            this.indexResultsIterator = indexResultsIterator;
+        }
+
+        public Iterator<AtlasVertex> getGremlinResultIterator() {
+            return gremlinResultIterator;
+        }
+
+        public void setGremlinResultIterator(Iterator<AtlasVertex> gremlinResultIterator) {
+            this.gremlinResultIterator = gremlinResultIterator;
+        }
+
+        public boolean hasIndexResults() {
+            return null != indexResultsIterator && indexResultsIterator.hasNext();
+        }
+
+        public boolean hasGremlinResults() {
+            return null != gremlinResultIterator && gremlinResultIterator.hasNext();
+        }
+
+
+        public boolean isTagProcessingComplete() {
+            return CollectionUtils.isEmpty(tagSearchAttributes) ||
+                    CollectionUtils.isEqualCollection(tagSearchAttributes, tagAttrProcessedBySolr);
+        }
+
+        public boolean isEntityProcessingComplete() {
+            return CollectionUtils.isEmpty(entitySearchAttributes) ||
+                    CollectionUtils.isEqualCollection(entitySearchAttributes, entityAttrProcessedBySolr);
+        }
+
+        public boolean isProcessingComplete() {
+            return isTagProcessingComplete() && isEntityProcessingComplete();
+        }
+
+        public void incrementOffset(int increment) {
+            currentOffset += increment;
+        }
+
+        public void incrementSearchRound() {
+            iterationCount ++;
+            incrementOffset(searchParameters.getLimit());
+        }
+
+        public int getCurrentOffset() {
+            return currentOffset;
+        }
+
+        public boolean addTagSearchAttribute(String attribute) {
+            return tagSearchAttributes.add(attribute);
+        }
+
+        public boolean addProcessedTagAttribute(String attribute) {
+            return tagAttrProcessedBySolr.add(attribute);
+        }
+
+        public boolean addEntitySearchAttribute(String attribute) {
+            return tagSearchAttributes.add(attribute);
+        }
+
+        public boolean addProcessedEntityAttribute(String attribute) {
+            return entityAttrProcessedBySolr.add(attribute);
+        }
+
+        public void cacheGraphQuery(String name, AtlasGraphQuery graphQuery) {
+            cachedGraphQueries.put(name, graphQuery);
+        }
+
+        public void cacheIndexQuery(String name, AtlasIndexQuery indexQuery) {
+            cachedIndexQueries.put(name, indexQuery);
+        }
+
+        public AtlasIndexQuery getIndexQuery(String name){
+            return cachedIndexQueries.get(name);
+        }
+
+        public AtlasGraphQuery getGraphQuery(String name) {
+            return cachedGraphQueries.get(name);
+        }
+
+        public IndexResultType getIndexResultType() {
+            return indexResultType;
+        }
+
+        public void setIndexResultType(IndexResultType indexResultType) {
+            this.indexResultType = indexResultType;
+        }
+
+        public int getMaxLimit() {
+            return maxLimit;
+        }
+
+        @Override
+        public String toString() {
+            return new ToStringBuilder(this)
+                    .append("iterationCount", iterationCount)
+                    .append("forceTerminate", forceTerminate)
+                    .append("currentOffset", currentOffset)
+                    .append("maxLimit", maxLimit)
+                    .append("searchParameters", searchParameters)
+                    .append("tagSearchAttributes", tagSearchAttributes)
+                    .append("entitySearchAttributes", entitySearchAttributes)
+                    .append("tagAttrProcessedBySolr", tagAttrProcessedBySolr)
+                    .append("entityAttrProcessedBySolr", entityAttrProcessedBySolr)
+                    .append("indexResultType", indexResultType)
+                    .append("cachedIndexQueries", cachedIndexQueries)
+                    .append("cachedGraphQueries", cachedGraphQueries)
+                    .toString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/discovery/SolrStep.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/discovery/SolrStep.java b/repository/src/main/java/org/apache/atlas/discovery/SolrStep.java
new file mode 100644
index 0000000..6a5dd5a
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/discovery/SolrStep.java
@@ -0,0 +1,288 @@
+/**
+ * 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.discovery.SearchPipeline.IndexResultType;
+import org.apache.atlas.discovery.SearchPipeline.PipelineContext;
+import org.apache.atlas.discovery.SearchPipeline.PipelineStep;
+import org.apache.atlas.exception.AtlasBaseException;
+import org.apache.atlas.model.discovery.SearchParameters;
+import org.apache.atlas.model.discovery.SearchParameters.Operator;
+import org.apache.atlas.repository.Constants;
+import org.apache.atlas.repository.graphdb.AtlasGraph;
+import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
+import org.apache.atlas.type.*;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.inject.Inject;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.atlas.model.discovery.SearchParameters.*;
+
+@Component
+public class SolrStep implements PipelineStep {
+    private static final Logger LOG = LoggerFactory.getLogger(SolrStep.class);
+
+    private static final Pattern STRAY_AND_PATTERN     = Pattern.compile("(AND\\s+)+\\)");
+    private static final Pattern STRAY_OR_PATTERN      = Pattern.compile("(OR\\s+)+\\)");
+    private static final Pattern STRAY_ELIPSIS_PATTERN = Pattern.compile("(\\(\\s*)\\)");
+    private static final String  AND_STR         = " AND ";
+    private static final String  EMPTY_STRING    = "";
+    private static final String  SPACE_STRING    = " ";
+    private static final String  BRACE_OPEN_STR  = "( ";
+    private static final String  BRACE_CLOSE_STR = " )";
+
+    private static final Map<Operator, String> operatorMap = new HashMap<>();
+
+    static
+    {
+        operatorMap.put(Operator.LT,"v.\"%s\": [* TO %s}");
+        operatorMap.put(Operator.GT,"v.\"%s\": {%s TO *]");
+        operatorMap.put(Operator.LTE,"v.\"%s\": [* TO %s]");
+        operatorMap.put(Operator.GTE,"v.\"%s\": [%s TO *]");
+        operatorMap.put(Operator.EQ,"v.\"%s\": %s");
+        operatorMap.put(Operator.NEQ,"v.\"%s\": (NOT %s)");
+        operatorMap.put(Operator.IN, "v.\"%s\": (%s)");
+        operatorMap.put(Operator.LIKE, "v.\"%s\": (%s)");
+        operatorMap.put(Operator.STARTS_WITH, "v.\"%s\": (%s*)");
+        operatorMap.put(Operator.ENDS_WITH, "v.\"%s\": (*%s)");
+        operatorMap.put(Operator.CONTAINS, "v.\"%s\": (*%s*)");
+    }
+
+    private final AtlasGraph graph;
+
+    @Inject
+    public SolrStep(AtlasGraph graph) {
+        this.graph = graph;
+    }
+
+    @Override
+    public void execute(PipelineContext context) throws AtlasBaseException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> SolrStep.execute({})", context);
+        }
+
+        if (context == null) {
+            throw new AtlasBaseException("Can't start search without any context");
+        }
+
+        SearchParameters searchParameters = context.getSearchParameters();
+
+        final Iterator<AtlasIndexQuery.Result> result;
+
+        if (StringUtils.isNotEmpty(searchParameters.getQuery())) {
+            result = executeAgainstFulltextIndex(context);
+        } else {
+            result = executeAgainstVertexIndex(context);
+        }
+
+        context.setIndexResultsIterator(result);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== SolrStep.execute({})", context);
+        }
+    }
+
+    private Iterator<AtlasIndexQuery.Result> executeAgainstFulltextIndex(PipelineContext context) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> SolrStep.executeAgainstFulltextIndex()");
+        }
+
+        final Iterator<AtlasIndexQuery.Result> ret;
+
+        AtlasIndexQuery query = context.getIndexQuery("FULLTEXT");
+
+        if (query == null) {
+            // Compute only once
+            SearchParameters searchParameters = context.getSearchParameters();
+            String           indexQuery       = String.format("v.\"%s\":(%s)", Constants.ENTITY_TEXT_PROPERTY_KEY, searchParameters.getQuery());
+
+            query = graph.indexQuery(Constants.FULLTEXT_INDEX, indexQuery);
+
+            context.cacheIndexQuery("FULLTEXT", query);
+        }
+
+        context.setIndexResultType(IndexResultType.TEXT);
+
+        ret = query.vertices(context.getCurrentOffset(), context.getMaxLimit());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== SolrStep.executeAgainstFulltextIndex()");
+        }
+
+        return ret;
+    }
+
+    private Iterator<AtlasIndexQuery.Result> executeAgainstVertexIndex(PipelineContext context) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> SolrStep.executeAgainstVertexIndex()");
+        }
+
+        final Iterator<AtlasIndexQuery.Result> ret;
+
+        SearchParameters searchParameters = context.getSearchParameters();
+        AtlasIndexQuery  query            = context.getIndexQuery("VERTEX_INDEX");
+
+        if (query == null) {
+            StringBuilder solrQuery = new StringBuilder();
+
+            // If tag is specified then let's start processing using tag and it's attributes, entity filters will
+            // be pushed to Gremlin
+            if (context.getClassificationType() != null) {
+                context.setIndexResultType(IndexResultType.TAG);
+
+                constructTypeTestQuery(solrQuery, context.getClassificationType().getTypeAndAllSubTypes());
+                constructFilterQuery(solrQuery, context.getClassificationType(), searchParameters.getTagFilters(), context);
+            } else if (context.getEntityType() != null) {
+                context.setIndexResultType(IndexResultType.ENTITY);
+
+                constructTypeTestQuery(solrQuery, context.getEntityType().getTypeAndAllSubTypes());
+                constructFilterQuery(solrQuery, context.getEntityType(), searchParameters.getEntityFilters(), context);
+
+                // Set the status flag
+                if (searchParameters.getExcludeDeletedEntities()) {
+                    if (solrQuery.length() > 0) {
+                        solrQuery.append(" AND ");
+                    }
+
+                    solrQuery.append("v.\"__state\":").append("ACTIVE");
+                }
+            }
+
+            // No query was formed, doesn't make sense to do anything beyond this point
+            if (solrQuery.length() > 0) {
+                String validSolrQuery = STRAY_AND_PATTERN.matcher(solrQuery).replaceAll(")");
+                validSolrQuery = STRAY_OR_PATTERN.matcher(validSolrQuery).replaceAll(")");
+                validSolrQuery = STRAY_ELIPSIS_PATTERN.matcher(validSolrQuery).replaceAll(EMPTY_STRING);
+
+                query = graph.indexQuery(Constants.VERTEX_INDEX, validSolrQuery);
+                context.cacheIndexQuery("VERTEX_INDEX", query);
+            }
+        }
+
+        // Execute solr query and return the index results in the context
+        if (query != null) {
+            ret = query.vertices(context.getCurrentOffset(), context.getMaxLimit());
+        } else {
+            ret = null;
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== SolrStep.executeAgainstVertexIndex()");
+        }
+
+        return ret;
+    }
+
+    private void constructTypeTestQuery(StringBuilder solrQuery, Set<String> typeAndAllSubTypes) {
+        String typeAndSubtypesString = StringUtils.join(typeAndAllSubTypes, SPACE_STRING);
+
+        solrQuery.append("v.\"__typeName\": (")
+                .append(typeAndSubtypesString)
+                .append(")");
+    }
+
+    private void constructFilterQuery(StringBuilder solrQuery, AtlasStructType type, FilterCriteria filterCriteria, PipelineContext context) {
+        if (filterCriteria != null) {
+            LOG.debug("Processing Filters");
+
+            String filterQuery = toSolrQuery(type, filterCriteria, context);
+
+            if (StringUtils.isNotEmpty(filterQuery)) {
+                solrQuery.append(AND_STR).append(filterQuery);
+            }
+        }
+    }
+
+    private String toSolrQuery(AtlasStructType type, FilterCriteria criteria, PipelineContext context) {
+        return toSolrQuery(type, criteria, context, new StringBuilder());
+    }
+
+    private String toSolrQuery(AtlasStructType type, FilterCriteria criteria, PipelineContext context, StringBuilder sb) {
+        if (criteria.getCondition() != null && CollectionUtils.isNotEmpty(criteria.getCriterion())) {
+            StringBuilder nestedExpression = new StringBuilder();
+
+            for (FilterCriteria filterCriteria : criteria.getCriterion()) {
+                String nestedQuery = toSolrQuery(type, filterCriteria, context);
+
+                if (StringUtils.isNotEmpty(nestedQuery)) {
+                    if (nestedExpression.length() > 0) {
+                        nestedExpression.append(SPACE_STRING).append(criteria.getCondition()).append(SPACE_STRING);
+                    }
+
+                    nestedExpression.append(nestedQuery);
+                }
+            }
+
+            return nestedExpression.length() > 0 ? sb.append(BRACE_OPEN_STR).append(nestedExpression.toString()).append(BRACE_CLOSE_STR).toString() : EMPTY_STRING;
+        } else {
+            return toSolrExpression(type, criteria.getAttributeName(), criteria.getOperator(), criteria.getAttributeValue(), context);
+        }
+    }
+
+    private String toSolrExpression(AtlasStructType type, String attrName, Operator op, String attrVal, PipelineContext context) {
+        String ret = EMPTY_STRING;
+
+        try {
+            String    indexKey      = type.getQualifiedAttributeName(attrName);
+            AtlasType attributeType = type.getAttributeType(attrName);
+
+            switch (context.getIndexResultType()) {
+                case TAG:
+                    context.addTagSearchAttribute(indexKey);
+                    break;
+
+                case ENTITY:
+                    context.addEntitySearchAttribute(indexKey);
+                    break;
+
+                default:
+                    // Do nothing
+            }
+
+            if (attributeType != null && AtlasTypeUtil.isBuiltInType(attributeType.getTypeName()) && context.getIndexedKeys().contains(indexKey)) {
+                if (operatorMap.get(op) != null) {
+                    // If there's a chance of multi-value then we need some additional processing here
+                    switch (context.getIndexResultType()) {
+                        case TAG:
+                            context.addProcessedTagAttribute(indexKey);
+                            break;
+
+                        case ENTITY:
+                            context.addProcessedEntityAttribute(indexKey);
+                            break;
+                    }
+
+                    ret = String.format(operatorMap.get(op), indexKey, attrVal);
+                }
+            }
+        } catch (AtlasBaseException ex) {
+            LOG.warn(ex.getMessage());
+        }
+
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
index 35dbf6c..94b6092 100755
--- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
+++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java
@@ -68,8 +68,10 @@ import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.*;
 
@@ -96,7 +98,10 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
 
     //allows injection of a dummy graph for testing
     private IAtlasGraphProvider provider;
-    
+
+    private boolean     recomputeIndexedKeys = true;
+    private Set<String> vertexIndexKeys      = new HashSet<>();
+
     @Inject
     public GraphBackedSearchIndexer(AtlasTypeRegistry typeRegistry) throws AtlasException {
         this(new AtlasGraphProvider(), ApplicationProperties.get(), typeRegistry);
@@ -130,6 +135,7 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
             if (management.containsPropertyKey(Constants.VERTEX_TYPE_PROPERTY_KEY)) {
                 LOG.info("Global indexes already exist for graph");
                 management.commit();
+
                 return;
             }
 
@@ -192,7 +198,6 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
             throw new RepositoryException(t);
         }
     }
-   
 
     private void createFullTextIndex(AtlasGraphManagement management) {
         AtlasPropertyKey fullText =
@@ -247,6 +252,34 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
         onAdd(dataTypes);
     }
 
+    public Set<String> getVertexIndexKeys() {
+        if (recomputeIndexedKeys) {
+            AtlasGraphManagement management = null;
+
+            try {
+                management = provider.get().getManagementSystem();
+            } catch (RepositoryException excp) {
+                LOG.error("failed to get indexedKeys from graph", excp);
+            }
+
+            if (management != null) {
+                recomputeIndexedKeys = false;
+
+                AtlasGraphIndex vertexIndex = management.getGraphIndex(Constants.VERTEX_INDEX);
+
+                Set<String> indexKeys = new HashSet<>();
+
+                for (AtlasPropertyKey fieldKey : vertexIndex.getFieldKeys()) {
+                    indexKeys.add(fieldKey.getName());
+                }
+
+                vertexIndexKeys = indexKeys;
+            }
+        }
+
+        return vertexIndexKeys;
+    }
+
     private void addIndexForType(AtlasGraphManagement management, AtlasBaseTypeDef typeDef) {
         if (typeDef instanceof AtlasEnumDef) {
             // Only handle complex types like Struct, Classification and Entity
@@ -577,6 +610,8 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
     private void commit(AtlasGraphManagement management) throws IndexException {
         try {
             management.commit();
+
+            recomputeIndexedKeys = true;
         } catch (Exception e) {
             LOG.error("Index commit failed", e);
             throw new IndexException("Index commit failed ", e);
@@ -586,6 +621,8 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
     private void rollback(AtlasGraphManagement management) throws IndexException {
         try {
             management.rollback();
+
+            recomputeIndexedKeys = true;
         } catch (Exception e) {
             LOG.error("Index rollback failed ", e);
             throw new IndexException("Index rollback failed ", e);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/repository/impexp/ImportService.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/impexp/ImportService.java b/repository/src/main/java/org/apache/atlas/repository/impexp/ImportService.java
index 3411f8d..9221717 100644
--- a/repository/src/main/java/org/apache/atlas/repository/impexp/ImportService.java
+++ b/repository/src/main/java/org/apache/atlas/repository/impexp/ImportService.java
@@ -30,12 +30,14 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
 
+import javax.inject.Inject;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 
-
+@Component
 public class ImportService {
     private static final Logger LOG = LoggerFactory.getLogger(ImportService.class);
 
@@ -46,6 +48,7 @@ public class ImportService {
     private long startTimestamp;
     private long endTimestamp;
 
+    @Inject
     public ImportService(final AtlasTypeDefStore typeDefStore, final AtlasEntityStore entityStore, AtlasTypeRegistry typeRegistry) {
         this.typeDefStore = typeDefStore;
         this.entityStore = entityStore;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
index 42bd58f..7b3f1e6 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/AtlasGraphUtilsV1.java
@@ -39,7 +39,10 @@ import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.*;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.Map;
 
 /**
  * Utility methods for Graph.

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
index 9a8695a..a5b5730 100644
--- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
+++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/EntityGraphRetriever.java
@@ -46,10 +46,12 @@ import org.slf4j.LoggerFactory;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGDECIMAL;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_BIGINTEGER;
@@ -123,7 +125,7 @@ public final class EntityGraphRetriever {
     }
 
     public AtlasEntityHeader toAtlasEntityHeader(AtlasVertex entityVertex) throws AtlasBaseException {
-        return entityVertex != null ? mapVertexToAtlasEntityHeader(entityVertex) : null;
+        return toAtlasEntityHeader(entityVertex, Collections.<String>emptySet());
     }
 
     private AtlasVertex getEntityVertex(String guid) throws AtlasBaseException {
@@ -185,6 +187,10 @@ public final class EntityGraphRetriever {
     }
 
     private AtlasEntityHeader mapVertexToAtlasEntityHeader(AtlasVertex entityVertex) throws AtlasBaseException {
+        return mapVertexToAtlasEntityHeader(entityVertex, Collections.<String>emptySet());
+    }
+
+    private AtlasEntityHeader mapVertexToAtlasEntityHeader(AtlasVertex entityVertex, Set<String> attributes) throws AtlasBaseException {
         AtlasEntityHeader ret = new AtlasEntityHeader();
 
         String typeName = entityVertex.getProperty(Constants.TYPE_NAME_PROPERTY_KEY, String.class);
@@ -218,6 +224,20 @@ public final class EntityGraphRetriever {
             if (displayText != null) {
                 ret.setDisplayText(displayText.toString());
             }
+
+            if (CollectionUtils.isNotEmpty(attributes)) {
+                for (String attrName : attributes) {
+                    if (ret.hasAttribute(attrName)) {
+                        continue;
+                    }
+
+                    Object attrValue = getVertexAttribute(entityVertex, entityType.getAttribute(attrName));
+
+                    if (attrValue != null) {
+                        ret.setAttribute(attrName, attrValue);
+                    }
+                }
+            }
         }
 
         return ret;
@@ -556,4 +576,8 @@ public final class EntityGraphRetriever {
     private Object getVertexAttribute(AtlasVertex vertex, AtlasAttribute attribute) throws AtlasBaseException {
         return vertex != null && attribute != null ? mapVertexToAttribute(vertex, attribute, null) : null;
     }
+
+    public AtlasEntityHeader toAtlasEntityHeader(AtlasVertex atlasVertex, Set<String> attributes) throws AtlasBaseException {
+        return atlasVertex != null ? mapVertexToAtlasEntityHeader(atlasVertex, attributes) : null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/main/java/org/apache/atlas/util/SearchTracker.java
----------------------------------------------------------------------
diff --git a/repository/src/main/java/org/apache/atlas/util/SearchTracker.java b/repository/src/main/java/org/apache/atlas/util/SearchTracker.java
new file mode 100644
index 0000000..15a8c20
--- /dev/null
+++ b/repository/src/main/java/org/apache/atlas/util/SearchTracker.java
@@ -0,0 +1,73 @@
+/**
+ * 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.util;
+
+import org.apache.atlas.annotation.AtlasService;
+import org.apache.atlas.discovery.SearchPipeline.PipelineContext;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+@AtlasService
+public class SearchTracker {
+    private Map<String, PipelineContext> activeSearches = new HashMap<>();
+
+    /**
+     *
+     * @param context
+     */
+    public String add(PipelineContext context) {
+        String searchId = Thread.currentThread().getName();
+
+        activeSearches.put(searchId, context);
+
+        return searchId;
+    }
+
+    /**
+     *
+     * @param searchId
+     * @return
+     */
+    public PipelineContext terminate(String searchId) {
+        PipelineContext ret = null;
+
+        if (activeSearches.containsKey(searchId)) {
+            PipelineContext pipelineToTerminate = activeSearches.remove(searchId);
+
+            pipelineToTerminate.setForceTerminate(true);
+
+            ret = pipelineToTerminate;
+        }
+
+        return ret;
+    }
+
+    public PipelineContext remove(String id) {
+        return activeSearches.remove(id);
+    }
+
+    /**
+     *
+     * @return
+     */
+    public Set<String> getActiveSearches() {
+        return activeSearches.keySet();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/test/java/org/apache/atlas/TestModules.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/TestModules.java b/repository/src/test/java/org/apache/atlas/TestModules.java
index fa2ac0d..d0da030 100644
--- a/repository/src/test/java/org/apache/atlas/TestModules.java
+++ b/repository/src/test/java/org/apache/atlas/TestModules.java
@@ -17,19 +17,14 @@
  */
 package org.apache.atlas;
 
+import com.google.inject.AbstractModule;
 import com.google.inject.Binder;
 import com.google.inject.Provider;
 import com.google.inject.Singleton;
 import com.google.inject.matcher.Matchers;
 import com.google.inject.multibindings.Multibinder;
 import org.apache.atlas.annotation.GraphTransaction;
-import org.apache.atlas.discovery.AtlasDiscoveryService;
-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.*;
 import org.apache.atlas.discovery.graph.GraphBackedDiscoveryService;
 import org.apache.atlas.graph.GraphSandboxUtil;
 import org.apache.atlas.listener.EntityChangeListener;
@@ -61,6 +56,7 @@ import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.atlas.typesystem.types.TypeSystem;
 import org.apache.atlas.typesystem.types.cache.TypeCache;
 import org.apache.atlas.util.AtlasRepositoryConfiguration;
+import org.apache.atlas.util.SearchTracker;
 import org.apache.commons.configuration.Configuration;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
@@ -76,7 +72,7 @@ public class TestModules {
     }
 
     // Test only DI modules
-    public static class TestOnlyModule extends com.google.inject.AbstractModule {
+    public static class TestOnlyModule extends AbstractModule {
 
         private static final Logger LOG = LoggerFactory.getLogger(TestOnlyModule.class);
 
@@ -147,6 +143,11 @@ public class TestModules {
             typeDefChangeListenerMultibinder.addBinding().to(DefaultMetadataService.class);
             typeDefChangeListenerMultibinder.addBinding().to(GraphBackedSearchIndexer.class).asEagerSingleton();
 
+            bind(SearchPipeline.class).asEagerSingleton();
+            bind(SearchTracker.class).asEagerSingleton();
+            bind(SolrStep.class).asEagerSingleton();
+            bind(GremlinStep.class).asEagerSingleton();
+
             bind(AtlasEntityStore.class).to(AtlasEntityStoreV1.class);
             bind(AtlasRelationshipStore.class).to(AtlasRelationshipStoreV1.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/repository/src/test/java/org/apache/atlas/services/EntityDiscoveryServiceTest.java
----------------------------------------------------------------------
diff --git a/repository/src/test/java/org/apache/atlas/services/EntityDiscoveryServiceTest.java b/repository/src/test/java/org/apache/atlas/services/EntityDiscoveryServiceTest.java
index 5d5b043..dfb2ee2 100644
--- a/repository/src/test/java/org/apache/atlas/services/EntityDiscoveryServiceTest.java
+++ b/repository/src/test/java/org/apache/atlas/services/EntityDiscoveryServiceTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas.services;
 
+import org.apache.atlas.TestModules;
 import org.apache.atlas.discovery.EntityDiscoveryService;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.typedef.AtlasEntityDef;
@@ -24,12 +25,16 @@ import org.apache.atlas.type.AtlasTypeRegistry;
 import org.apache.commons.lang.StringUtils;
 import org.powermock.reflect.Whitebox;
 import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import javax.inject.Inject;
+
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertTrue;
 
+@Guice(modules = TestModules.TestOnlyModule.class)
 public class EntityDiscoveryServiceTest {
 
     private final String TEST_TYPE                = "test";
@@ -47,6 +52,9 @@ public class EntityDiscoveryServiceTest {
 
     private final int maxTypesCountInIdxQuery = 10;
 
+    @Inject
+    EntityDiscoveryService discoveryService;
+
 
     @BeforeClass
     public void init() throws AtlasBaseException {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/webapp/src/main/java/org/apache/atlas/web/filters/StaleTransactionCleanupFilter.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/filters/StaleTransactionCleanupFilter.java b/webapp/src/main/java/org/apache/atlas/web/filters/StaleTransactionCleanupFilter.java
index cdb9064..8e74d39 100644
--- a/webapp/src/main/java/org/apache/atlas/web/filters/StaleTransactionCleanupFilter.java
+++ b/webapp/src/main/java/org/apache/atlas/web/filters/StaleTransactionCleanupFilter.java
@@ -50,7 +50,7 @@ public class StaleTransactionCleanupFilter implements Filter {
     @Override
     public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain)
     throws IOException, ServletException {
-        LOG.info("Cleaning stale transactions");
+        LOG.debug("Cleaning stale transactions");
         AtlasGraphProvider.getGraphInstance().rollback();
         filterChain.doFilter(request, response);
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java b/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
index 8c5623f..1a9f57a 100755
--- a/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
+++ b/webapp/src/main/java/org/apache/atlas/web/resources/AdminResource.java
@@ -25,6 +25,7 @@ import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.authorize.AtlasActionTypes;
 import org.apache.atlas.authorize.AtlasResourceTypes;
 import org.apache.atlas.authorize.simple.AtlasAuthorizationUtils;
+import org.apache.atlas.discovery.SearchPipeline;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.impexp.AtlasExportRequest;
 import org.apache.atlas.model.impexp.AtlasExportResult;
@@ -35,11 +36,9 @@ import org.apache.atlas.repository.impexp.ExportService;
 import org.apache.atlas.repository.impexp.ImportService;
 import org.apache.atlas.repository.impexp.ZipSink;
 import org.apache.atlas.repository.impexp.ZipSource;
-import org.apache.atlas.repository.store.graph.AtlasEntityStore;
 import org.apache.atlas.services.MetricsService;
-import org.apache.atlas.store.AtlasTypeDefStore;
 import org.apache.atlas.type.AtlasType;
-import org.apache.atlas.type.AtlasTypeRegistry;
+import org.apache.atlas.util.SearchTracker;
 import org.apache.atlas.web.filters.AtlasCSRFPreventionFilter;
 import org.apache.atlas.web.service.ServiceState;
 import org.apache.atlas.web.util.Servlets;
@@ -51,7 +50,6 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.springframework.context.ApplicationContext;
 import org.springframework.security.core.Authentication;
 import org.springframework.security.core.GrantedAuthority;
 import org.springframework.security.core.context.SecurityContextHolder;
@@ -62,9 +60,11 @@ import javax.inject.Singleton;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.WebApplicationException;
@@ -109,14 +109,10 @@ public class AdminResource {
 
     private final ServiceState      serviceState;
     private final MetricsService    metricsService;
-    private final AtlasTypeRegistry typeRegistry;
-    private final AtlasTypeDefStore typesDefStore;
-    private final AtlasEntityStore  entityStore;
     private static Configuration atlasProperties;
     private final ExportService exportService;
-
-    @Inject
-    ApplicationContext applicationContext;
+    private final ImportService importService;
+    private final SearchTracker activeSearches;
 
     static {
         try {
@@ -128,15 +124,13 @@ public class AdminResource {
 
     @Inject
     public AdminResource(ServiceState serviceState, MetricsService metricsService,
-                         AtlasTypeRegistry typeRegistry, AtlasTypeDefStore typeDefStore,
-                         AtlasEntityStore entityStore, ExportService exportService) {
+                         ExportService exportService, ImportService importService, SearchTracker activeSearches) {
         this.serviceState               = serviceState;
         this.metricsService             = metricsService;
-        this.typeRegistry               = typeRegistry;
-        this.typesDefStore              = typeDefStore;
-        this.entityStore                = entityStore;
         this.exportService = exportService;
-        this.importExportOperationLock  = new ReentrantLock();
+        this.importService = importService;
+        this.activeSearches = activeSearches;
+        importExportOperationLock = new ReentrantLock();
     }
 
     /**
@@ -377,7 +371,6 @@ public class AdminResource {
 
         try {
             AtlasImportRequest request = AtlasType.fromJson(jsonData, AtlasImportRequest.class);
-            ImportService importService = new ImportService(this.typesDefStore, this.entityStore, this.typeRegistry);
             ZipSource zipSource = new ZipSource(inputStream);
 
             result = importService.run(zipSource, request, Servlets.getUserName(httpServletRequest),
@@ -412,7 +405,6 @@ public class AdminResource {
 
         try {
             AtlasImportRequest request = AtlasType.fromJson(jsonData, AtlasImportRequest.class);
-            ImportService importService = new ImportService(this.typesDefStore, this.entityStore, this.typeRegistry);
             result = importService.run(request, Servlets.getUserName(httpServletRequest),
                                        Servlets.getHostName(httpServletRequest),
                                        AtlasAuthorizationUtils.getRequestIpAddress(httpServletRequest));
@@ -431,6 +423,21 @@ public class AdminResource {
         return result;
     }
 
+    @GET
+    @Path("activeSearches")
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public Set<String> getActiveSearches() {
+        return activeSearches.getActiveSearches();
+    }
+
+    @DELETE
+    @Path("activeSearches/{id}")
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public boolean terminateActiveSearch(@PathParam("id") String searchId) {
+        SearchPipeline.PipelineContext terminate = activeSearches.terminate(searchId);
+        return null != terminate;
+    }
+
     private String getEditableEntityTypes(Configuration config) {
         String ret = DEFAULT_EDITABLE_ENTITY_TYPES;
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/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 ea55021..dde300e 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
@@ -21,8 +21,10 @@ import org.apache.atlas.AtlasErrorCode;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.discovery.AtlasDiscoveryService;
 import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.discovery.SearchParameters;
 import org.apache.atlas.utils.AtlasPerfTracer;
 import org.apache.atlas.web.util.Servlets;
+import org.apache.commons.collections.CollectionUtils;
 import org.springframework.stereotype.Service;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -31,6 +33,7 @@ import javax.inject.Inject;
 import javax.inject.Singleton;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
@@ -213,6 +216,50 @@ public class DiscoveryREST {
         }
     }
 
+    /**
+     * Attribute based search for entities satisfying the search parameters
+     * @param parameters Search parameters
+     * @return Atlas search result
+     * @throws AtlasBaseException
+     *
+     * @HTTP 200 On successful search
+     * @HTTP 400 Tag/Entity doesn't exist or Tag/entity filter is present without tag/type name
+     */
+    @Path("basic")
+    @POST
+    @Consumes(Servlets.JSON_MEDIA_TYPE)
+    @Produces(Servlets.JSON_MEDIA_TYPE)
+    public AtlasSearchResult searchWithParameters(SearchParameters parameters) throws AtlasBaseException {
+        AtlasPerfTracer perf = null;
+
+        try {
+            if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) {
+                perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "DiscoveryREST.searchWithParameters("+ parameters + ")");
+            }
+
+            if (parameters.getLimit() < 0 || parameters.getOffset() < 0) {
+                throw new AtlasBaseException(AtlasErrorCode.BAD_REQUEST, "Limit/offset should be non-negative");
+            }
+
+            if (StringUtils.isEmpty(parameters.getTypeName()) && !isEmpty(parameters.getEntityFilters())) {
+                throw new AtlasBaseException(AtlasErrorCode.BAD_REQUEST, "EntityFilters specified without Type name");
+            }
+
+            if (StringUtils.isEmpty(parameters.getClassification()) && !isEmpty(parameters.getTagFilters())) {
+                throw new AtlasBaseException(AtlasErrorCode.BAD_REQUEST, "TagFilters specified without tag name");
+            }
+
+            return atlasDiscoveryService.searchUsingBasicQuery(parameters);
+        } finally {
+            AtlasPerfTracer.log(perf);
+        }
+    }
+
+    private boolean isEmpty(SearchParameters.FilterCriteria filterCriteria) {
+        return filterCriteria == null ||
+               (StringUtils.isEmpty(filterCriteria.getAttributeName()) && CollectionUtils.isEmpty(filterCriteria.getCriterion()));
+    }
+
     private String escapeTypeName(String typeName) {
         String ret;
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java b/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
index 1fe3119..c0bbf09 100644
--- a/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/AdminResourceTest.java
@@ -48,7 +48,7 @@ public class AdminResourceTest {
 
         when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.ACTIVE);
 
-        AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null);
+        AdminResource adminResource = new AdminResource(serviceState, null, null, null, null);
         Response response = adminResource.getStatus();
         assertEquals(response.getStatus(), HttpServletResponse.SC_OK);
         JSONObject entity = (JSONObject) response.getEntity();
@@ -59,7 +59,7 @@ public class AdminResourceTest {
     public void testResourceGetsValueFromServiceState() throws JSONException {
         when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.PASSIVE);
 
-        AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null);
+        AdminResource adminResource = new AdminResource(serviceState, null, null, null, null);
         Response response = adminResource.getStatus();
 
         verify(serviceState).getState();


[2/5] incubator-atlas git commit: ATLAS-1880: search API with support for entity/tag attribute filters

Posted by ma...@apache.org.
ATLAS-1880: search API with support for entity/tag attribute filters


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

Branch: refs/heads/master
Commit: 7c262b40ba4bbf6946c2423f36ddb0f3ee12c8f8
Parents: 8101883
Author: apoorvnaik <ap...@apache.org>
Authored: Wed Jun 28 06:22:34 2017 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Fri Jun 30 01:49:08 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/atlas/AtlasClientV2.java    |  22 +
 .../org/apache/atlas/repository/Constants.java  |   4 +-
 distro/src/conf/atlas-log4j.xml                 |  17 +
 .../repository/graphdb/AtlasGraphQuery.java     |  39 +-
 .../repository/graphdb/AtlasIndexQuery.java     |   8 +
 .../repository/graphdb/AtlasVertexQuery.java    |  16 +-
 .../titan/query/NativeTitanGraphQuery.java      |  25 +-
 .../graphdb/titan/query/TitanGraphQuery.java    |  65 +-
 .../graphdb/titan/query/expr/AndCondition.java  |   8 +-
 .../graphdb/titan/query/expr/HasPredicate.java  |   7 +-
 .../query/graph/GraphCentricQueryBuilder.java   |  28 +-
 .../graphdb/titan0/Titan0IndexQuery.java        |  21 +
 .../graphdb/titan0/Titan0VertexQuery.java       |  15 +
 .../titan0/query/NativeTitan0GraphQuery.java    |  92 ++-
 .../graphdb/titan1/Titan1IndexQuery.java        |  22 +
 .../graphdb/titan1/Titan1VertexQuery.java       |  14 +
 .../titan1/query/NativeTitan1GraphQuery.java    |  90 ++-
 .../model/discovery/AtlasSearchResult.java      |  28 +-
 .../atlas/model/discovery/SearchParameters.java | 366 +++++++++++
 .../atlas/model/impexp/AtlasImportRequest.java  |  15 +-
 .../atlas/discovery/AtlasDiscoveryService.java  |   9 +
 .../atlas/discovery/EntityDiscoveryService.java |  34 +-
 .../atlas/discovery/EntityLineageService.java   |   2 +
 .../org/apache/atlas/discovery/GremlinStep.java | 388 ++++++++++++
 .../apache/atlas/discovery/SearchPipeline.java  | 611 +++++++++++++++++++
 .../org/apache/atlas/discovery/SolrStep.java    | 288 +++++++++
 .../graph/GraphBackedSearchIndexer.java         |  41 +-
 .../atlas/repository/impexp/ImportService.java  |   5 +-
 .../store/graph/v1/AtlasGraphUtilsV1.java       |   5 +-
 .../store/graph/v1/EntityGraphRetriever.java    |  26 +-
 .../org/apache/atlas/util/SearchTracker.java    |  73 +++
 .../test/java/org/apache/atlas/TestModules.java |  17 +-
 .../services/EntityDiscoveryServiceTest.java    |   8 +
 .../filters/StaleTransactionCleanupFilter.java  |   2 +-
 .../atlas/web/resources/AdminResource.java      |  43 +-
 .../apache/atlas/web/rest/DiscoveryREST.java    |  47 ++
 .../atlas/web/resources/AdminResourceTest.java  |   4 +-
 37 files changed, 2381 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/client/src/main/java/org/apache/atlas/AtlasClientV2.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/atlas/AtlasClientV2.java b/client/src/main/java/org/apache/atlas/AtlasClientV2.java
index 6141342..7e287e7 100644
--- a/client/src/main/java/org/apache/atlas/AtlasClientV2.java
+++ b/client/src/main/java/org/apache/atlas/AtlasClientV2.java
@@ -22,6 +22,7 @@ import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.core.util.MultivaluedMapImpl;
 import org.apache.atlas.model.SearchFilter;
 import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.discovery.SearchParameters;
 import org.apache.atlas.model.instance.AtlasClassification;
 import org.apache.atlas.model.instance.AtlasClassification.AtlasClassifications;
 import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo;
@@ -100,9 +101,13 @@ public class AtlasClientV2 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 String BASIC_SEARCH_URI = DISCOVERY_URI + "/basic";
+    private static final String FACETED_SEARCH_URI = BASIC_SEARCH_URI;
 
     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);
+    private static final APIInfo BASIC_SEARCH = new APIInfo(BASIC_SEARCH_URI, HttpMethod.GET, Response.Status.OK);
+    private static final APIInfo FACETED_SEARCH = new APIInfo(FACETED_SEARCH_URI, HttpMethod.POST, Response.Status.OK);
 
 
     public AtlasClientV2(String[] baseUrl, String[] basicAuthUserNamePassword) {
@@ -398,6 +403,23 @@ public class AtlasClientV2 extends AtlasBaseClient {
         return callAPI(FULL_TEXT_SEARCH, AtlasSearchResult.class, queryParams);
     }
 
+    public AtlasSearchResult basicSearch(final String typeName, final String classification, final String query,
+                                         final boolean excludeDeletedEntities, final int limit, final int offset) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add("typeName", typeName);
+        queryParams.add("classification", classification);
+        queryParams.add(QUERY, query);
+        queryParams.add("excludeDeletedEntities", String.valueOf(excludeDeletedEntities));
+        queryParams.add(LIMIT, String.valueOf(limit));
+        queryParams.add(OFFSET, String.valueOf(offset));
+
+        return callAPI(BASIC_SEARCH, AtlasSearchResult.class, queryParams);
+    }
+
+    public AtlasSearchResult facetedSearch(SearchParameters searchParameters) throws AtlasServiceException {
+        return callAPI(FACETED_SEARCH, AtlasSearchResult.class, searchParameters);
+    }
+
     private <T> T getTypeDefByName(final String name, Class<T> typeDefClass) throws AtlasServiceException {
         String atlasPath = getAtlasPath(typeDefClass);
         APIInfo apiInfo = new APIInfo(String.format(GET_BY_NAME_TEMPLATE, atlasPath, name), HttpMethod.GET, Response.Status.OK);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/common/src/main/java/org/apache/atlas/repository/Constants.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/atlas/repository/Constants.java b/common/src/main/java/org/apache/atlas/repository/Constants.java
index ac02252..e8621cf 100644
--- a/common/src/main/java/org/apache/atlas/repository/Constants.java
+++ b/common/src/main/java/org/apache/atlas/repository/Constants.java
@@ -96,7 +96,9 @@ public final class Constants {
 
     public static final String QUALIFIED_NAME = "Referenceable.qualifiedName";
     public static final String TYPE_NAME_PROPERTY_KEY = INTERNAL_PROPERTY_KEY_PREFIX + "typeName";
-
+    public static final String INDEX_SEARCH_MAX_RESULT_SET_SIZE = "atlas.graph.index.search.max-result-set-size";
+    public static final String INDEX_SEARCH_MAX_TYPES_COUNT = "atlas.graph.index.search.max-types-count";
+    public static final String INDEX_SEARCH_MAX_TAGS_COUNT = "atlas.graph.index.search.max-tags-count";
 
     private Constants() {
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/distro/src/conf/atlas-log4j.xml
----------------------------------------------------------------------
diff --git a/distro/src/conf/atlas-log4j.xml b/distro/src/conf/atlas-log4j.xml
index e6c0d9f..a0ef6a9 100755
--- a/distro/src/conf/atlas-log4j.xml
+++ b/distro/src/conf/atlas-log4j.xml
@@ -59,6 +59,23 @@
         </layout>
     </appender>
 
+    <!-- Uncomment the following for perf logs -->
+    <!--
+    <appender name="perf_appender" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="file" value="${atlas.log.dir}/atlas_perf.log" />
+        <param name="datePattern" value="'.'yyyy-MM-dd" />
+        <param name="append" value="true" />
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d|%t|%m%n" />
+        </layout>
+    </appender>
+
+    <logger name="org.apache.atlas.perf" additivity="false">
+        <level value="debug" />
+        <appender-ref ref="perf_appender" />
+    </logger>
+    -->
+
     <logger name="org.apache.atlas" additivity="false">
         <level value="info"/>
         <appender-ref ref="FILE"/>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasGraphQuery.java b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasGraphQuery.java
index 841edf7..73db22e 100644
--- a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasGraphQuery.java
+++ b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasGraphQuery.java
@@ -47,7 +47,7 @@ public interface AtlasGraphQuery<V, E> {
      * the specified list of values.
      *
      * @param propertyKey
-     * @param value
+     * @param values
      * @return
      */
     AtlasGraphQuery<V, E> in(String propertyKey, Collection<?> values);
@@ -56,7 +56,6 @@ public interface AtlasGraphQuery<V, E> {
     /**
      * Executes the query and returns the matching vertices.
      * @return
-     * @throws AtlasException
      */
     Iterable<AtlasVertex<V, E>> vertices();
 
@@ -66,16 +65,32 @@ public interface AtlasGraphQuery<V, E> {
      */
     Iterable<AtlasEdge<V, E>> edges();
 
+    /**
+     * Executes the query and returns the matching vertices from given offset till the max limit
+     * @param limit max number of vertices
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int limit);
+
+    /**
+     * Executes the query and returns the matching vertices from given offset till the max limit
+     * @param offset starting offset
+     * @param limit max number of vertices
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int offset, int limit);
+
 
     /**
      * Adds a predicate that the returned vertices must have the specified
      * property and that its value matches the criterion specified.
      *
      * @param propertyKey
-     * @param value
+     * @param op
+     * @param values
      * @return
      */
-    AtlasGraphQuery<V, E> has(String propertyKey, ComparisionOperator compMethod, Object values);
+    AtlasGraphQuery<V, E> has(String propertyKey, QueryOperator op, Object values);
 
     /**
      * Adds a predicate that the vertices returned must satisfy the
@@ -94,17 +109,31 @@ public interface AtlasGraphQuery<V, E> {
     AtlasGraphQuery<V, E> createChildQuery();
 
 
+    interface QueryOperator {}
+
     /**
      * Comparison operators that can be used in an AtlasGraphQuery.
      */
-    enum ComparisionOperator {
+    enum ComparisionOperator implements QueryOperator {
+        GREATER_THAN,
         GREATER_THAN_EQUAL,
         EQUAL,
+        LESS_THAN,
         LESS_THAN_EQUAL,
         NOT_EQUAL
     }
 
     /**
+     * String/text matching that can be used in AtlasGraphQuery
+     */
+    enum MatchingOperator implements QueryOperator {
+        CONTAINS,
+        PREFIX,
+        SUFFIX,
+        REGEX
+    }
+
+    /**
      * Adds all of the predicates that have been added to this query to the
      * specified query.
      * @param otherQuery

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasIndexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasIndexQuery.java b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasIndexQuery.java
index 1ff9d5e..6bad173 100644
--- a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasIndexQuery.java
+++ b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasIndexQuery.java
@@ -36,6 +36,14 @@ public interface AtlasIndexQuery<V, E> {
     Iterator<Result<V, E>> vertices();
 
     /**
+     * Gets the query results
+     * @param offset starting offset
+     * @param limit max number of results
+     * @return
+     */
+    Iterator<Result<V, E>> vertices(int offset, int limit);
+
+    /**
      * Query result from an index query.
      *
      * @param <V>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasVertexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasVertexQuery.java b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasVertexQuery.java
index 53f490f..9beb6a3 100644
--- a/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasVertexQuery.java
+++ b/graphdb/api/src/main/java/org/apache/atlas/repository/graphdb/AtlasVertexQuery.java
@@ -42,15 +42,29 @@ public interface AtlasVertexQuery<V, E> {
     Iterable<AtlasVertex<V, E>> vertices();
 
     /**
+     * Returns the vertices that satisfy the query condition.
+     *
+     * @param limit Max number of vertices
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int limit);
+
+    /**
      * Returns the incident edges that satisfy the query condition.
      * @return
      */
     Iterable<AtlasEdge<V, E>> edges();
 
     /**
+     * Returns the incident edges that satisfy the query condition.
+     * @param limit Max number of edges
+     * @return
+     */
+    Iterable<AtlasEdge<V, E>> edges(int limit);
+
+    /**
      * Returns the number of elements that match the query.
      * @return
      */
     long count();
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
index 0211ff0..288b325 100644
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/NativeTitanGraphQuery.java
@@ -17,12 +17,12 @@
  */
 package org.apache.atlas.repository.graphdb.titan.query;
 
-import java.util.Collection;
-
 import org.apache.atlas.repository.graphdb.AtlasEdge;
-import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 
+import java.util.Collection;
+
 /**
  * Interfaces that provides a thin wrapper around GraphQuery (used by Titan0) and
  * TitanGraphQuery (used by Titan 1).
@@ -47,6 +47,22 @@ public interface NativeTitanGraphQuery<V, E> {
     Iterable<AtlasEdge<V, E>> edges();
 
     /**
+     * Executes graph query
+     * @param limit Max vertices to return
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int limit);
+
+    /**
+     * Executes graph query
+     * @param offset Starting offset
+     * @param limit Max vertices to return
+     * @return
+     */
+    Iterable<AtlasVertex<V, E>> vertices(int offset, int limit);
+
+
+    /**
      * Adds an in condition to the query.
      *
      * @param propertyName
@@ -61,6 +77,5 @@ public interface NativeTitanGraphQuery<V, E> {
      * @param op
      * @param value
      */
-    void has(String propertyName, ComparisionOperator op, Object value);
-
+    void has(String propertyName, QueryOperator op, Object value);
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
index 0077a21..c38f6cc 100644
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/TitanGraphQuery.java
@@ -17,11 +17,8 @@
  */
 package org.apache.atlas.repository.graphdb.titan.query;
 
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasGraph;
 import org.apache.atlas.repository.graphdb.AtlasGraphQuery;
@@ -33,6 +30,13 @@ import org.apache.atlas.repository.graphdb.titan.query.expr.OrCondition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
 /**
  * Abstract implementation of AtlasGraphQuery that is used by both Titan 0.5.4
  * and Titan 1.0.0.
@@ -123,11 +127,10 @@ public abstract class TitanGraphQuery<V, E> implements AtlasGraphQuery<V, E> {
     @Override
     public Iterable<AtlasVertex<V, E>> vertices() {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("Executing: " + queryCondition.toString());
+            LOG.debug("Executing: " + queryCondition);
         }
 
-        //compute the overall result by unioning the results from all of the
-        //AndConditions together.
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
         Set<AtlasVertex<V, E>> result = new HashSet<>();
         for(AndCondition andExpr : queryCondition.getAndTerms()) {
             NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
@@ -141,11 +144,10 @@ public abstract class TitanGraphQuery<V, E> implements AtlasGraphQuery<V, E> {
     @Override
     public Iterable<AtlasEdge<V, E>> edges() {
         if (LOG.isDebugEnabled()) {
-            LOG.debug("Executing: " + queryCondition.toString());
+            LOG.debug("Executing: " + queryCondition);
         }
 
-        //compute the overall result by unioning the results from all of the
-        //AndConditions together.
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
         Set<AtlasEdge<V, E>> result = new HashSet<>();
         for(AndCondition andExpr : queryCondition.getAndTerms()) {
             NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
@@ -157,7 +159,46 @@ public abstract class TitanGraphQuery<V, E> implements AtlasGraphQuery<V, E> {
     }
 
     @Override
-    public AtlasGraphQuery<V, E> has(String propertyKey, ComparisionOperator operator,
+    public Iterable<AtlasVertex<V, E>> vertices(int limit) {
+        return vertices(0, limit);
+    }
+
+    @Override
+    public Iterable<AtlasVertex<V, E>> vertices(int offset, int limit) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Executing: " + queryCondition);
+        }
+
+        Preconditions.checkArgument(offset >= 0, "Offset must be non-negative");
+        Preconditions.checkArgument(limit >= 0, "Limit must be non-negative");
+
+        // Compute the overall result by combining the results of all the AndConditions (nested within OR) together.
+        Set<AtlasVertex<V, E>> result = new HashSet<>();
+        long resultIdx = 0;
+        for(AndCondition andExpr : queryCondition.getAndTerms()) {
+            if (result.size() == limit) {
+                break;
+            }
+
+            NativeTitanGraphQuery<V, E> andQuery = andExpr.create(getQueryFactory());
+            for(AtlasVertex<V, E> vertex : andQuery.vertices(offset + limit)) {
+                if (resultIdx >= offset) {
+                    result.add(vertex);
+
+                    if (result.size() == limit) {
+                        break;
+                    }
+                }
+
+                resultIdx++;
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public AtlasGraphQuery<V, E> has(String propertyKey, QueryOperator operator,
             Object value) {
         queryCondition.andWith(new HasPredicate(propertyKey, operator, value));
         return this;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
index 68f0eb2..db5093f 100644
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/AndCondition.java
@@ -17,12 +17,12 @@
  */
 package org.apache.atlas.repository.graphdb.titan.query.expr;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
 import org.apache.atlas.repository.graphdb.titan.query.NativeTitanQueryFactory;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Represents an AndCondition in a graph query.  Only vertices that
  * satisfy the conditions in all of the query predicates will be returned
@@ -78,7 +78,7 @@ public class AndCondition {
     /**
      * Creates a NativeTitanGraphQuery that can be used to evaluate this condition.
      *
-     * @param graph
+     * @param factory
      * @return
      */
     public <V, E> NativeTitanGraphQuery<V, E> create(NativeTitanQueryFactory<V, E> factory) {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
----------------------------------------------------------------------
diff --git a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
index 24e4f5b..0652c41 100644
--- a/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
+++ b/graphdb/common/src/main/java/org/apache/atlas/repository/graphdb/titan/query/expr/HasPredicate.java
@@ -17,7 +17,7 @@
  */
 package org.apache.atlas.repository.graphdb.titan.query.expr;
 
-import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
 import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
 
 /**
@@ -27,11 +27,10 @@ import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
 public class HasPredicate implements QueryPredicate {
 
     private String propertyName;
-    private ComparisionOperator op;
+    private QueryOperator op;
     private Object value;
 
-    public HasPredicate(String propertyName, ComparisionOperator op, Object value) {
-        super();
+    public HasPredicate(String propertyName, QueryOperator op, Object value) {
         this.propertyName = propertyName;
         this.op = op;
         this.value = value;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan0/src/main/java/com/thinkaurelius/titan/graphdb/query/graph/GraphCentricQueryBuilder.java
----------------------------------------------------------------------
diff --git a/graphdb/titan0/src/main/java/com/thinkaurelius/titan/graphdb/query/graph/GraphCentricQueryBuilder.java b/graphdb/titan0/src/main/java/com/thinkaurelius/titan/graphdb/query/graph/GraphCentricQueryBuilder.java
index 54ff7cb..8e0928c 100644
--- a/graphdb/titan0/src/main/java/com/thinkaurelius/titan/graphdb/query/graph/GraphCentricQueryBuilder.java
+++ b/graphdb/titan0/src/main/java/com/thinkaurelius/titan/graphdb/query/graph/GraphCentricQueryBuilder.java
@@ -285,7 +285,33 @@ public class GraphCentricQueryBuilder implements TitanGraphQuery<GraphCentricQue
                 }
 
                 if (index.isCompositeIndex()) {
-                    subcondition = indexCover((CompositeIndexType) index, conditions, subcover);
+                    CompositeIndexType compositeIndex = (CompositeIndexType)index;
+
+                    subcondition = indexCover(compositeIndex, conditions, subcover);
+
+                    // if this is unique index, use it!!
+                    if (compositeIndex.getCardinality() == Cardinality.SINGLE && subcondition != null) {
+                        bestCandidate         = null; // will cause the outer while() to bail out
+                        candidateSubcover     = subcover;
+                        candidateSubcondition = subcondition;
+                        candidateSupportsSort = supportsSort;
+
+                        if (log.isDebugEnabled()) {
+                            log.debug("selected unique index {}", compositeIndex.getName());
+                        }
+
+                        if (coveredClauses.isEmpty()) {
+                            isSorted = candidateSupportsSort;
+                        }
+
+                        coveredClauses.clear();;
+                        coveredClauses.addAll(candidateSubcover);
+
+                        jointQuery = new JointIndexQuery();
+                        jointQuery.add(compositeIndex, serializer.getQuery(compositeIndex, (List<Object[]>)candidateSubcondition));
+
+                        break;
+                    }
                 } else {
                     subcondition = indexCover((MixedIndexType) index, conditions, serializer, subcover);
                     if (coveredClauses.isEmpty() && !supportsSort

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0IndexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0IndexQuery.java b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0IndexQuery.java
index 1ed1734..c4a312d 100644
--- a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0IndexQuery.java
+++ b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0IndexQuery.java
@@ -19,6 +19,7 @@ package org.apache.atlas.repository.graphdb.titan0;
 
 import java.util.Iterator;
 
+import com.google.common.base.Preconditions;
 import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 
@@ -56,6 +57,26 @@ public class Titan0IndexQuery implements AtlasIndexQuery<Titan0Vertex, Titan0Edg
         return Iterators.transform(results, function);
     }
 
+    @Override
+    public Iterator<Result<Titan0Vertex, Titan0Edge>> vertices(int offset, int limit) {
+        Preconditions.checkArgument(offset >=0, "Index offset should be greater than or equals to 0");
+        Preconditions.checkArgument(limit >=0, "Index limit should be greater than or equals to 0");
+        Iterator<TitanIndexQuery.Result<Vertex>> results = wrappedIndexQuery
+                .offset(offset)
+                .limit(limit)
+                .vertices().iterator();
+
+        Function<TitanIndexQuery.Result<Vertex>, AtlasIndexQuery.Result<Titan0Vertex, Titan0Edge>> function =
+                new Function<TitanIndexQuery.Result<Vertex>, AtlasIndexQuery.Result<Titan0Vertex, Titan0Edge>>() {
+
+                    @Override
+                    public AtlasIndexQuery.Result<Titan0Vertex, Titan0Edge> apply(TitanIndexQuery.Result<Vertex> source) {
+                        return new ResultImpl(source);
+                    }
+                };
+        return Iterators.transform(results, function);
+    }
+
     private final class ResultImpl implements AtlasIndexQuery.Result<Titan0Vertex, Titan0Edge> {
         private TitanIndexQuery.Result<Vertex> wrappedResult;
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0VertexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0VertexQuery.java b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0VertexQuery.java
index bd8b897..091e7d4 100644
--- a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0VertexQuery.java
+++ b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/Titan0VertexQuery.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas.repository.graphdb.titan0;
 
+import com.google.common.base.Preconditions;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
@@ -53,12 +54,26 @@ public class Titan0VertexQuery implements AtlasVertexQuery<Titan0Vertex, Titan0E
     }
 
     @Override
+    public Iterable<AtlasVertex<Titan0Vertex, Titan0Edge>> vertices(int limit) {
+        Preconditions.checkArgument(limit >=0, "Limit should be greater than or equals to 0");
+        Iterable<Vertex> vertices = vertexQuery.limit(limit).vertices();
+        return graph.wrapVertices(vertices);
+    }
+
+    @Override
     public Iterable<AtlasEdge<Titan0Vertex, Titan0Edge>> edges() {
         Iterable<Edge> edges = vertexQuery.edges();
         return graph.wrapEdges(edges);
     }
 
     @Override
+    public Iterable<AtlasEdge<Titan0Vertex, Titan0Edge>> edges(int limit) {
+        Preconditions.checkArgument(limit >=0, "Limit should be greater than or equals to 0");
+        Iterable<Edge> edges = vertexQuery.limit(limit).edges();
+        return graph.wrapEdges(edges);
+    }
+
+    @Override
     public long count() {
         return vertexQuery.count();
     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/query/NativeTitan0GraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/query/NativeTitan0GraphQuery.java b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/query/NativeTitan0GraphQuery.java
index 7ec6ffe..f1f1adb 100644
--- a/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/query/NativeTitan0GraphQuery.java
+++ b/graphdb/titan0/src/main/java/org/apache/atlas/repository/graphdb/titan0/query/NativeTitan0GraphQuery.java
@@ -17,21 +17,25 @@
  */
 package org.apache.atlas.repository.graphdb.titan0.query;
 
-import java.util.Collection;
-
+import com.google.common.collect.Lists;
+import com.thinkaurelius.titan.core.TitanGraphQuery;
+import com.thinkaurelius.titan.core.attribute.Contain;
+import com.thinkaurelius.titan.core.attribute.Text;
+import com.thinkaurelius.titan.graphdb.query.TitanPredicate;
+import com.tinkerpop.blueprints.Compare;
+import com.tinkerpop.blueprints.Vertex;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.MatchingOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
-import org.apache.atlas.repository.graphdb.titan0.Titan0GraphDatabase;
 import org.apache.atlas.repository.graphdb.titan0.Titan0Edge;
 import org.apache.atlas.repository.graphdb.titan0.Titan0Graph;
+import org.apache.atlas.repository.graphdb.titan0.Titan0GraphDatabase;
 import org.apache.atlas.repository.graphdb.titan0.Titan0Vertex;
 
-import com.thinkaurelius.titan.core.TitanGraphQuery;
-import com.thinkaurelius.titan.core.attribute.Contain;
-import com.thinkaurelius.titan.graphdb.query.TitanPredicate;
-import com.tinkerpop.blueprints.Compare;
+import java.util.*;
 
 /**
  * Titan 0.5.4 implementation of NativeTitanGraphQuery.
@@ -60,6 +64,28 @@ public class NativeTitan0GraphQuery implements NativeTitanGraphQuery<Titan0Verte
         Iterable it = query.edges();
         return graph.wrapEdges(it);
     }
+    @Override
+    public Iterable<AtlasVertex<Titan0Vertex, Titan0Edge>> vertices(int limit) {
+        Iterable it = query.limit(limit).vertices();
+        return graph.wrapVertices(it);
+    }
+
+    @Override
+    public Iterable<AtlasVertex<Titan0Vertex, Titan0Edge>> vertices(int offset, int limit) {
+        List<Vertex>     result = new ArrayList<>(limit);
+        Iterator<Vertex> iter   = query.limit(offset + limit).vertices().iterator();
+
+        for (long resultIdx = 0; iter.hasNext() && result.size() < limit; resultIdx++) {
+            if (resultIdx < offset) {
+                continue;
+            }
+
+            result.add(iter.next());
+        }
+
+        return graph.wrapVertices(result);
+    }
+
 
     @Override
     public void in(String propertyName, Collection<?> values) {
@@ -68,26 +94,48 @@ public class NativeTitan0GraphQuery implements NativeTitanGraphQuery<Titan0Verte
     }
 
     @Override
-    public void has(String propertyName, ComparisionOperator op, Object value) {
-
-        Compare c = getGremlinPredicate(op);
-        TitanPredicate pred = TitanPredicate.Converter.convert(c);
+    public void has(String propertyName, QueryOperator op, Object value) {
+        TitanPredicate pred;
+        if (op instanceof ComparisionOperator) {
+            Compare c = getGremlinPredicate((ComparisionOperator) op);
+            pred = TitanPredicate.Converter.convert(c);
+        } else {
+            pred = getGremlinPredicate((MatchingOperator) op);
+        }
         query.has(propertyName, pred, value);
     }
 
+    private Text getGremlinPredicate(MatchingOperator op) {
+        switch (op) {
+            case CONTAINS:
+                return Text.CONTAINS;
+            case PREFIX:
+                return Text.PREFIX;
+            case SUFFIX:
+                return Text.CONTAINS_REGEX;
+            case REGEX:
+                return Text.REGEX;
+            default:
+                throw new RuntimeException("Unsupported matching operator:" + op);
+        }
+    }
+
     private Compare getGremlinPredicate(ComparisionOperator op) {
         switch (op) {
-        case EQUAL:
-            return Compare.EQUAL;
-        case GREATER_THAN_EQUAL:
-            return Compare.GREATER_THAN_EQUAL;
-        case LESS_THAN_EQUAL:
-            return Compare.LESS_THAN_EQUAL;
-        case NOT_EQUAL:
-            return Compare.NOT_EQUAL;
-
-        default:
-            throw new RuntimeException("Unsupported comparison operator:" + op);
+            case EQUAL:
+                return Compare.EQUAL;
+            case GREATER_THAN:
+                return Compare.GREATER_THAN;
+            case GREATER_THAN_EQUAL:
+                return Compare.GREATER_THAN_EQUAL;
+            case LESS_THAN:
+                return Compare.LESS_THAN;
+            case LESS_THAN_EQUAL:
+                return Compare.LESS_THAN_EQUAL;
+            case NOT_EQUAL:
+                return Compare.NOT_EQUAL;
+            default:
+                throw new RuntimeException("Unsupported comparison operator:" + op);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1IndexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1IndexQuery.java b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1IndexQuery.java
index 4073dd2..c7e4150 100644
--- a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1IndexQuery.java
+++ b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1IndexQuery.java
@@ -19,6 +19,7 @@ package org.apache.atlas.repository.graphdb.titan1;
 
 import java.util.Iterator;
 
+import com.google.common.base.Preconditions;
 import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 
@@ -56,6 +57,27 @@ public class Titan1IndexQuery implements AtlasIndexQuery<Titan1Vertex, Titan1Edg
         return Iterators.transform(results, function);
     }
 
+    @Override
+    public Iterator<Result<Titan1Vertex, Titan1Edge>> vertices(int offset, int limit) {
+        Preconditions.checkArgument(offset >=0, "Index offset should be greater than or equals to 0");
+        Preconditions.checkArgument(limit >=0, "Index limit should be greater than or equals to 0");
+        Iterator<TitanIndexQuery.Result<TitanVertex>> results = query
+                .offset(offset)
+                .limit(limit)
+                .vertices().iterator();
+
+        Function<TitanIndexQuery.Result<TitanVertex>, Result<Titan1Vertex, Titan1Edge>> function =
+                new Function<TitanIndexQuery.Result<TitanVertex>, Result<Titan1Vertex, Titan1Edge>>() {
+
+                    @Override
+                    public Result<Titan1Vertex, Titan1Edge> apply(TitanIndexQuery.Result<TitanVertex> source) {
+                        return new ResultImpl(source);
+                    }
+                };
+
+        return Iterators.transform(results, function);
+    }
+
     /**
      * Titan 1.0.0 implementation of AtlasIndexQuery.Result.
      */

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1VertexQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1VertexQuery.java b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1VertexQuery.java
index 4452bcd..a761a74 100644
--- a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1VertexQuery.java
+++ b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/Titan1VertexQuery.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas.repository.graphdb.titan1;
 
+import com.google.common.base.Preconditions;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
@@ -51,10 +52,23 @@ public class Titan1VertexQuery implements AtlasVertexQuery<Titan1Vertex, Titan1E
     }
 
     @Override
+    public Iterable<AtlasVertex<Titan1Vertex, Titan1Edge>> vertices(int limit) {
+        Preconditions.checkArgument(limit >=0, "Limit should be greater than or equals to 0");
+        Iterable vertices = query.limit(limit).vertices();
+        return graph.wrapVertices(vertices);
+    }
+
+    @Override
     public Iterable<AtlasEdge<Titan1Vertex, Titan1Edge>> edges() {
         Iterable edges = query.edges();
         return graph.wrapEdges(edges);
+    }
 
+    @Override
+    public Iterable<AtlasEdge<Titan1Vertex, Titan1Edge>> edges(int limit) {
+        Preconditions.checkArgument(limit >=0, "Limit should be greater than or equals to 0");
+        Iterable edges = query.limit(limit).edges();
+        return graph.wrapEdges(edges);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/query/NativeTitan1GraphQuery.java
----------------------------------------------------------------------
diff --git a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/query/NativeTitan1GraphQuery.java b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/query/NativeTitan1GraphQuery.java
index 1ca900d..9293dbd 100644
--- a/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/query/NativeTitan1GraphQuery.java
+++ b/graphdb/titan1/src/main/java/org/apache/atlas/repository/graphdb/titan1/query/NativeTitan1GraphQuery.java
@@ -17,11 +17,16 @@
  */
 package org.apache.atlas.repository.graphdb.titan1.query;
 
-import java.util.Collection;
-
 import com.thinkaurelius.titan.core.TitanEdge;
+import com.thinkaurelius.titan.core.TitanGraphQuery;
+import com.thinkaurelius.titan.core.TitanVertex;
+import com.thinkaurelius.titan.core.attribute.Contain;
+import com.thinkaurelius.titan.core.attribute.Text;
+import com.thinkaurelius.titan.graphdb.query.TitanPredicate;
 import org.apache.atlas.repository.graphdb.AtlasEdge;
 import org.apache.atlas.repository.graphdb.AtlasGraphQuery.ComparisionOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.MatchingOperator;
+import org.apache.atlas.repository.graphdb.AtlasGraphQuery.QueryOperator;
 import org.apache.atlas.repository.graphdb.AtlasVertex;
 import org.apache.atlas.repository.graphdb.titan.query.NativeTitanGraphQuery;
 import org.apache.atlas.repository.graphdb.titan1.Titan1Edge;
@@ -29,11 +34,9 @@ import org.apache.atlas.repository.graphdb.titan1.Titan1Graph;
 import org.apache.atlas.repository.graphdb.titan1.Titan1GraphDatabase;
 import org.apache.atlas.repository.graphdb.titan1.Titan1Vertex;
 import org.apache.tinkerpop.gremlin.process.traversal.Compare;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 
-import com.thinkaurelius.titan.core.TitanGraphQuery;
-import com.thinkaurelius.titan.core.TitanVertex;
-import com.thinkaurelius.titan.core.attribute.Contain;
-import com.thinkaurelius.titan.graphdb.query.TitanPredicate;
+import java.util.*;
 
 /**
  * Titan 1.0.0 implementation of NativeTitanGraphQuery.
@@ -61,32 +64,77 @@ public class NativeTitan1GraphQuery implements NativeTitanGraphQuery<Titan1Verte
     }
 
     @Override
+    public Iterable<AtlasVertex<Titan1Vertex, Titan1Edge>> vertices(int limit) {
+        Iterable<TitanVertex> it = query.limit(limit).vertices();
+        return graph.wrapVertices(it);
+    }
+
+    @Override
+    public Iterable<AtlasVertex<Titan1Vertex, Titan1Edge>> vertices(int offset, int limit) {
+        List<Vertex>               result = new ArrayList<>(limit);
+        Iterator<? extends Vertex> iter   = query.limit(offset + limit).vertices().iterator();
+
+        for (long resultIdx = 0; iter.hasNext() && result.size() < limit; resultIdx++) {
+            if (resultIdx < offset) {
+                continue;
+            }
+
+            result.add(iter.next());
+        }
+
+        return graph.wrapVertices(result);
+    }
+
+    @Override
     public void in(String propertyName, Collection<? extends Object> values) {
         query.has(propertyName, Contain.IN, values);
 
     }
 
     @Override
-    public void has(String propertyName, ComparisionOperator op, Object value) {
-
-        Compare c = getGremlinPredicate(op);
-        TitanPredicate pred = TitanPredicate.Converter.convert(c);
+    public void has(String propertyName, QueryOperator op, Object value) {
+        TitanPredicate pred;
+        if (op instanceof ComparisionOperator) {
+            Compare c = getGremlinPredicate((ComparisionOperator) op);
+            pred = TitanPredicate.Converter.convert(c);
+        } else {
+            pred = getGremlinPredicate((MatchingOperator)op);
+        }
         query.has(propertyName, pred, value);
     }
 
+    private Text getGremlinPredicate(MatchingOperator op) {
+        switch (op) {
+            case CONTAINS:
+                return Text.CONTAINS;
+            case PREFIX:
+                return Text.PREFIX;
+            case SUFFIX:
+                return Text.CONTAINS_REGEX;
+            case REGEX:
+                return Text.REGEX;
+            default:
+                throw new RuntimeException("Unsupported matching operator:" + op);
+        }
+    }
+
     private Compare getGremlinPredicate(ComparisionOperator op) {
         switch (op) {
-        case EQUAL:
-            return Compare.eq;
-        case GREATER_THAN_EQUAL:
-            return Compare.gte;
-        case LESS_THAN_EQUAL:
-            return Compare.lte;
-        case NOT_EQUAL:
-            return Compare.neq;
-
-        default:
-            throw new RuntimeException("Unsupported comparison operator:" + op);
+            case EQUAL:
+                return Compare.eq;
+            case GREATER_THAN:
+                return Compare.gt;
+            case GREATER_THAN_EQUAL:
+                return Compare.gte;
+            case LESS_THAN:
+                return Compare.lt;
+            case LESS_THAN_EQUAL:
+                return Compare.lte;
+            case NOT_EQUAL:
+                return Compare.neq;
+
+            default:
+                throw new RuntimeException("Unsupported comparison operator:" + op);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/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 a402c62..9513dcb 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
@@ -39,10 +39,9 @@ import static org.codehaus.jackson.annotate.JsonAutoDetect.Visibility.PUBLIC_ONL
 @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 AtlasQueryType            queryType;
+    private SearchParameters          searchParameters;
     private String                    queryText;
     private String                    type;
     private String                    classification;
@@ -59,11 +58,24 @@ public class AtlasSearchResult implements Serializable {
     public AtlasSearchResult(String queryText, AtlasQueryType queryType) {
         setQueryText(queryText);
         setQueryType(queryType);
+        setSearchParameters(null);
         setEntities(null);
         setAttributes(null);
         setFullTextResult(null);
     }
 
+    public AtlasSearchResult(SearchParameters searchParameters) {
+        setQueryType(AtlasQueryType.BASIC);
+
+        if (searchParameters != null) {
+            setQueryText(searchParameters.getQuery());
+            setSearchParameters(searchParameters);
+            setEntities(null);
+            setAttributes(null);
+            setFullTextResult(null);
+        }
+    }
+
     public AtlasQueryType getQueryType() { return queryType; }
 
     public void setQueryType(AtlasQueryType queryType) { this.queryType = queryType; }
@@ -98,6 +110,7 @@ public class AtlasSearchResult implements Serializable {
         if (o == null || getClass() != o.getClass()) return false;
         AtlasSearchResult that = (AtlasSearchResult) o;
         return Objects.equals(queryType, that.queryType) &&
+               Objects.equals(searchParameters, that.searchParameters) &&
                Objects.equals(queryText, that.queryText) &&
                Objects.equals(type, that.type) &&
                Objects.equals(classification, that.classification) &&
@@ -107,12 +120,13 @@ public class AtlasSearchResult implements Serializable {
     }
 
     @Override
-    public int hashCode() { return Objects.hash(queryText, queryType, entities, attributes, fullTextResult, type, classification); }
+    public int hashCode() { return Objects.hash(queryType, searchParameters, queryText, type, classification, entities, attributes, fullTextResult); }
 
     @Override
     public String toString() {
         return "AtlasSearchResult{" +
                 "queryType=" + queryType +
+                ", searchParameters='" + searchParameters + '\'' +
                 ", queryText='" + queryText + '\'' +
                 ", type=" + type +
                 ", classification=" + classification +
@@ -149,6 +163,14 @@ public class AtlasSearchResult implements Serializable {
         }
     }
 
+    public void setSearchParameters(SearchParameters searchParameters) {
+        this.searchParameters = searchParameters;
+    }
+
+    public SearchParameters getSearchParameters() {
+        return searchParameters;
+    }
+
     public enum AtlasQueryType { DSL, FULL_TEXT, GREMLIN, BASIC, ATTRIBUTE }
 
     @JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/intg/src/main/java/org/apache/atlas/model/discovery/SearchParameters.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/discovery/SearchParameters.java b/intg/src/main/java/org/apache/atlas/model/discovery/SearchParameters.java
new file mode 100644
index 0000000..30855dc
--- /dev/null
+++ b/intg/src/main/java/org/apache/atlas/model/discovery/SearchParameters.java
@@ -0,0 +1,366 @@
+/**
+ * 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.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonValue;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+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)
+public class SearchParameters {
+    private String  query;
+    private String  typeName;
+    private String  classification;
+    private boolean excludeDeletedEntities;
+    private int     limit;
+    private int     offset;
+
+    private FilterCriteria entityFilters;
+    private FilterCriteria tagFilters;
+    private Set<String>    attributes;
+
+    /**
+     * @return The type of query
+     */
+    public String getQuery() {
+        return query;
+    }
+
+    /**
+     * Set query type
+     * @param query type
+     */
+    public void setQuery(String query) {
+        this.query = query;
+    }
+
+    /**
+     * @return Type name to search on
+     */
+    public String getTypeName() {
+        return typeName;
+    }
+
+    /**
+     * Set the type name to search on
+     * @param typeName type name
+     */
+    public void setTypeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     *
+     * @return Classification/tag to search on
+     */
+    public String getClassification() {
+        return classification;
+    }
+
+    /**
+     * Set the classification/tag to search on
+     * @param classification classification/tag name
+     */
+    public void setClassification(String classification) {
+        this.classification = classification;
+    }
+
+    /**
+     * @return True iff deleted entities are excluded
+     */
+    public boolean getExcludeDeletedEntities() {
+        return excludeDeletedEntities;
+    }
+
+    /**
+     * Exclude deleted entities from search
+     * @param excludeDeletedEntities boolean flag
+     */
+    public void setExcludeDeletedEntities(boolean excludeDeletedEntities) {
+        this.excludeDeletedEntities = excludeDeletedEntities;
+    }
+
+    /**
+     * @return Max number of results to be returned
+     */
+    public int getLimit() {
+        return limit;
+    }
+
+    /**
+     * Restrict the results to the specified limit
+     * @param limit max number of results
+     */
+    public void setLimit(int limit) {
+        this.limit = limit;
+    }
+
+    /**
+     * @return Offset(pagination) of the results
+     */
+    public int getOffset() {
+        return offset;
+    }
+
+    /**
+     * @param offset
+     */
+    public void setOffset(int offset) {
+        this.offset = offset;
+    }
+
+    /**
+     * Entity attribute filters for the type (if type name is specified)
+     * @return
+     */
+    public FilterCriteria getEntityFilters() {
+        return entityFilters;
+    }
+
+    /**
+     * Filter the entities on this criteria
+     * @param entityFilters
+     */
+    public void setEntityFilters(FilterCriteria entityFilters) {
+        this.entityFilters = entityFilters;
+    }
+
+    /**
+     * Tag attribute filters for the classification/tag (if tag name is specified)
+     * @return
+     */
+    public FilterCriteria getTagFilters() {
+        return tagFilters;
+    }
+
+    /**
+     * Filter the tag/classification on this criteria
+     * @param tagFilters
+     */
+    public void setTagFilters(FilterCriteria tagFilters) {
+        this.tagFilters = tagFilters;
+    }
+
+    /**
+     * Attribute values included in the results
+     * @return
+     */
+    public Set<String> getAttributes() {
+        return attributes;
+    }
+
+    /**
+     * Return these attributes in the result response
+     * @param attributes
+     */
+    public void setAttributes(Set<String> attributes) {
+        this.attributes = attributes;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        SearchParameters that = (SearchParameters) o;
+        return excludeDeletedEntities == that.excludeDeletedEntities &&
+                limit == that.limit &&
+                offset == that.offset &&
+                Objects.equals(query, that.query) &&
+                Objects.equals(typeName, that.typeName) &&
+                Objects.equals(classification, that.classification) &&
+                Objects.equals(entityFilters, that.entityFilters) &&
+                Objects.equals(tagFilters, that.tagFilters) &&
+                Objects.equals(attributes, that.attributes);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(query, typeName, classification, excludeDeletedEntities, limit, offset, entityFilters, tagFilters, attributes);
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder sb = new StringBuilder("SearchParameters{");
+        sb.append("query='").append(query).append('\'');
+        sb.append(", typeName='").append(typeName).append('\'');
+        sb.append(", classification='").append(classification).append('\'');
+        sb.append(", excludeDeletedEntities=").append(excludeDeletedEntities);
+        sb.append(", limit=").append(limit);
+        sb.append(", offset=").append(offset);
+        sb.append(", entityFilters=").append(entityFilters);
+        sb.append(", tagFilters=").append(tagFilters);
+        sb.append(", attributes=").append(attributes);
+        sb.append('}');
+        return sb.toString();
+    }
+
+
+    @JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
+    @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+    @JsonIgnoreProperties(ignoreUnknown = true)
+    public static class FilterCriteria {
+        // Can be presented as a group of conditions or a single condition
+        public enum Condition { AND, OR }
+
+        // Single condition
+        private String   attributeName;
+        private Operator operator;
+        private String   attributeValue;
+
+        // Complex conditions
+        private Condition            condition;
+        private List<FilterCriteria> criterion;
+
+        public String getAttributeName() {
+            return attributeName;
+        }
+
+        public void setAttributeName(String attributeName) {
+            this.attributeName = attributeName;
+        }
+
+        public Operator getOperator() {
+            return operator;
+        }
+
+        public void setOperator(Operator operator) {
+            this.operator = operator;
+        }
+
+        public String getAttributeValue() {
+            return attributeValue;
+        }
+
+        public void setAttributeValue(String attributeValue) {
+            this.attributeValue = attributeValue;
+        }
+
+        public Condition getCondition() {
+            return condition;
+        }
+
+        public void setCondition(Condition condition) {
+            this.condition = condition;
+        }
+
+        public List<FilterCriteria> getCriterion() {
+            return criterion;
+        }
+
+        public void setCriterion(List<FilterCriteria> criterion) {
+            this.criterion = criterion;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            FilterCriteria that = (FilterCriteria) o;
+            return Objects.equals(attributeName, that.attributeName) &&
+                    Objects.equals(operator, that.operator) &&
+                    Objects.equals(attributeValue, that.attributeValue) &&
+                    condition == that.condition &&
+                    Objects.equals(criterion, that.criterion);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(attributeName, operator, attributeValue, condition, criterion);
+        }
+
+        @Override
+        public String toString() {
+            final StringBuilder sb = new StringBuilder("FilterCriteria{");
+            sb.append("attributeName='").append(attributeName).append('\'');
+            sb.append(", operator=").append(operator);
+            sb.append(", attributeValue='").append(attributeValue).append('\'');
+            sb.append(", condition=").append(condition);
+            sb.append(", criterion=").append(criterion);
+            sb.append('}');
+            return sb.toString();
+        }
+    }
+
+    /**
+     * Supported search operations
+     * Logical comparision operators can only be used with numbers or dates
+     * IN, LIKE, startsWith, endsWith, CONTAINS can only be used with strings or text
+     */
+    public enum Operator {
+        LT(new String[]{"<", "lt"}),
+        GT(new String[]{">", "gt"}),
+        LTE(new String[]{"<=", "lte"}),
+        GTE(new String[]{">=", "gte"}),
+        EQ(new String[]{"eq", "="}),
+        NEQ(new String[]{"neq", "!="}),
+        IN(new String[]{"in", "IN"}),
+        LIKE(new String[]{"like", "LIKE"}),
+        STARTS_WITH(new String[]{"startsWith", "STARTSWITH", "begins_with", "BEGINS_WITH"}),
+        ENDS_WITH(new String[]{"endsWith", "ENDSWITH", "ends_with", "BEGINS_WITH"}),
+        CONTAINS(new String[]{"contains", "CONTAINS"})
+        ;
+        static final Map<String, Operator> operatorsMap = new HashMap<>();
+
+        private String[] symbols;
+
+        static  {
+            for (Operator operator : Operator.values()) {
+                for (String s : operator.symbols) {
+                    operatorsMap.put(s, operator);
+                }
+            }
+        }
+
+        Operator(String[] symbols) {
+            this.symbols = symbols;
+        }
+
+        @JsonCreator
+        public static Operator fromString(String symbol) {
+            return operatorsMap.get(symbol);
+        }
+
+        @JsonValue
+        public String getSymbol() {
+            return symbols[0];
+        }
+
+        public String[] getSymbols() {
+            return symbols;
+        }
+
+        @Override
+        public String toString() {
+            return getSymbol();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/intg/src/main/java/org/apache/atlas/model/impexp/AtlasImportRequest.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/model/impexp/AtlasImportRequest.java b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasImportRequest.java
index 21b6427..4d2ac62 100644
--- a/intg/src/main/java/org/apache/atlas/model/impexp/AtlasImportRequest.java
+++ b/intg/src/main/java/org/apache/atlas/model/impexp/AtlasImportRequest.java
@@ -18,14 +18,12 @@
 package org.apache.atlas.model.impexp;
 
 import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.codehaus.jackson.annotate.JsonAnySetter;
 import org.codehaus.jackson.annotate.JsonAutoDetect;
 import org.codehaus.jackson.annotate.JsonIgnore;
 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.HashMap;
 import java.util.Map;
@@ -33,12 +31,9 @@ import java.util.Map;
 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 AtlasImportRequest implements Serializable {
     private static final long   serialVersionUID = 1L;
     public  static final String TRANSFORMS_KEY   = "transforms";
@@ -97,4 +92,10 @@ public class AtlasImportRequest implements Serializable {
 
         return (String) this.options.get(key);
     }
-}
+ @JsonAnySetter
+    public void setOption(String key, String value) {
+        if (null == options) {
+            options = new HashMap<>();
+        }
+        options.put(key, value);
+    }}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/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 923a198..030a957 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/AtlasDiscoveryService.java
@@ -21,6 +21,7 @@ package org.apache.atlas.discovery;
 
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.discovery.SearchParameters;
 
 public interface AtlasDiscoveryService {
     /**
@@ -56,4 +57,12 @@ public interface AtlasDiscoveryService {
      */
     AtlasSearchResult searchUsingBasicQuery(String query, String type, String classification, String attrName,
                                             String attrValuePrefix, boolean excludeDeletedEntities, int limit, int offset) throws AtlasBaseException;
+
+    /**
+     * Search for entities matching the search criteria
+     * @param searchParameters Search criteria
+     * @return Matching entities
+     * @throws AtlasBaseException
+     */
+    AtlasSearchResult searchUsingBasicQuery(SearchParameters searchParameters) throws AtlasBaseException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/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 128cdbf..5068fa5 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntityDiscoveryService.java
@@ -20,12 +20,14 @@ package org.apache.atlas.discovery;
 import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasConfiguration;
 import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.annotation.GraphTransaction;
 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.discovery.AtlasSearchResult.AtlasFullTextResult;
 import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasQueryType;
 import org.apache.atlas.model.discovery.AtlasSearchResult.AttributeSearchResult;
+import org.apache.atlas.model.discovery.SearchParameters;
 import org.apache.atlas.model.instance.AtlasEntity.Status;
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
@@ -86,23 +88,28 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
     private final EntityGraphRetriever            entityRetriever;
     private final AtlasGremlinQueryProvider       gremlinQueryProvider;
     private final AtlasTypeRegistry               typeRegistry;
+    private final SearchPipeline                  searchPipeline;
     private final int                             maxResultSetSize;
     private final int                             maxTypesCountInIdxQuery;
     private final int                             maxTagsCountInIdxQuery;
 
     @Inject
-    EntityDiscoveryService(MetadataRepository metadataRepository, AtlasTypeRegistry typeRegistry, AtlasGraph graph) throws AtlasException {
+    EntityDiscoveryService(MetadataRepository metadataRepository, AtlasTypeRegistry typeRegistry,
+                           AtlasGraph graph, SearchPipeline searchPipeline) throws AtlasException {
         this.graph                    = graph;
         this.graphPersistenceStrategy = new DefaultGraphPersistenceStrategy(metadataRepository);
         this.entityRetriever          = new EntityGraphRetriever(typeRegistry);
         this.gremlinQueryProvider     = AtlasGremlinQueryProvider.INSTANCE;
         this.typeRegistry             = typeRegistry;
-        this.maxResultSetSize         = ApplicationProperties.get().getInt("atlas.graph.index.search.max-result-set-size", 150);
-        this.maxTypesCountInIdxQuery  = ApplicationProperties.get().getInt("atlas.graph.index.search.max-types-count", 10);
-        this.maxTagsCountInIdxQuery   = ApplicationProperties.get().getInt("atlas.graph.index.search.max-tags-count", 10);
+        this.searchPipeline           = searchPipeline;
+
+        this.maxResultSetSize         = ApplicationProperties.get().getInt(Constants.INDEX_SEARCH_MAX_RESULT_SET_SIZE, 150);
+        this.maxTypesCountInIdxQuery  = ApplicationProperties.get().getInt(Constants.INDEX_SEARCH_MAX_TYPES_COUNT, 10);
+        this.maxTagsCountInIdxQuery   = ApplicationProperties.get().getInt(Constants.INDEX_SEARCH_MAX_TAGS_COUNT, 10);
     }
 
     @Override
+    @GraphTransaction
     public AtlasSearchResult searchUsingDslQuery(String dslQuery, int limit, int offset) throws AtlasBaseException {
         AtlasSearchResult ret = new AtlasSearchResult(dslQuery, AtlasQueryType.DSL);
         GremlinQuery gremlinQuery = toGremlinQuery(dslQuery, limit, offset);
@@ -155,6 +162,7 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
     }
 
     @Override
+    @GraphTransaction
     public AtlasSearchResult searchUsingFullTextQuery(String fullTextQuery, boolean excludeDeletedEntities, int limit, int offset)
                                                       throws AtlasBaseException {
         AtlasSearchResult ret      = new AtlasSearchResult(fullTextQuery, AtlasQueryType.FULL_TEXT);
@@ -170,6 +178,7 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
     }
 
     @Override
+    @GraphTransaction
     public AtlasSearchResult searchUsingBasicQuery(String query, String typeName, String classification, String attrName,
                                                    String attrValuePrefix, boolean excludeDeletedEntities, int limit,
                                                    int offset) throws AtlasBaseException {
@@ -393,6 +402,22 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
         return ret;
     }
 
+    @Override
+    @GraphTransaction
+    public AtlasSearchResult searchUsingBasicQuery(SearchParameters searchParameters) throws AtlasBaseException {
+        AtlasSearchResult ret = new AtlasSearchResult(searchParameters);
+
+        List<AtlasVertex> resultList = searchPipeline.run(searchParameters);
+
+        for (AtlasVertex atlasVertex : resultList) {
+            AtlasEntityHeader entity = entityRetriever.toAtlasEntityHeader(atlasVertex, searchParameters.getAttributes());
+
+            ret.addEntity(entity);
+        }
+
+        return ret;
+    }
+
     private String getQueryForFullTextSearch(String userKeyedString, String typeName, String classification) {
         String typeFilter          = getTypeFilter(typeRegistry, typeName, maxTypesCountInIdxQuery);
         String classficationFilter = getClassificationFilter(typeRegistry, classification, maxTagsCountInIdxQuery);
@@ -548,4 +573,5 @@ public class EntityDiscoveryService implements AtlasDiscoveryService {
     public int getMaxResultSetSize() {
         return maxResultSetSize;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/7c262b40/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 b07091a..3ae41c8 100644
--- a/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
+++ b/repository/src/main/java/org/apache/atlas/discovery/EntityLineageService.java
@@ -21,6 +21,7 @@ package org.apache.atlas.discovery;
 
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasErrorCode;
+import org.apache.atlas.annotation.GraphTransaction;
 import org.apache.atlas.exception.AtlasBaseException;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.atlas.model.lineage.AtlasLineageInfo;
@@ -62,6 +63,7 @@ public class EntityLineageService implements AtlasLineageService {
     }
 
     @Override
+    @GraphTransaction
     public AtlasLineageInfo getAtlasLineageInfo(String guid, LineageDirection direction, int depth) throws AtlasBaseException {
         AtlasLineageInfo lineageInfo;
 


[4/5] incubator-atlas git commit: ATLAS-1894: Search using entity/tag attribute filters

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.js b/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.js
deleted file mode 100644
index bb930fb..0000000
--- a/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.js
+++ /dev/null
@@ -1,2444 +0,0 @@
-/*! version : 4.14.30
- =========================================================
- bootstrap-datetimejs
- https://github.com/Eonasdan/bootstrap-datetimepicker
- Copyright (c) 2015 Jonathan Peterson
- =========================================================
- */
-/*
- The MIT License (MIT)
-
- Copyright (c) 2015 Jonathan Peterson
-
- Permission is hereby granted, free of charge, to any person obtaining a copy
- of this software and associated documentation files (the "Software"), to deal
- in the Software without restriction, including without limitation the rights
- to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
- copies of the Software, and to permit persons to whom the Software is
- furnished to do so, subject to the following conditions:
-
- The above copyright notice and this permission notice shall be included in
- all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
- IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
- FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
- AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
- LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
- OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
- THE SOFTWARE.
- */
-/*global define:false */
-/*global exports:false */
-/*global require:false */
-/*global jQuery:false */
-/*global moment:false */
-(function (factory) {
-    'use strict';
-    if (typeof define === 'function' && define.amd) {
-        // AMD is used - Register as an anonymous module.
-        define(['jquery', 'moment'], factory);
-    } else if (typeof exports === 'object') {
-        factory(require('jquery'), require('moment'));
-    } else {
-        // Neither AMD nor CommonJS used. Use global variables.
-        if (typeof jQuery === 'undefined') {
-            throw 'bootstrap-datetimepicker requires jQuery to be loaded first';
-        }
-        if (typeof moment === 'undefined') {
-            throw 'bootstrap-datetimepicker requires Moment.js to be loaded first';
-        }
-        factory(jQuery, moment);
-    }
-}(function ($, moment) {
-    'use strict';
-    if (!moment) {
-        throw new Error('bootstrap-datetimepicker requires Moment.js to be loaded first');
-    }
-
-    var dateTimePicker = function (element, options) {
-        var picker = {},
-            date = moment().startOf('d'),
-            viewDate = date.clone(),
-            unset = true,
-            input,
-            component = false,
-            widget = false,
-            use24Hours,
-            minViewModeNumber = 0,
-            actualFormat,
-            parseFormats,
-            currentViewMode,
-            datePickerModes = [
-                {
-                    clsName: 'days',
-                    navFnc: 'M',
-                    navStep: 1
-                },
-                {
-                    clsName: 'months',
-                    navFnc: 'y',
-                    navStep: 1
-                },
-                {
-                    clsName: 'years',
-                    navFnc: 'y',
-                    navStep: 10
-                },
-                {
-                    clsName: 'decades',
-                    navFnc: 'y',
-                    navStep: 100
-                }
-            ],
-            viewModes = ['days', 'months', 'years', 'decades'],
-            verticalModes = ['top', 'bottom', 'auto'],
-            horizontalModes = ['left', 'right', 'auto'],
-            toolbarPlacements = ['default', 'top', 'bottom'],
-            keyMap = {
-                'up': 38,
-                38: 'up',
-                'down': 40,
-                40: 'down',
-                'left': 37,
-                37: 'left',
-                'right': 39,
-                39: 'right',
-                'tab': 9,
-                9: 'tab',
-                'escape': 27,
-                27: 'escape',
-                'enter': 13,
-                13: 'enter',
-                'pageUp': 33,
-                33: 'pageUp',
-                'pageDown': 34,
-                34: 'pageDown',
-                'shift': 16,
-                16: 'shift',
-                'control': 17,
-                17: 'control',
-                'space': 32,
-                32: 'space',
-                't': 84,
-                84: 't',
-                'delete': 46,
-                46: 'delete'
-            },
-            keyState = {},
-
-            /********************************************************************************
-             *
-             * Private functions
-             *
-             ********************************************************************************/
-            isEnabled = function (granularity) {
-                if (typeof granularity !== 'string' || granularity.length > 1) {
-                    throw new TypeError('isEnabled expects a single character string parameter');
-                }
-                switch (granularity) {
-                    case 'y':
-                        return actualFormat.indexOf('Y') !== -1;
-                    case 'M':
-                        return actualFormat.indexOf('M') !== -1;
-                    case 'd':
-                        return actualFormat.toLowerCase().indexOf('d') !== -1;
-                    case 'h':
-                    case 'H':
-                        return actualFormat.toLowerCase().indexOf('h') !== -1;
-                    case 'm':
-                        return actualFormat.indexOf('m') !== -1;
-                    case 's':
-                        return actualFormat.indexOf('s') !== -1;
-                    default:
-                        return false;
-                }
-            },
-            hasTime = function () {
-                return (isEnabled('h') || isEnabled('m') || isEnabled('s'));
-            },
-
-            hasDate = function () {
-                return (isEnabled('y') || isEnabled('M') || isEnabled('d'));
-            },
-
-            getDatePickerTemplate = function () {
-                var headTemplate = $('<thead>')
-                        .append($('<tr>')
-                            .append($('<th>').addClass('prev').attr('data-action', 'previous')
-                                .append($('<span>').addClass(options.icons.previous))
-                                )
-                            .append($('<th>').addClass('picker-switch').attr('data-action', 'pickerSwitch').attr('colspan', (options.calendarWeeks ? '6' : '5')))
-                            .append($('<th>').addClass('next').attr('data-action', 'next')
-                                .append($('<span>').addClass(options.icons.next))
-                                )
-                            ),
-                    contTemplate = $('<tbody>')
-                        .append($('<tr>')
-                            .append($('<td>').attr('colspan', (options.calendarWeeks ? '8' : '7')))
-                            );
-
-                return [
-                    $('<div>').addClass('datepicker-days')
-                        .append($('<table>').addClass('table-condensed')
-                            .append(headTemplate)
-                            .append($('<tbody>'))
-                            ),
-                    $('<div>').addClass('datepicker-months')
-                        .append($('<table>').addClass('table-condensed')
-                            .append(headTemplate.clone())
-                            .append(contTemplate.clone())
-                            ),
-                    $('<div>').addClass('datepicker-years')
-                        .append($('<table>').addClass('table-condensed')
-                            .append(headTemplate.clone())
-                            .append(contTemplate.clone())
-                            ),
-                    $('<div>').addClass('datepicker-decades')
-                        .append($('<table>').addClass('table-condensed')
-                            .append(headTemplate.clone())
-                            .append(contTemplate.clone())
-                            )
-                ];
-            },
-
-            getTimePickerMainTemplate = function () {
-                var topRow = $('<tr>'),
-                    middleRow = $('<tr>'),
-                    bottomRow = $('<tr>');
-
-                if (isEnabled('h')) {
-                    topRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Increment Hour'}).addClass('btn').attr('data-action', 'incrementHours')
-                            .append($('<span>').addClass(options.icons.up))));
-                    middleRow.append($('<td>')
-                        .append($('<span>').addClass('timepicker-hour').attr({'data-time-component':'hours', 'title':'Pick Hour'}).attr('data-action', 'showHours')));
-                    bottomRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Decrement Hour'}).addClass('btn').attr('data-action', 'decrementHours')
-                            .append($('<span>').addClass(options.icons.down))));
-                }
-                if (isEnabled('m')) {
-                    if (isEnabled('h')) {
-                        topRow.append($('<td>').addClass('separator'));
-                        middleRow.append($('<td>').addClass('separator').html(':'));
-                        bottomRow.append($('<td>').addClass('separator'));
-                    }
-                    topRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Increment Minute'}).addClass('btn').attr('data-action', 'incrementMinutes')
-                            .append($('<span>').addClass(options.icons.up))));
-                    middleRow.append($('<td>')
-                        .append($('<span>').addClass('timepicker-minute').attr({'data-time-component': 'minutes', 'title':'Pick Minute'}).attr('data-action', 'showMinutes')));
-                    bottomRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Decrement Minute'}).addClass('btn').attr('data-action', 'decrementMinutes')
-                            .append($('<span>').addClass(options.icons.down))));
-                }
-                if (isEnabled('s')) {
-                    if (isEnabled('m')) {
-                        topRow.append($('<td>').addClass('separator'));
-                        middleRow.append($('<td>').addClass('separator').html(':'));
-                        bottomRow.append($('<td>').addClass('separator'));
-                    }
-                    topRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Increment Second'}).addClass('btn').attr('data-action', 'incrementSeconds')
-                            .append($('<span>').addClass(options.icons.up))));
-                    middleRow.append($('<td>')
-                        .append($('<span>').addClass('timepicker-second').attr({'data-time-component': 'seconds', 'title':'Pick Second'}).attr('data-action', 'showSeconds')));
-                    bottomRow.append($('<td>')
-                        .append($('<a>').attr({href: '#', tabindex: '-1', 'title':'Decrement Second'}).addClass('btn').attr('data-action', 'decrementSeconds')
-                            .append($('<span>').addClass(options.icons.down))));
-                }
-
-                if (!use24Hours) {
-                    topRow.append($('<td>').addClass('separator'));
-                    middleRow.append($('<td>')
-                        .append($('<button>').addClass('btn btn-primary').attr({'data-action': 'togglePeriod', tabindex: '-1', 'title':'Toggle Period'})));
-                    bottomRow.append($('<td>').addClass('separator'));
-                }
-
-                return $('<div>').addClass('timepicker-picker')
-                    .append($('<table>').addClass('table-condensed')
-                        .append([topRow, middleRow, bottomRow]));
-            },
-
-            getTimePickerTemplate = function () {
-                var hoursView = $('<div>').addClass('timepicker-hours')
-                        .append($('<table>').addClass('table-condensed')),
-                    minutesView = $('<div>').addClass('timepicker-minutes')
-                        .append($('<table>').addClass('table-condensed')),
-                    secondsView = $('<div>').addClass('timepicker-seconds')
-                        .append($('<table>').addClass('table-condensed')),
-                    ret = [getTimePickerMainTemplate()];
-
-                if (isEnabled('h')) {
-                    ret.push(hoursView);
-                }
-                if (isEnabled('m')) {
-                    ret.push(minutesView);
-                }
-                if (isEnabled('s')) {
-                    ret.push(secondsView);
-                }
-
-                return ret;
-            },
-
-            getToolbar = function () {
-                var row = [];
-                if (options.showTodayButton) {
-                    row.push($('<td>').append($('<a>').attr({'data-action':'today', 'title':'Go to today'}).append($('<span>').addClass(options.icons.today))));
-                }
-                if (!options.sideBySide && hasDate() && hasTime()) {
-                    row.push($('<td>').append($('<a>').attr({'data-action':'togglePicker', 'title':'Select Time'}).append($('<span>').addClass(options.icons.time))));
-                }
-                if (options.showClear) {
-                    row.push($('<td>').append($('<a>').attr({'data-action':'clear', 'title':'Clear selection'}).append($('<span>').addClass(options.icons.clear))));
-                }
-                if (options.showClose) {
-                    row.push($('<td>').append($('<a>').attr({'data-action':'close', 'title':'Close the picker'}).append($('<span>').addClass(options.icons.close))));
-                }
-                return $('<table>').addClass('table-condensed').append($('<tbody>').append($('<tr>').append(row)));
-            },
-
-            getTemplate = function () {
-                var template = $('<div>').addClass('bootstrap-datetimepicker-widget dropdown-menu'),
-                    dateView = $('<div>').addClass('datepicker').append(getDatePickerTemplate()),
-                    timeView = $('<div>').addClass('timepicker').append(getTimePickerTemplate()),
-                    content = $('<ul>').addClass('list-unstyled'),
-                    toolbar = $('<li>').addClass('picker-switch' + (options.collapse ? ' accordion-toggle' : '')).append(getToolbar());
-
-                if (options.inline) {
-                    template.removeClass('dropdown-menu');
-                }
-
-                if (use24Hours) {
-                    template.addClass('usetwentyfour');
-                }
-                if (isEnabled('s') && !use24Hours) {
-                    template.addClass('wider');
-                }
-                if (options.sideBySide && hasDate() && hasTime()) {
-                    template.addClass('timepicker-sbs');
-                    template.append(
-                        $('<div>').addClass('row')
-                            .append(dateView.addClass('col-sm-6'))
-                            .append(timeView.addClass('col-sm-6'))
-                    );
-                    template.append(toolbar);
-                    return template;
-                }
-
-                if (options.toolbarPlacement === 'top') {
-                    content.append(toolbar);
-                }
-                if (hasDate()) {
-                    content.append($('<li>').addClass((options.collapse && hasTime() ? 'collapse in' : '')).append(dateView));
-                }
-                if (options.toolbarPlacement === 'default') {
-                    content.append(toolbar);
-                }
-                if (hasTime()) {
-                    content.append($('<li>').addClass((options.collapse && hasDate() ? 'collapse' : '')).append(timeView));
-                }
-                if (options.toolbarPlacement === 'bottom') {
-                    content.append(toolbar);
-                }
-                return template.append(content);
-            },
-
-            dataToOptions = function () {
-                var eData,
-                    dataOptions = {};
-
-                if (element.is('input') || options.inline) {
-                    eData = element.data();
-                } else {
-                    eData = element.find('input').data();
-                }
-
-                if (eData.dateOptions && eData.dateOptions instanceof Object) {
-                    dataOptions = $.extend(true, dataOptions, eData.dateOptions);
-                }
-
-                $.each(options, function (key) {
-                    var attributeName = 'date' + key.charAt(0).toUpperCase() + key.slice(1);
-                    if (eData[attributeName] !== undefined) {
-                        dataOptions[key] = eData[attributeName];
-                    }
-                });
-                return dataOptions;
-            },
-
-            place = function () {
-                var position = (component || element).position(),
-                    offset = (component || element).offset(),
-                    vertical = options.widgetPositioning.vertical,
-                    horizontal = options.widgetPositioning.horizontal,
-                    parent;
-
-                if (options.widgetParent) {
-                    parent = options.widgetParent.append(widget);
-                } else if (element.is('input')) {
-                    parent = element.after(widget).parent();
-                } else if (options.inline) {
-                    parent = element.append(widget);
-                    return;
-                } else {
-                    parent = element;
-                    element.children().first().after(widget);
-                }
-
-                // Top and bottom logic
-                if (vertical === 'auto') {
-                    if (offset.top + widget.height() * 1.5 >= $(window).height() + $(window).scrollTop() &&
-                        widget.height() + element.outerHeight() < offset.top) {
-                        vertical = 'top';
-                    } else {
-                        vertical = 'bottom';
-                    }
-                }
-
-                // Left and right logic
-                if (horizontal === 'auto') {
-                    if (parent.width() < offset.left + widget.outerWidth() / 2 &&
-                        offset.left + widget.outerWidth() > $(window).width()) {
-                        horizontal = 'right';
-                    } else {
-                        horizontal = 'left';
-                    }
-                }
-
-                if (vertical === 'top') {
-                    widget.addClass('top').removeClass('bottom');
-                } else {
-                    widget.addClass('bottom').removeClass('top');
-                }
-
-                if (horizontal === 'right') {
-                    widget.addClass('pull-right');
-                } else {
-                    widget.removeClass('pull-right');
-                }
-
-                // find the first parent element that has a relative css positioning
-                if (parent.css('position') !== 'relative') {
-                    parent = parent.parents().filter(function () {
-                        return $(this).css('position') === 'relative';
-                    }).first();
-                }
-
-                if (parent.length === 0) {
-                    throw new Error('datetimepicker component should be placed within a relative positioned container');
-                }
-
-                widget.css({
-                    top: vertical === 'top' ? 'auto' : position.top + element.outerHeight(),
-                    bottom: vertical === 'top' ? position.top + element.outerHeight() : 'auto',
-                    left: horizontal === 'left' ? (parent === element ? 0 : position.left) : 'auto',
-                    right: horizontal === 'left' ? 'auto' : parent.outerWidth() - element.outerWidth() - (parent === element ? 0 : position.left)
-                });
-            },
-
-            notifyEvent = function (e) {
-                if (e.type === 'dp.change' && ((e.date && e.date.isSame(e.oldDate)) || (!e.date && !e.oldDate))) {
-                    return;
-                }
-                element.trigger(e);
-            },
-
-            viewUpdate = function (e) {
-                if (e === 'y') {
-                    e = 'YYYY';
-                }
-                notifyEvent({
-                    type: 'dp.update',
-                    change: e,
-                    viewDate: viewDate.clone()
-                });
-            },
-
-            showMode = function (dir) {
-                if (!widget) {
-                    return;
-                }
-                if (dir) {
-                    currentViewMode = Math.max(minViewModeNumber, Math.min(3, currentViewMode + dir));
-                }
-                widget.find('.datepicker > div').hide().filter('.datepicker-' + datePickerModes[currentViewMode].clsName).show();
-            },
-
-            fillDow = function () {
-                var row = $('<tr>'),
-                    currentDate = viewDate.clone().startOf('w').startOf('d');
-
-                if (options.calendarWeeks === true) {
-                    row.append($('<th>').addClass('cw').text('#'));
-                }
-
-                while (currentDate.isBefore(viewDate.clone().endOf('w'))) {
-                    row.append($('<th>').addClass('dow').text(currentDate.format('dd')));
-                    currentDate.add(1, 'd');
-                }
-                widget.find('.datepicker-days thead').append(row);
-            },
-
-            isInDisabledDates = function (testDate) {
-                return options.disabledDates[testDate.format('YYYY-MM-DD')] === true;
-            },
-
-            isInEnabledDates = function (testDate) {
-                return options.enabledDates[testDate.format('YYYY-MM-DD')] === true;
-            },
-
-            isInDisabledHours = function (testDate) {
-                return options.disabledHours[testDate.format('H')] === true;
-            },
-
-            isInEnabledHours = function (testDate) {
-                return options.enabledHours[testDate.format('H')] === true;
-            },
-
-            isValid = function (targetMoment, granularity) {
-                if (!targetMoment.isValid()) {
-                    return false;
-                }
-                if (options.disabledDates && granularity === 'd' && isInDisabledDates(targetMoment)) {
-                    return false;
-                }
-                if (options.enabledDates && granularity === 'd' && !isInEnabledDates(targetMoment)) {
-                    return false;
-                }
-                if (options.minDate && targetMoment.isBefore(options.minDate, granularity)) {
-                    return false;
-                }
-                if (options.maxDate && targetMoment.isAfter(options.maxDate, granularity)) {
-                    return false;
-                }
-                if (options.daysOfWeekDisabled && granularity === 'd' && options.daysOfWeekDisabled.indexOf(targetMoment.day()) !== -1) {
-                    return false;
-                }
-                if (options.disabledHours && (granularity === 'h' || granularity === 'm' || granularity === 's') && isInDisabledHours(targetMoment)) {
-                    return false;
-                }
-                if (options.enabledHours && (granularity === 'h' || granularity === 'm' || granularity === 's') && !isInEnabledHours(targetMoment)) {
-                    return false;
-                }
-                if (options.disabledTimeIntervals && (granularity === 'h' || granularity === 'm' || granularity === 's')) {
-                    var found = false;
-                    $.each(options.disabledTimeIntervals, function () {
-                        if (targetMoment.isBetween(this[0], this[1])) {
-                            found = true;
-                            return false;
-                        }
-                    });
-                    if (found) {
-                        return false;
-                    }
-                }
-                return true;
-            },
-
-            fillMonths = function () {
-                var spans = [],
-                    monthsShort = viewDate.clone().startOf('y').startOf('d');
-                while (monthsShort.isSame(viewDate, 'y')) {
-                    spans.push($('<span>').attr('data-action', 'selectMonth').addClass('month').text(monthsShort.format('MMM')));
-                    monthsShort.add(1, 'M');
-                }
-                widget.find('.datepicker-months td').empty().append(spans);
-            },
-
-            updateMonths = function () {
-                var monthsView = widget.find('.datepicker-months'),
-                    monthsViewHeader = monthsView.find('th'),
-                    months = monthsView.find('tbody').find('span');
-
-                monthsViewHeader.eq(0).find('span').attr('title', 'Previous Year');
-                monthsViewHeader.eq(1).attr('title', 'Select Year');
-                monthsViewHeader.eq(2).find('span').attr('title', 'Next Year');
-
-                monthsView.find('.disabled').removeClass('disabled');
-
-                if (!isValid(viewDate.clone().subtract(1, 'y'), 'y')) {
-                    monthsViewHeader.eq(0).addClass('disabled');
-                }
-
-                monthsViewHeader.eq(1).text(viewDate.year());
-
-                if (!isValid(viewDate.clone().add(1, 'y'), 'y')) {
-                    monthsViewHeader.eq(2).addClass('disabled');
-                }
-
-                months.removeClass('active');
-                if (date.isSame(viewDate, 'y') && !unset) {
-                    months.eq(date.month()).addClass('active');
-                }
-
-                months.each(function (index) {
-                    if (!isValid(viewDate.clone().month(index), 'M')) {
-                        $(this).addClass('disabled');
-                    }
-                });
-            },
-
-            updateYears = function () {
-                var yearsView = widget.find('.datepicker-years'),
-                    yearsViewHeader = yearsView.find('th'),
-                    startYear = viewDate.clone().subtract(5, 'y'),
-                    endYear = viewDate.clone().add(6, 'y'),
-                    html = '';
-
-                yearsViewHeader.eq(0).find('span').attr('title', 'Previous Decade');
-                yearsViewHeader.eq(1).attr('title', 'Select Decade');
-                yearsViewHeader.eq(2).find('span').attr('title', 'Next Decade');
-
-                yearsView.find('.disabled').removeClass('disabled');
-
-                if (options.minDate && options.minDate.isAfter(startYear, 'y')) {
-                    yearsViewHeader.eq(0).addClass('disabled');
-                }
-
-                yearsViewHeader.eq(1).text(startYear.year() + '-' + endYear.year());
-
-                if (options.maxDate && options.maxDate.isBefore(endYear, 'y')) {
-                    yearsViewHeader.eq(2).addClass('disabled');
-                }
-
-                while (!startYear.isAfter(endYear, 'y')) {
-                    html += '<span data-action="selectYear" class="year' + (startYear.isSame(date, 'y') && !unset ? ' active' : '') + (!isValid(startYear, 'y') ? ' disabled' : '') + '">' + startYear.year() + '</span>';
-                    startYear.add(1, 'y');
-                }
-
-                yearsView.find('td').html(html);
-            },
-
-            updateDecades = function () {
-                var decadesView = widget.find('.datepicker-decades'),
-                    decadesViewHeader = decadesView.find('th'),
-                    startDecade = viewDate.isBefore(moment({y: 1999})) ? moment({y: 1899}) : moment({y: 1999}),
-                    endDecade = startDecade.clone().add(100, 'y'),
-                    html = '';
-
-                decadesViewHeader.eq(0).find('span').attr('title', 'Previous Century');
-                decadesViewHeader.eq(2).find('span').attr('title', 'Next Century');
-
-                decadesView.find('.disabled').removeClass('disabled');
-
-                if (startDecade.isSame(moment({y: 1900})) || (options.minDate && options.minDate.isAfter(startDecade, 'y'))) {
-                    decadesViewHeader.eq(0).addClass('disabled');
-                }
-
-                decadesViewHeader.eq(1).text(startDecade.year() + '-' + endDecade.year());
-
-                if (startDecade.isSame(moment({y: 2000})) || (options.maxDate && options.maxDate.isBefore(endDecade, 'y'))) {
-                    decadesViewHeader.eq(2).addClass('disabled');
-                }
-
-                while (!startDecade.isAfter(endDecade, 'y')) {
-                    html += '<span data-action="selectDecade" class="decade' + (startDecade.isSame(date, 'y') ? ' active' : '') +
-                        (!isValid(startDecade, 'y') ? ' disabled' : '') + '" data-selection="' + (startDecade.year() + 6) + '">' + (startDecade.year() + 1) + ' - ' + (startDecade.year() + 12) + '</span>';
-                    startDecade.add(12, 'y');
-                }
-                html += '<span></span><span></span><span></span>'; //push the dangling block over, at least this way it's even
-
-                decadesView.find('td').html(html);
-            },
-
-            fillDate = function () {
-                var daysView = widget.find('.datepicker-days'),
-                    daysViewHeader = daysView.find('th'),
-                    currentDate,
-                    html = [],
-                    row,
-                    clsName,
-                    i;
-
-                if (!hasDate()) {
-                    return;
-                }
-
-                daysViewHeader.eq(0).find('span').attr('title', 'Previous Month');
-                daysViewHeader.eq(1).attr('title', 'Select Month');
-                daysViewHeader.eq(2).find('span').attr('title', 'Next Month');
-
-                daysView.find('.disabled').removeClass('disabled');
-                daysViewHeader.eq(1).text(viewDate.format(options.dayViewHeaderFormat));
-
-                if (!isValid(viewDate.clone().subtract(1, 'M'), 'M')) {
-                    daysViewHeader.eq(0).addClass('disabled');
-                }
-                if (!isValid(viewDate.clone().add(1, 'M'), 'M')) {
-                    daysViewHeader.eq(2).addClass('disabled');
-                }
-
-                currentDate = viewDate.clone().startOf('M').startOf('w').startOf('d');
-
-                for (i = 0; i < 42; i++) { //always display 42 days (should show 6 weeks)
-                    if (currentDate.weekday() === 0) {
-                        row = $('<tr>');
-                        if (options.calendarWeeks) {
-                            row.append('<td class="cw">' + currentDate.week() + '</td>');
-                        }
-                        html.push(row);
-                    }
-                    clsName = '';
-                    if (currentDate.isBefore(viewDate, 'M')) {
-                        clsName += ' old';
-                    }
-                    if (currentDate.isAfter(viewDate, 'M')) {
-                        clsName += ' new';
-                    }
-                    if (currentDate.isSame(date, 'd') && !unset) {
-                        clsName += ' active';
-                    }
-                    if (!isValid(currentDate, 'd')) {
-                        clsName += ' disabled';
-                    }
-                    if (currentDate.isSame(moment(), 'd')) {
-                        clsName += ' today';
-                    }
-                    if (currentDate.day() === 0 || currentDate.day() === 6) {
-                        clsName += ' weekend';
-                    }
-                    row.append('<td data-action="selectDay" data-day="' + currentDate.format('L') + '" class="day' + clsName + '">' + currentDate.date() + '</td>');
-                    currentDate.add(1, 'd');
-                }
-
-                daysView.find('tbody').empty().append(html);
-
-                updateMonths();
-
-                updateYears();
-
-                updateDecades();
-            },
-
-            fillHours = function () {
-                var table = widget.find('.timepicker-hours table'),
-                    currentHour = viewDate.clone().startOf('d'),
-                    html = [],
-                    row = $('<tr>');
-
-                if (viewDate.hour() > 11 && !use24Hours) {
-                    currentHour.hour(12);
-                }
-                while (currentHour.isSame(viewDate, 'd') && (use24Hours || (viewDate.hour() < 12 && currentHour.hour() < 12) || viewDate.hour() > 11)) {
-                    if (currentHour.hour() % 4 === 0) {
-                        row = $('<tr>');
-                        html.push(row);
-                    }
-                    row.append('<td data-action="selectHour" class="hour' + (!isValid(currentHour, 'h') ? ' disabled' : '') + '">' + currentHour.format(use24Hours ? 'HH' : 'hh') + '</td>');
-                    currentHour.add(1, 'h');
-                }
-                table.empty().append(html);
-            },
-
-            fillMinutes = function () {
-                var table = widget.find('.timepicker-minutes table'),
-                    currentMinute = viewDate.clone().startOf('h'),
-                    html = [],
-                    row = $('<tr>'),
-                    step = options.stepping === 1 ? 5 : options.stepping;
-
-                while (viewDate.isSame(currentMinute, 'h')) {
-                    if (currentMinute.minute() % (step * 4) === 0) {
-                        row = $('<tr>');
-                        html.push(row);
-                    }
-                    row.append('<td data-action="selectMinute" class="minute' + (!isValid(currentMinute, 'm') ? ' disabled' : '') + '">' + currentMinute.format('mm') + '</td>');
-                    currentMinute.add(step, 'm');
-                }
-                table.empty().append(html);
-            },
-
-            fillSeconds = function () {
-                var table = widget.find('.timepicker-seconds table'),
-                    currentSecond = viewDate.clone().startOf('m'),
-                    html = [],
-                    row = $('<tr>');
-
-                while (viewDate.isSame(currentSecond, 'm')) {
-                    if (currentSecond.second() % 20 === 0) {
-                        row = $('<tr>');
-                        html.push(row);
-                    }
-                    row.append('<td data-action="selectSecond" class="second' + (!isValid(currentSecond, 's') ? ' disabled' : '') + '">' + currentSecond.format('ss') + '</td>');
-                    currentSecond.add(5, 's');
-                }
-
-                table.empty().append(html);
-            },
-
-            fillTime = function () {
-                var toggle, newDate, timeComponents = widget.find('.timepicker span[data-time-component]');
-
-                if (!use24Hours) {
-                    toggle = widget.find('.timepicker [data-action=togglePeriod]');
-                    newDate = date.clone().add((date.hours() >= 12) ? -12 : 12, 'h');
-
-                    toggle.text(date.format('A'));
-
-                    if (isValid(newDate, 'h')) {
-                        toggle.removeClass('disabled');
-                    } else {
-                        toggle.addClass('disabled');
-                    }
-                }
-                timeComponents.filter('[data-time-component=hours]').text(date.format(use24Hours ? 'HH' : 'hh'));
-                timeComponents.filter('[data-time-component=minutes]').text(date.format('mm'));
-                timeComponents.filter('[data-time-component=seconds]').text(date.format('ss'));
-
-                fillHours();
-                fillMinutes();
-                fillSeconds();
-            },
-
-            update = function () {
-                if (!widget) {
-                    return;
-                }
-                fillDate();
-                fillTime();
-            },
-
-            setValue = function (targetMoment) {
-                var oldDate = unset ? null : date;
-
-                // case of calling setValue(null or false)
-                if (!targetMoment) {
-                    unset = true;
-                    input.val('');
-                    element.data('date', '');
-                    notifyEvent({
-                        type: 'dp.change',
-                        date: false,
-                        oldDate: oldDate
-                    });
-                    update();
-                    return;
-                }
-
-                targetMoment = targetMoment.clone().locale(options.locale);
-
-                if (options.stepping !== 1) {
-                    targetMoment.minutes((Math.round(targetMoment.minutes() / options.stepping) * options.stepping) % 60).seconds(0);
-                }
-
-                if (isValid(targetMoment)) {
-                    date = targetMoment;
-                    viewDate = date.clone();
-                    input.val(date.format(actualFormat));
-                    element.data('date', date.format(actualFormat));
-                    unset = false;
-                    update();
-                    notifyEvent({
-                        type: 'dp.change',
-                        date: date.clone(),
-                        oldDate: oldDate
-                    });
-                } else {
-                    if (!options.keepInvalid) {
-                        input.val(unset ? '' : date.format(actualFormat));
-                    }
-                    notifyEvent({
-                        type: 'dp.error',
-                        date: targetMoment
-                    });
-                }
-            },
-
-            hide = function () {
-                ///<summary>Hides the widget. Possibly will emit dp.hide</summary>
-                var transitioning = false;
-                if (!widget) {
-                    return picker;
-                }
-                // Ignore event if in the middle of a picker transition
-                widget.find('.collapse').each(function () {
-                    var collapseData = $(this).data('collapse');
-                    if (collapseData && collapseData.transitioning) {
-                        transitioning = true;
-                        return false;
-                    }
-                    return true;
-                });
-                if (transitioning) {
-                    return picker;
-                }
-                if (component && component.hasClass('btn')) {
-                    component.toggleClass('active');
-                }
-                widget.hide();
-
-                $(window).off('resize', place);
-                widget.off('click', '[data-action]');
-                widget.off('mousedown', false);
-
-                widget.remove();
-                widget = false;
-
-                notifyEvent({
-                    type: 'dp.hide',
-                    date: date.clone()
-                });
-                return picker;
-            },
-
-            clear = function () {
-                setValue(null);
-            },
-
-            /********************************************************************************
-             *
-             * Widget UI interaction functions
-             *
-             ********************************************************************************/
-            actions = {
-                next: function () {
-                    var navFnc = datePickerModes[currentViewMode].navFnc;
-                    viewDate.add(datePickerModes[currentViewMode].navStep, navFnc);
-                    fillDate();
-                    viewUpdate(navFnc);
-                },
-
-                previous: function () {
-                    var navFnc = datePickerModes[currentViewMode].navFnc;
-                    viewDate.subtract(datePickerModes[currentViewMode].navStep, navFnc);
-                    fillDate();
-                    viewUpdate(navFnc);
-                },
-
-                pickerSwitch: function () {
-                    showMode(1);
-                },
-
-                selectMonth: function (e) {
-                    var month = $(e.target).closest('tbody').find('span').index($(e.target));
-                    viewDate.month(month);
-                    if (currentViewMode === minViewModeNumber) {
-                        setValue(date.clone().year(viewDate.year()).month(viewDate.month()));
-                        if (!options.inline) {
-                            hide();
-                        }
-                    } else {
-                        showMode(-1);
-                        fillDate();
-                    }
-                    viewUpdate('M');
-                },
-
-                selectYear: function (e) {
-                    var year = parseInt($(e.target).text(), 10) || 0;
-                    viewDate.year(year);
-                    if (currentViewMode === minViewModeNumber) {
-                        setValue(date.clone().year(viewDate.year()));
-                        if (!options.inline) {
-                            hide();
-                        }
-                    } else {
-                        showMode(-1);
-                        fillDate();
-                    }
-                    viewUpdate('YYYY');
-                },
-
-                selectDecade: function (e) {
-                    var year = parseInt($(e.target).data('selection'), 10) || 0;
-                    viewDate.year(year);
-                    if (currentViewMode === minViewModeNumber) {
-                        setValue(date.clone().year(viewDate.year()));
-                        if (!options.inline) {
-                            hide();
-                        }
-                    } else {
-                        showMode(-1);
-                        fillDate();
-                    }
-                    viewUpdate('YYYY');
-                },
-
-                selectDay: function (e) {
-                    var day = viewDate.clone();
-                    if ($(e.target).is('.old')) {
-                        day.subtract(1, 'M');
-                    }
-                    if ($(e.target).is('.new')) {
-                        day.add(1, 'M');
-                    }
-                    setValue(day.date(parseInt($(e.target).text(), 10)));
-                    if (!hasTime() && !options.keepOpen && !options.inline) {
-                        hide();
-                    }
-                },
-
-                incrementHours: function () {
-                    var newDate = date.clone().add(1, 'h');
-                    if (isValid(newDate, 'h')) {
-                        setValue(newDate);
-                    }
-                },
-
-                incrementMinutes: function () {
-                    var newDate = date.clone().add(options.stepping, 'm');
-                    if (isValid(newDate, 'm')) {
-                        setValue(newDate);
-                    }
-                },
-
-                incrementSeconds: function () {
-                    var newDate = date.clone().add(1, 's');
-                    if (isValid(newDate, 's')) {
-                        setValue(newDate);
-                    }
-                },
-
-                decrementHours: function () {
-                    var newDate = date.clone().subtract(1, 'h');
-                    if (isValid(newDate, 'h')) {
-                        setValue(newDate);
-                    }
-                },
-
-                decrementMinutes: function () {
-                    var newDate = date.clone().subtract(options.stepping, 'm');
-                    if (isValid(newDate, 'm')) {
-                        setValue(newDate);
-                    }
-                },
-
-                decrementSeconds: function () {
-                    var newDate = date.clone().subtract(1, 's');
-                    if (isValid(newDate, 's')) {
-                        setValue(newDate);
-                    }
-                },
-
-                togglePeriod: function () {
-                    setValue(date.clone().add((date.hours() >= 12) ? -12 : 12, 'h'));
-                },
-
-                togglePicker: function (e) {
-                    var $this = $(e.target),
-                        $parent = $this.closest('ul'),
-                        expanded = $parent.find('.in'),
-                        closed = $parent.find('.collapse:not(.in)'),
-                        collapseData;
-
-                    if (expanded && expanded.length) {
-                        collapseData = expanded.data('collapse');
-                        if (collapseData && collapseData.transitioning) {
-                            return;
-                        }
-                        if (expanded.collapse) { // if collapse plugin is available through bootstrap.js then use it
-                            expanded.collapse('hide');
-                            closed.collapse('show');
-                        } else { // otherwise just toggle in class on the two views
-                            expanded.removeClass('in');
-                            closed.addClass('in');
-                        }
-                        if ($this.is('span')) {
-                            $this.toggleClass(options.icons.time + ' ' + options.icons.date);
-                        } else {
-                            $this.find('span').toggleClass(options.icons.time + ' ' + options.icons.date);
-                        }
-
-                        // NOTE: uncomment if toggled state will be restored in show()
-                        //if (component) {
-                        //    component.find('span').toggleClass(options.icons.time + ' ' + options.icons.date);
-                        //}
-                    }
-                },
-
-                showPicker: function () {
-                    widget.find('.timepicker > div:not(.timepicker-picker)').hide();
-                    widget.find('.timepicker .timepicker-picker').show();
-                },
-
-                showHours: function () {
-                    widget.find('.timepicker .timepicker-picker').hide();
-                    widget.find('.timepicker .timepicker-hours').show();
-                },
-
-                showMinutes: function () {
-                    widget.find('.timepicker .timepicker-picker').hide();
-                    widget.find('.timepicker .timepicker-minutes').show();
-                },
-
-                showSeconds: function () {
-                    widget.find('.timepicker .timepicker-picker').hide();
-                    widget.find('.timepicker .timepicker-seconds').show();
-                },
-
-                selectHour: function (e) {
-                    var hour = parseInt($(e.target).text(), 10);
-
-                    if (!use24Hours) {
-                        if (date.hours() >= 12) {
-                            if (hour !== 12) {
-                                hour += 12;
-                            }
-                        } else {
-                            if (hour === 12) {
-                                hour = 0;
-                            }
-                        }
-                    }
-                    setValue(date.clone().hours(hour));
-                    actions.showPicker.call(picker);
-                },
-
-                selectMinute: function (e) {
-                    setValue(date.clone().minutes(parseInt($(e.target).text(), 10)));
-                    actions.showPicker.call(picker);
-                },
-
-                selectSecond: function (e) {
-                    setValue(date.clone().seconds(parseInt($(e.target).text(), 10)));
-                    actions.showPicker.call(picker);
-                },
-
-                clear: clear,
-
-                today: function () {
-                    if (isValid(moment(), 'd')) {
-                        setValue(moment());
-                    }
-                },
-
-                close: hide
-            },
-
-            doAction = function (e) {
-                if ($(e.currentTarget).is('.disabled')) {
-                    return false;
-                }
-                actions[$(e.currentTarget).data('action')].apply(picker, arguments);
-                return false;
-            },
-
-            show = function () {
-                ///<summary>Shows the widget. Possibly will emit dp.show and dp.change</summary>
-                var currentMoment,
-                    useCurrentGranularity = {
-                        'year': function (m) {
-                            return m.month(0).date(1).hours(0).seconds(0).minutes(0);
-                        },
-                        'month': function (m) {
-                            return m.date(1).hours(0).seconds(0).minutes(0);
-                        },
-                        'day': function (m) {
-                            return m.hours(0).seconds(0).minutes(0);
-                        },
-                        'hour': function (m) {
-                            return m.seconds(0).minutes(0);
-                        },
-                        'minute': function (m) {
-                            return m.seconds(0);
-                        }
-                    };
-
-                if (input.prop('disabled') || (!options.ignoreReadonly && input.prop('readonly')) || widget) {
-                    return picker;
-                }
-                if (input.val() !== undefined && input.val().trim().length !== 0) {
-                    setValue(parseInputDate(input.val().trim()));
-                } else if (options.useCurrent && unset && ((input.is('input') && input.val().trim().length === 0) || options.inline)) {
-                    currentMoment = moment();
-                    if (typeof options.useCurrent === 'string') {
-                        currentMoment = useCurrentGranularity[options.useCurrent](currentMoment);
-                    }
-                    setValue(currentMoment);
-                }
-
-                widget = getTemplate();
-
-                fillDow();
-                fillMonths();
-
-                widget.find('.timepicker-hours').hide();
-                widget.find('.timepicker-minutes').hide();
-                widget.find('.timepicker-seconds').hide();
-
-                update();
-                showMode();
-
-                $(window).on('resize', place);
-                widget.on('click', '[data-action]', doAction); // this handles clicks on the widget
-                widget.on('mousedown', false);
-
-                if (component && component.hasClass('btn')) {
-                    component.toggleClass('active');
-                }
-                widget.show();
-                place();
-
-                if (options.focusOnShow && !input.is(':focus')) {
-                    input.focus();
-                }
-
-                notifyEvent({
-                    type: 'dp.show'
-                });
-                return picker;
-            },
-
-            toggle = function () {
-                /// <summary>Shows or hides the widget</summary>
-                return (widget ? hide() : show());
-            },
-
-            parseInputDate = function (inputDate) {
-                if (options.parseInputDate === undefined) {
-                    if (moment.isMoment(inputDate) || inputDate instanceof Date) {
-                        inputDate = moment(inputDate);
-                    } else {
-                        inputDate = moment(inputDate, parseFormats, options.useStrict);
-                    }
-                } else {
-                    inputDate = options.parseInputDate(inputDate);
-                }
-                inputDate.locale(options.locale);
-                return inputDate;
-            },
-
-            keydown = function (e) {
-                var handler = null,
-                    index,
-                    index2,
-                    pressedKeys = [],
-                    pressedModifiers = {},
-                    currentKey = e.which,
-                    keyBindKeys,
-                    allModifiersPressed,
-                    pressed = 'p';
-
-                keyState[currentKey] = pressed;
-
-                for (index in keyState) {
-                    if (keyState.hasOwnProperty(index) && keyState[index] === pressed) {
-                        pressedKeys.push(index);
-                        if (parseInt(index, 10) !== currentKey) {
-                            pressedModifiers[index] = true;
-                        }
-                    }
-                }
-
-                for (index in options.keyBinds) {
-                    if (options.keyBinds.hasOwnProperty(index) && typeof (options.keyBinds[index]) === 'function') {
-                        keyBindKeys = index.split(' ');
-                        if (keyBindKeys.length === pressedKeys.length && keyMap[currentKey] === keyBindKeys[keyBindKeys.length - 1]) {
-                            allModifiersPressed = true;
-                            for (index2 = keyBindKeys.length - 2; index2 >= 0; index2--) {
-                                if (!(keyMap[keyBindKeys[index2]] in pressedModifiers)) {
-                                    allModifiersPressed = false;
-                                    break;
-                                }
-                            }
-                            if (allModifiersPressed) {
-                                handler = options.keyBinds[index];
-                                break;
-                            }
-                        }
-                    }
-                }
-
-                if (handler) {
-                    handler.call(picker, widget);
-                    e.stopPropagation();
-                    e.preventDefault();
-                }
-            },
-
-            keyup = function (e) {
-                keyState[e.which] = 'r';
-                e.stopPropagation();
-                e.preventDefault();
-            },
-
-            change = function (e) {
-                var val = $(e.target).val().trim(),
-                    parsedDate = val ? parseInputDate(val) : null;
-                setValue(parsedDate);
-                e.stopImmediatePropagation();
-                return false;
-            },
-
-            attachDatePickerElementEvents = function () {
-                input.on({
-                    'change': change,
-                    'blur': options.debug ? '' : hide,
-                    'keydown': keydown,
-                    'keyup': keyup,
-                    'focus': options.allowInputToggle ? show : ''
-                });
-
-                if (element.is('input')) {
-                    input.on({
-                        'focus': show
-                    });
-                } else if (component) {
-                    component.on('click', toggle);
-                    component.on('mousedown', false);
-                }
-            },
-
-            detachDatePickerElementEvents = function () {
-                input.off({
-                    'change': change,
-                    'blur': hide,
-                    'keydown': keydown,
-                    'keyup': keyup,
-                    'focus': options.allowInputToggle ? hide : ''
-                });
-
-                if (element.is('input')) {
-                    input.off({
-                        'focus': show
-                    });
-                } else if (component) {
-                    component.off('click', toggle);
-                    component.off('mousedown', false);
-                }
-            },
-
-            indexGivenDates = function (givenDatesArray) {
-                // Store given enabledDates and disabledDates as keys.
-                // This way we can check their existence in O(1) time instead of looping through whole array.
-                // (for example: options.enabledDates['2014-02-27'] === true)
-                var givenDatesIndexed = {};
-                $.each(givenDatesArray, function () {
-                    var dDate = parseInputDate(this);
-                    if (dDate.isValid()) {
-                        givenDatesIndexed[dDate.format('YYYY-MM-DD')] = true;
-                    }
-                });
-                return (Object.keys(givenDatesIndexed).length) ? givenDatesIndexed : false;
-            },
-
-            indexGivenHours = function (givenHoursArray) {
-                // Store given enabledHours and disabledHours as keys.
-                // This way we can check their existence in O(1) time instead of looping through whole array.
-                // (for example: options.enabledHours['2014-02-27'] === true)
-                var givenHoursIndexed = {};
-                $.each(givenHoursArray, function () {
-                    givenHoursIndexed[this] = true;
-                });
-                return (Object.keys(givenHoursIndexed).length) ? givenHoursIndexed : false;
-            },
-
-            initFormatting = function () {
-                var format = options.format || 'L LT';
-
-                actualFormat = format.replace(/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g, function (formatInput) {
-                    var newinput = date.localeData().longDateFormat(formatInput) || formatInput;
-                    return newinput.replace(/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g, function (formatInput2) { //temp fix for #740
-                        return date.localeData().longDateFormat(formatInput2) || formatInput2;
-                    });
-                });
-
-
-                parseFormats = options.extraFormats ? options.extraFormats.slice() : [];
-                if (parseFormats.indexOf(format) < 0 && parseFormats.indexOf(actualFormat) < 0) {
-                    parseFormats.push(actualFormat);
-                }
-
-                use24Hours = (actualFormat.toLowerCase().indexOf('a') < 1 && actualFormat.replace(/\[.*?\]/g, '').indexOf('h') < 1);
-
-                if (isEnabled('y')) {
-                    minViewModeNumber = 2;
-                }
-                if (isEnabled('M')) {
-                    minViewModeNumber = 1;
-                }
-                if (isEnabled('d')) {
-                    minViewModeNumber = 0;
-                }
-
-                currentViewMode = Math.max(minViewModeNumber, currentViewMode);
-
-                if (!unset) {
-                    setValue(date);
-                }
-            };
-
-        /********************************************************************************
-         *
-         * Public API functions
-         * =====================
-         *
-         * Important: Do not expose direct references to private objects or the options
-         * object to the outer world. Always return a clone when returning values or make
-         * a clone when setting a private variable.
-         *
-         ********************************************************************************/
-        picker.destroy = function () {
-            ///<summary>Destroys the widget and removes all attached event listeners</summary>
-            hide();
-            detachDatePickerElementEvents();
-            element.removeData('DateTimePicker');
-            element.removeData('date');
-        };
-
-        picker.toggle = toggle;
-
-        picker.show = show;
-
-        picker.hide = hide;
-
-        picker.disable = function () {
-            ///<summary>Disables the input element, the component is attached to, by adding a disabled="true" attribute to it.
-            ///If the widget was visible before that call it is hidden. Possibly emits dp.hide</summary>
-            hide();
-            if (component && component.hasClass('btn')) {
-                component.addClass('disabled');
-            }
-            input.prop('disabled', true);
-            return picker;
-        };
-
-        picker.enable = function () {
-            ///<summary>Enables the input element, the component is attached to, by removing disabled attribute from it.</summary>
-            if (component && component.hasClass('btn')) {
-                component.removeClass('disabled');
-            }
-            input.prop('disabled', false);
-            return picker;
-        };
-
-        picker.ignoreReadonly = function (ignoreReadonly) {
-            if (arguments.length === 0) {
-                return options.ignoreReadonly;
-            }
-            if (typeof ignoreReadonly !== 'boolean') {
-                throw new TypeError('ignoreReadonly () expects a boolean parameter');
-            }
-            options.ignoreReadonly = ignoreReadonly;
-            return picker;
-        };
-
-        picker.options = function (newOptions) {
-            if (arguments.length === 0) {
-                return $.extend(true, {}, options);
-            }
-
-            if (!(newOptions instanceof Object)) {
-                throw new TypeError('options() options parameter should be an object');
-            }
-            $.extend(true, options, newOptions);
-            $.each(options, function (key, value) {
-                if (picker[key] !== undefined) {
-                    picker[key](value);
-                } else {
-                    throw new TypeError('option ' + key + ' is not recognized!');
-                }
-            });
-            return picker;
-        };
-
-        picker.date = function (newDate) {
-            ///<signature helpKeyword="$.fn.datetimepicker.date">
-            ///<summary>Returns the component's model current date, a moment object or null if not set.</summary>
-            ///<returns type="Moment">date.clone()</returns>
-            ///</signature>
-            ///<signature>
-            ///<summary>Sets the components model current moment to it. Passing a null value unsets the components model current moment. Parsing of the newDate parameter is made using moment library with the options.format and options.useStrict components configuration.</summary>
-            ///<param name="newDate" locid="$.fn.datetimepicker.date_p:newDate">Takes string, Date, moment, null parameter.</param>
-            ///</signature>
-            if (arguments.length === 0) {
-                if (unset) {
-                    return null;
-                }
-                return date.clone();
-            }
-
-            if (newDate !== null && typeof newDate !== 'string' && !moment.isMoment(newDate) && !(newDate instanceof Date)) {
-                throw new TypeError('date() parameter must be one of [null, string, moment or Date]');
-            }
-
-            setValue(newDate === null ? null : parseInputDate(newDate));
-            return picker;
-        };
-
-        picker.format = function (newFormat) {
-            ///<summary>test su</summary>
-            ///<param name="newFormat">info about para</param>
-            ///<returns type="string|boolean">returns foo</returns>
-            if (arguments.length === 0) {
-                return options.format;
-            }
-
-            if ((typeof newFormat !== 'string') && ((typeof newFormat !== 'boolean') || (newFormat !== false))) {
-                throw new TypeError('format() expects a sting or boolean:false parameter ' + newFormat);
-            }
-
-            options.format = newFormat;
-            if (actualFormat) {
-                initFormatting(); // reinit formatting
-            }
-            return picker;
-        };
-
-        picker.dayViewHeaderFormat = function (newFormat) {
-            if (arguments.length === 0) {
-                return options.dayViewHeaderFormat;
-            }
-
-            if (typeof newFormat !== 'string') {
-                throw new TypeError('dayViewHeaderFormat() expects a string parameter');
-            }
-
-            options.dayViewHeaderFormat = newFormat;
-            return picker;
-        };
-
-        picker.extraFormats = function (formats) {
-            if (arguments.length === 0) {
-                return options.extraFormats;
-            }
-
-            if (formats !== false && !(formats instanceof Array)) {
-                throw new TypeError('extraFormats() expects an array or false parameter');
-            }
-
-            options.extraFormats = formats;
-            if (parseFormats) {
-                initFormatting(); // reinit formatting
-            }
-            return picker;
-        };
-
-        picker.disabledDates = function (dates) {
-            ///<signature helpKeyword="$.fn.datetimepicker.disabledDates">
-            ///<summary>Returns an array with the currently set disabled dates on the component.</summary>
-            ///<returns type="array">options.disabledDates</returns>
-            ///</signature>
-            ///<signature>
-            ///<summary>Setting this takes precedence over options.minDate, options.maxDate configuration. Also calling this function removes the configuration of
-            ///options.enabledDates if such exist.</summary>
-            ///<param name="dates" locid="$.fn.datetimepicker.disabledDates_p:dates">Takes an [ string or Date or moment ] of values and allows the user to select only from those days.</param>
-            ///</signature>
-            if (arguments.length === 0) {
-                return (options.disabledDates ? $.extend({}, options.disabledDates) : options.disabledDates);
-            }
-
-            if (!dates) {
-                options.disabledDates = false;
-                update();
-                return picker;
-            }
-            if (!(dates instanceof Array)) {
-                throw new TypeError('disabledDates() expects an array parameter');
-            }
-            options.disabledDates = indexGivenDates(dates);
-            options.enabledDates = false;
-            update();
-            return picker;
-        };
-
-        picker.enabledDates = function (dates) {
-            ///<signature helpKeyword="$.fn.datetimepicker.enabledDates">
-            ///<summary>Returns an array with the currently set enabled dates on the component.</summary>
-            ///<returns type="array">options.enabledDates</returns>
-            ///</signature>
-            ///<signature>
-            ///<summary>Setting this takes precedence over options.minDate, options.maxDate configuration. Also calling this function removes the configuration of options.disabledDates if such exist.</summary>
-            ///<param name="dates" locid="$.fn.datetimepicker.enabledDates_p:dates">Takes an [ string or Date or moment ] of values and allows the user to select only from those days.</param>
-            ///</signature>
-            if (arguments.length === 0) {
-                return (options.enabledDates ? $.extend({}, options.enabledDates) : options.enabledDates);
-            }
-
-            if (!dates) {
-                options.enabledDates = false;
-                update();
-                return picker;
-            }
-            if (!(dates instanceof Array)) {
-                throw new TypeError('enabledDates() expects an array parameter');
-            }
-            options.enabledDates = indexGivenDates(dates);
-            options.disabledDates = false;
-            update();
-            return picker;
-        };
-
-        picker.daysOfWeekDisabled = function (daysOfWeekDisabled) {
-            if (arguments.length === 0) {
-                return options.daysOfWeekDisabled.splice(0);
-            }
-
-            if ((typeof daysOfWeekDisabled === 'boolean') && !daysOfWeekDisabled) {
-                options.daysOfWeekDisabled = false;
-                update();
-                return picker;
-            }
-
-            if (!(daysOfWeekDisabled instanceof Array)) {
-                throw new TypeError('daysOfWeekDisabled() expects an array parameter');
-            }
-            options.daysOfWeekDisabled = daysOfWeekDisabled.reduce(function (previousValue, currentValue) {
-                currentValue = parseInt(currentValue, 10);
-                if (currentValue > 6 || currentValue < 0 || isNaN(currentValue)) {
-                    return previousValue;
-                }
-                if (previousValue.indexOf(currentValue) === -1) {
-                    previousValue.push(currentValue);
-                }
-                return previousValue;
-            }, []).sort();
-            if (options.useCurrent && !options.keepInvalid) {
-                var tries = 0;
-                while (!isValid(date, 'd')) {
-                    date.add(1, 'd');
-                    if (tries === 7) {
-                        throw 'Tried 7 times to find a valid date';
-                    }
-                    tries++;
-                }
-                setValue(date);
-            }
-            update();
-            return picker;
-        };
-
-        picker.maxDate = function (maxDate) {
-            if (arguments.length === 0) {
-                return options.maxDate ? options.maxDate.clone() : options.maxDate;
-            }
-
-            if ((typeof maxDate === 'boolean') && maxDate === false) {
-                options.maxDate = false;
-                update();
-                return picker;
-            }
-
-            if (typeof maxDate === 'string') {
-                if (maxDate === 'now' || maxDate === 'moment') {
-                    maxDate = moment();
-                }
-            }
-
-            var parsedDate = parseInputDate(maxDate);
-
-            if (!parsedDate.isValid()) {
-                throw new TypeError('maxDate() Could not parse date parameter: ' + maxDate);
-            }
-            if (options.minDate && parsedDate.isBefore(options.minDate)) {
-                throw new TypeError('maxDate() date parameter is before options.minDate: ' + parsedDate.format(actualFormat));
-            }
-            options.maxDate = parsedDate;
-            if (options.useCurrent && !options.keepInvalid && date.isAfter(maxDate)) {
-                setValue(options.maxDate);
-            }
-            if (viewDate.isAfter(parsedDate)) {
-                viewDate = parsedDate.clone();
-            }
-            update();
-            return picker;
-        };
-
-        picker.minDate = function (minDate) {
-            if (arguments.length === 0) {
-                return options.minDate ? options.minDate.clone() : options.minDate;
-            }
-
-            if ((typeof minDate === 'boolean') && minDate === false) {
-                options.minDate = false;
-                update();
-                return picker;
-            }
-
-            if (typeof minDate === 'string') {
-                if (minDate === 'now' || minDate === 'moment') {
-                    minDate = moment();
-                }
-            }
-
-            var parsedDate = parseInputDate(minDate);
-
-            if (!parsedDate.isValid()) {
-                throw new TypeError('minDate() Could not parse date parameter: ' + minDate);
-            }
-            if (options.maxDate && parsedDate.isAfter(options.maxDate)) {
-                throw new TypeError('minDate() date parameter is after options.maxDate: ' + parsedDate.format(actualFormat));
-            }
-            options.minDate = parsedDate;
-            if (options.useCurrent && !options.keepInvalid && date.isBefore(minDate)) {
-                setValue(options.minDate);
-            }
-            if (viewDate.isBefore(parsedDate)) {
-                viewDate = parsedDate.clone();
-            }
-            update();
-            return picker;
-        };
-
-        picker.defaultDate = function (defaultDate) {
-            ///<signature helpKeyword="$.fn.datetimepicker.defaultDate">
-            ///<summary>Returns a moment with the options.defaultDate option configuration or false if not set</summary>
-            ///<returns type="Moment">date.clone()</returns>
-            ///</signature>
-            ///<signature>
-            ///<summary>Will set the picker's inital date. If a boolean:false value is passed the options.defaultDate parameter is cleared.</summary>
-            ///<param name="defaultDate" locid="$.fn.datetimepicker.defaultDate_p:defaultDate">Takes a string, Date, moment, boolean:false</param>
-            ///</signature>
-            if (arguments.length === 0) {
-                return options.defaultDate ? options.defaultDate.clone() : options.defaultDate;
-            }
-            if (!defaultDate) {
-                options.defaultDate = false;
-                return picker;
-            }
-
-            if (typeof defaultDate === 'string') {
-                if (defaultDate === 'now' || defaultDate === 'moment') {
-                    defaultDate = moment();
-                }
-            }
-
-            var parsedDate = parseInputDate(defaultDate);
-            if (!parsedDate.isValid()) {
-                throw new TypeError('defaultDate() Could not parse date parameter: ' + defaultDate);
-            }
-            if (!isValid(parsedDate)) {
-                throw new TypeError('defaultDate() date passed is invalid according to component setup validations');
-            }
-
-            options.defaultDate = parsedDate;
-
-            if (options.defaultDate && options.inline || (input.val().trim() === '' && input.attr('placeholder') === undefined)) {
-                setValue(options.defaultDate);
-            }
-            return picker;
-        };
-
-        picker.locale = function (locale) {
-            if (arguments.length === 0) {
-                return options.locale;
-            }
-
-            if (!moment.localeData(locale)) {
-                throw new TypeError('locale() locale ' + locale + ' is not loaded from moment locales!');
-            }
-
-            options.locale = locale;
-            date.locale(options.locale);
-            viewDate.locale(options.locale);
-
-            if (actualFormat) {
-                initFormatting(); // reinit formatting
-            }
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.stepping = function (stepping) {
-            if (arguments.length === 0) {
-                return options.stepping;
-            }
-
-            stepping = parseInt(stepping, 10);
-            if (isNaN(stepping) || stepping < 1) {
-                stepping = 1;
-            }
-            options.stepping = stepping;
-            return picker;
-        };
-
-        picker.useCurrent = function (useCurrent) {
-            var useCurrentOptions = ['year', 'month', 'day', 'hour', 'minute'];
-            if (arguments.length === 0) {
-                return options.useCurrent;
-            }
-
-            if ((typeof useCurrent !== 'boolean') && (typeof useCurrent !== 'string')) {
-                throw new TypeError('useCurrent() expects a boolean or string parameter');
-            }
-            if (typeof useCurrent === 'string' && useCurrentOptions.indexOf(useCurrent.toLowerCase()) === -1) {
-                throw new TypeError('useCurrent() expects a string parameter of ' + useCurrentOptions.join(', '));
-            }
-            options.useCurrent = useCurrent;
-            return picker;
-        };
-
-        picker.collapse = function (collapse) {
-            if (arguments.length === 0) {
-                return options.collapse;
-            }
-
-            if (typeof collapse !== 'boolean') {
-                throw new TypeError('collapse() expects a boolean parameter');
-            }
-            if (options.collapse === collapse) {
-                return picker;
-            }
-            options.collapse = collapse;
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.icons = function (icons) {
-            if (arguments.length === 0) {
-                return $.extend({}, options.icons);
-            }
-
-            if (!(icons instanceof Object)) {
-                throw new TypeError('icons() expects parameter to be an Object');
-            }
-            $.extend(options.icons, icons);
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.useStrict = function (useStrict) {
-            if (arguments.length === 0) {
-                return options.useStrict;
-            }
-
-            if (typeof useStrict !== 'boolean') {
-                throw new TypeError('useStrict() expects a boolean parameter');
-            }
-            options.useStrict = useStrict;
-            return picker;
-        };
-
-        picker.sideBySide = function (sideBySide) {
-            if (arguments.length === 0) {
-                return options.sideBySide;
-            }
-
-            if (typeof sideBySide !== 'boolean') {
-                throw new TypeError('sideBySide() expects a boolean parameter');
-            }
-            options.sideBySide = sideBySide;
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.viewMode = function (viewMode) {
-            if (arguments.length === 0) {
-                return options.viewMode;
-            }
-
-            if (typeof viewMode !== 'string') {
-                throw new TypeError('viewMode() expects a string parameter');
-            }
-
-            if (viewModes.indexOf(viewMode) === -1) {
-                throw new TypeError('viewMode() parameter must be one of (' + viewModes.join(', ') + ') value');
-            }
-
-            options.viewMode = viewMode;
-            currentViewMode = Math.max(viewModes.indexOf(viewMode), minViewModeNumber);
-
-            showMode();
-            return picker;
-        };
-
-        picker.toolbarPlacement = function (toolbarPlacement) {
-            if (arguments.length === 0) {
-                return options.toolbarPlacement;
-            }
-
-            if (typeof toolbarPlacement !== 'string') {
-                throw new TypeError('toolbarPlacement() expects a string parameter');
-            }
-            if (toolbarPlacements.indexOf(toolbarPlacement) === -1) {
-                throw new TypeError('toolbarPlacement() parameter must be one of (' + toolbarPlacements.join(', ') + ') value');
-            }
-            options.toolbarPlacement = toolbarPlacement;
-
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.widgetPositioning = function (widgetPositioning) {
-            if (arguments.length === 0) {
-                return $.extend({}, options.widgetPositioning);
-            }
-
-            if (({}).toString.call(widgetPositioning) !== '[object Object]') {
-                throw new TypeError('widgetPositioning() expects an object variable');
-            }
-            if (widgetPositioning.horizontal) {
-                if (typeof widgetPositioning.horizontal !== 'string') {
-                    throw new TypeError('widgetPositioning() horizontal variable must be a string');
-                }
-                widgetPositioning.horizontal = widgetPositioning.horizontal.toLowerCase();
-                if (horizontalModes.indexOf(widgetPositioning.horizontal) === -1) {
-                    throw new TypeError('widgetPositioning() expects horizontal parameter to be one of (' + horizontalModes.join(', ') + ')');
-                }
-                options.widgetPositioning.horizontal = widgetPositioning.horizontal;
-            }
-            if (widgetPositioning.vertical) {
-                if (typeof widgetPositioning.vertical !== 'string') {
-                    throw new TypeError('widgetPositioning() vertical variable must be a string');
-                }
-                widgetPositioning.vertical = widgetPositioning.vertical.toLowerCase();
-                if (verticalModes.indexOf(widgetPositioning.vertical) === -1) {
-                    throw new TypeError('widgetPositioning() expects vertical parameter to be one of (' + verticalModes.join(', ') + ')');
-                }
-                options.widgetPositioning.vertical = widgetPositioning.vertical;
-            }
-            update();
-            return picker;
-        };
-
-        picker.calendarWeeks = function (calendarWeeks) {
-            if (arguments.length === 0) {
-                return options.calendarWeeks;
-            }
-
-            if (typeof calendarWeeks !== 'boolean') {
-                throw new TypeError('calendarWeeks() expects parameter to be a boolean value');
-            }
-
-            options.calendarWeeks = calendarWeeks;
-            update();
-            return picker;
-        };
-
-        picker.showTodayButton = function (showTodayButton) {
-            if (arguments.length === 0) {
-                return options.showTodayButton;
-            }
-
-            if (typeof showTodayButton !== 'boolean') {
-                throw new TypeError('showTodayButton() expects a boolean parameter');
-            }
-
-            options.showTodayButton = showTodayButton;
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.showClear = function (showClear) {
-            if (arguments.length === 0) {
-                return options.showClear;
-            }
-
-            if (typeof showClear !== 'boolean') {
-                throw new TypeError('showClear() expects a boolean parameter');
-            }
-
-            options.showClear = showClear;
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.widgetParent = function (widgetParent) {
-            if (arguments.length === 0) {
-                return options.widgetParent;
-            }
-
-            if (typeof widgetParent === 'string') {
-                widgetParent = $(widgetParent);
-            }
-
-            if (widgetParent !== null && (typeof widgetParent !== 'string' && !(widgetParent instanceof $))) {
-                throw new TypeError('widgetParent() expects a string or a jQuery object parameter');
-            }
-
-            options.widgetParent = widgetParent;
-            if (widget) {
-                hide();
-                show();
-            }
-            return picker;
-        };
-
-        picker.keepOpen = function (keepOpen) {
-            if (arguments.length === 0) {
-                return options.keepOpen;
-            }
-
-            if (typeof keepOpen !== 'boolean') {
-                throw new TypeError('keepOpen() expects a boolean parameter');
-            }
-
-            options.keepOpen = keepOpen;
-            return picker;
-        };
-
-        picker.focusOnShow = function (focusOnShow) {
-            if (arguments.length === 0) {
-                return options.focusOnShow;
-            }
-
-            if (typeof focusOnShow !== 'boolean') {
-                throw new TypeError('focusOnShow() expects a boolean parameter');
-            }
-
-            options.focusOnShow = focusOnShow;
-            return picker;
-        };
-
-        picker.inline = function (inline) {
-            if (arguments.length === 0) {
-                return options.inline;
-            }
-
-            if (typeof inline !== 'boolean') {
-                throw new TypeError('inline() expects a boolean parameter');
-            }
-
-            options.inline = inline;
-            return picker;
-        };
-
-        picker.clear = function () {
-            clear();
-            return picker;
-        };
-
-        picker.keyBinds = function (keyBinds) {
-            options.keyBinds = keyBinds;
-            return picker;
-        };
-
-        picker.debug = function (debug) {
-            if (typeof debug !== 'boolean') {
-                throw new TypeError('debug() expects a boolean parameter');
-            }
-
-            options.debug = debug;
-            return picker;
-        };
-
-        picker.allowInputToggle = function (allowInputToggle) {
-            if (arguments.length === 0) {
-                return options.allowInputToggle;
-            }
-
-            if (typeof allowInputToggle !== 'boolean') {
-                throw new TypeError('allowInputToggle() expects a boolean parameter');
-            }
-
-            options.allowInputToggle = allowInputToggle;
-            return picker;
-        };
-
-        picker.showClose = function (showClose) {
-            if (arguments.length === 0) {
-                return options.showClose;
-            }
-
-            if (typeof showClose !== 'boolean') {
-                throw new TypeError('showClose() expects a boolean parameter');
-            }
-
-            options.showClose = showClose;
-            return picker;
-        };
-
-        picker.keepInvalid = function (keepInvalid) {
-            if (arguments.length === 0) {
-                return options.keepInvalid;
-            }
-
-            if (typeof keepInvalid !== 'boolean') {
-                throw new TypeError('keepInvalid() expects a boolean parameter');
-            }
-            options.keepInvalid = keepInvalid;
-            return picker;
-        };
-
-        picker.datepickerInput = function (datepickerInput) {
-            if (arguments.length === 0) {
-                return options.datepickerInput;
-            }
-
-            if (typeof datepickerInput !== 'string') {
-                throw new TypeError('datepickerInput() expects a string parameter');
-            }
-
-            options.datepickerInput = datepickerInput;
-            return picker;
-        };
-
-        picker.parseInputDate = function (parseInputDate) {
-            if (arguments.length === 0) {
-                return options.parseInputDate;
-            }
-
-            if (typeof parseInputDate !== 'function') {
-                throw new TypeError('parseInputDate() sholud be as function');
-            }
-
-            options.parseInputDate = parseInputDate;
-
-            return picker;
-        };
-
-        picker.disabledTimeIntervals = function (disabledTimeIntervals) {
-            ///<signature helpKeyword="$.fn.datetimepicker.disabledTimeIntervals">
-            ///<summary>Returns an array with the currently set disabled dates on the component.</summary>
-            ///<returns type="array">options.disabledTimeIntervals</returns>
-            ///</signature>
-            ///<signature>
-            ///<summary>Setting this takes precedence over options.minDate, options.maxDate configuration. Also calling this function removes the configuration of
-            ///options.enabledDates if such exist.</summary>
-            ///<param name="dates" locid="$.fn.datetimepicker.disabledTimeIntervals_p:dates">Takes an [ string or Date or moment ] of values and allows the user to select only from those days.</param>
-            ///</signature>
-            if (arguments.length === 0) {
-                return (options.disabledTimeIntervals ? $.extend({}, options.disabledTimeIntervals) : options.disabledTimeIntervals);
-            }
-
-            if (!disabledTimeIntervals) {
-                options.disabledTimeIntervals = false;
-                update();
-                return picker;
-            }
-            if (!(disabledTimeIntervals instanceof Array)) {
-                throw new TypeError('disabledTimeIntervals() expects an array parameter');
-            }
-            options.disabledTimeIntervals = disabledTimeIntervals;
-            update();
-            return picker;
-        };
-
-        picker.disabledHours = function (hours) {
-          

<TRUNCATED>


[5/5] incubator-atlas git commit: ATLAS-1894: Search using entity/tag attribute filters

Posted by ma...@apache.org.
ATLAS-1894: Search using entity/tag attribute filters

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/9a4ed469
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/9a4ed469
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/9a4ed469

Branch: refs/heads/master
Commit: 9a4ed469cc64cf107b98fba1851525017fec2024
Parents: 7c262b4
Author: kevalbhatt <kb...@apache.org>
Authored: Thu Jun 22 18:45:36 2017 +0530
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Fri Jun 30 01:49:15 2017 -0700

----------------------------------------------------------------------
 3party-licenses/bootstrap-datepicker-LICENSE    |   21 -
 .../bootstrap-daterangepicker-LICENSE           |    9 +
 3party-licenses/jQuery-QueryBuilder-LICENSE     |   21 +
 LICENSE                                         |    9 +-
 dashboardv2/gruntfile.js                        |  114 +-
 dashboardv2/package.json                        |    5 +
 dashboardv2/public/css/scss/form.scss           |    4 +
 dashboardv2/public/css/scss/override.scss       |   21 +
 dashboardv2/public/css/scss/search.scss         |   17 +
 dashboardv2/public/index.html                   |    3 +-
 dashboardv2/public/js/collection/VSearchList.js |   11 +
 dashboardv2/public/js/collection/VTagList.js    |    2 +-
 .../datetimepicker/bootstrap-datetimepicker.js  | 2444 ------------------
 .../bootstrap-datetimepicker.min.css            |    5 -
 dashboardv2/public/js/main.js                   |   28 +-
 dashboardv2/public/js/models/VSearch.js         |    2 +-
 dashboardv2/public/js/models/VTag.js            |    6 +-
 dashboardv2/public/js/router/Router.js          |   65 +-
 .../js/templates/search/QueryBuilder_tmpl.html  |   17 +
 .../templates/search/SearchLayoutView_tmpl.html |   22 +-
 .../js/templates/search/SearchQuery_tmpl.html   |   21 +
 .../search/SearchResultLayoutView_tmpl.html     |   15 +
 .../templates/tag/AddTagAttributeView_tmpl.html |    2 +-
 .../templates/tag/createTagLayoutView_tmpl.html |    2 +-
 dashboardv2/public/js/utils/Overrides.js        |    5 +-
 dashboardv2/public/js/utils/UrlLinks.js         |   38 +-
 dashboardv2/public/js/utils/Utils.js            |   68 +-
 .../views/business_catalog/SideNavLayoutView.js |   10 +-
 .../js/views/entity/CreateEntityLayoutView.js   |   29 +-
 .../public/js/views/search/QueryBuilderView.js  |  181 ++
 .../js/views/search/SearchDetailLayoutView.js   |    3 +-
 .../public/js/views/search/SearchLayoutView.js  |  181 +-
 .../public/js/views/search/SearchQueryView.js   |  113 +
 .../js/views/search/SearchResultLayoutView.js   |  113 +-
 .../public/js/views/tag/TagDetailLayoutView.js  |    3 +-
 .../public/js/views/tag/TagLayoutView.js        |    6 +-
 36 files changed, 975 insertions(+), 2641 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/3party-licenses/bootstrap-datepicker-LICENSE
----------------------------------------------------------------------
diff --git a/3party-licenses/bootstrap-datepicker-LICENSE b/3party-licenses/bootstrap-datepicker-LICENSE
deleted file mode 100644
index 3fd94fa..0000000
--- a/3party-licenses/bootstrap-datepicker-LICENSE
+++ /dev/null
@@ -1,21 +0,0 @@
-The MIT License (MIT)
-
-Copyright (c) 2015 Jonathan Peterson (@Eonasdan)
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all
-copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/3party-licenses/bootstrap-daterangepicker-LICENSE
----------------------------------------------------------------------
diff --git a/3party-licenses/bootstrap-daterangepicker-LICENSE b/3party-licenses/bootstrap-daterangepicker-LICENSE
new file mode 100644
index 0000000..aacee25
--- /dev/null
+++ b/3party-licenses/bootstrap-daterangepicker-LICENSE
@@ -0,0 +1,9 @@
+The MIT License (MIT)
+
+Copyright (c) 2012-2017 Dan Grossman
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/3party-licenses/jQuery-QueryBuilder-LICENSE
----------------------------------------------------------------------
diff --git a/3party-licenses/jQuery-QueryBuilder-LICENSE b/3party-licenses/jQuery-QueryBuilder-LICENSE
new file mode 100644
index 0000000..99070e3
--- /dev/null
+++ b/3party-licenses/jQuery-QueryBuilder-LICENSE
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c) 2014-2015 Damien Sorel
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 9722907..435e340 100755
--- a/LICENSE
+++ b/LICENSE
@@ -248,9 +248,12 @@ MIT License. For details, see 3party-licenses/es5-shim-LICENSE
 This product bundles Respond, which is available under
 MIT License. For details, see 3party-licenses/Respond-LICENSE
 
-This product bundles bootstrap-datepicker 4.14.30, which is available under
-MIT License.  For details, see 3party-licenses/bootstrap-datepicker-LICENSE
-
 This product bundles Google Fonts, which are available under
 Open Font License.  For details, see 3party-licenses/google-fonts-LICENSE
+
+This product bundles bootstrap-daterangepicker 2.1.25, which is available under
+MIT License.  For details, see 3party-licenses/bootstrap-daterangepicker-LICENSE
+
+This product bundles jQuery QueryBuilder 2.4.3, which is available under
+MIT License.  For details, see 3party-licenses/jQuery-QueryBuilder-LICENSE
 =======================================================================

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/gruntfile.js
----------------------------------------------------------------------
diff --git a/dashboardv2/gruntfile.js b/dashboardv2/gruntfile.js
index cf076b6..c0e315f 100644
--- a/dashboardv2/gruntfile.js
+++ b/dashboardv2/gruntfile.js
@@ -32,9 +32,6 @@ module.exports = function(grunt) {
         modulesPath = '../public/'
     }
 
-    var proxySnippet = require('grunt-connect-proxy/lib/utils').proxyRequest;
-
-
     grunt.initConfig({
         watch: {
             js: {
@@ -130,7 +127,9 @@ module.exports = function(grunt) {
                     'backgrid-select-all': 'backgrid-select-all/backgrid-select-all.min.js',
                     'moment/js': 'moment/min/moment.min.js',
                     'jquery-placeholder/js': 'jquery-placeholder/jquery.placeholder.js',
-                    'platform': 'platform/platform.js'
+                    'platform': 'platform/platform.js',
+                    'jQueryQueryBuilder/js': 'jQuery-QueryBuilder/dist/js/query-builder.standalone.min.js',
+                    'bootstrap-daterangepicker/js': 'bootstrap-daterangepicker/daterangepicker.js'
                 }
             },
             css: {
@@ -149,7 +148,9 @@ module.exports = function(grunt) {
                     'select2/css': 'select2/dist/css/select2.min.css',
                     'backgrid-select-all': 'backgrid-select-all/backgrid-select-all.min.css',
                     'font-awesome/css': 'font-awesome/css/font-awesome.min.css',
-                    'font-awesome/fonts': 'font-awesome/fonts'
+                    'font-awesome/fonts': 'font-awesome/fonts',
+                    'jQueryQueryBuilder/css': 'jQuery-QueryBuilder/dist/css/query-builder.default.min.css',
+                    'bootstrap-daterangepicker/css': 'bootstrap-daterangepicker/daterangepicker.css'
                 }
 
             },
@@ -177,13 +178,19 @@ module.exports = function(grunt) {
                     'dagre-d3': 'dagre-d3/LICENSE',
                     'backgrid-select-all': 'backgrid-select-all/LICENSE-MIT',
                     'jquery-placeholder': 'jquery-placeholder/LICENSE.txt',
-                    'platform/': 'platform/LICENSE'
+                    'platform/': 'platform/LICENSE',
+                    'jQueryQueryBuilder/': 'jQuery-QueryBuilder/LICENSE'
                 }
             }
         },
         sass: {
             dist: {
                 files: {
+                    'dist/css/style.css': 'public/css/scss/style.scss'
+                }
+            },
+            build: {
+                files: {
                     'dist/css/style.css': 'dist/css/scss/style.scss'
                 }
             }
@@ -192,7 +199,13 @@ module.exports = function(grunt) {
             dist: {
                 expand: true,
                 cwd: modulesPath,
-                src: ['**', '!**/*.sass'],
+                src: ['**', '!**/scss/**'],
+                dest: distPath
+            },
+            build: {
+                expand: true,
+                cwd: modulesPath,
+                src: ['**'],
                 dest: distPath
             }
         },
@@ -201,6 +214,41 @@ module.exports = function(grunt) {
             options: {
                 force: true
             }
+        },
+        uglify: {
+            build: {
+                files: [{
+                    expand: true,
+                    cwd: 'dist/js',
+                    src: '**/*.js',
+                    dest: 'dist/js'
+                }]
+            }
+        },
+        cssmin: {
+            build: {
+                files: [{
+                    expand: true,
+                    cwd: 'dist/css',
+                    src: '*.css',
+                    dest: 'dist/css'
+                }]
+            }
+
+        },
+        htmlmin: {
+            build: {
+                options: {
+                    removeComments: true,
+                    collapseWhitespace: true
+                },
+                files: [{
+                    expand: true,
+                    cwd: 'dist/js/templates',
+                    src: '**/*.html',
+                    dest: 'dist/js/templates'
+                }]
+            }
         }
     });
 
@@ -208,7 +256,9 @@ module.exports = function(grunt) {
     grunt.loadNpmTasks('grunt-contrib-connect');
     grunt.loadNpmTasks('grunt-contrib-watch');
     grunt.loadNpmTasks('grunt-npmcopy');
-
+    grunt.loadNpmTasks('grunt-contrib-uglify');
+    grunt.loadNpmTasks('grunt-contrib-cssmin');
+    grunt.loadNpmTasks('grunt-contrib-htmlmin');
 
     require('load-grunt-tasks')(grunt);
 
@@ -226,33 +276,45 @@ module.exports = function(grunt) {
         'npmcopy:css',
         'npmcopy:license',
         'copy:dist',
-        'sass',
+        'sass:dist',
         'configureProxies:server',
         'connect:server',
-        /* 'concurrent',*/
-        'watch',
-        /*'connect:livereload'*/
+        'watch'
     ]);
 
     grunt.registerTask('build', [
+        'clean',
+        'npmcopy:js',
+        'npmcopy:css',
+        'npmcopy:license',
+        'copy:build',
+        'sass:build'
+    ]);
+
+    grunt.registerTask('dev-minify', [
+        'clean',
         'npmcopy:js',
         'npmcopy:css',
         'npmcopy:license',
         'copy:dist',
-        'sass'
+        'sass:dist',
+        'uglify:build',
+        'cssmin:build',
+        'htmlmin:build',
+        'configureProxies:server',
+        'connect:server',
+        'watch'
     ]);
 
-    grunt.registerTask('minify', 'Minify the all js', function() {
-        var done = this.async();
-        grunt.task.run(['shell:min']);
-        done();
-    });
-    grunt.registerTask('release', 'Create release package', function() {
-        var done = this.async();
-        git.short(function(str) {
-            gitHash = str;
-            grunt.task.run(['minify', 'compress:release']);
-            done();
-        });
-    });
+    grunt.registerTask('build-minify', [
+        'clean',
+        'npmcopy:js',
+        'npmcopy:css',
+        'npmcopy:license',
+        'copy:build',
+        'sass:build',
+        'uglify:build',
+        'cssmin:build',
+        'htmlmin:build'
+    ]);
 };

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/package.json
----------------------------------------------------------------------
diff --git a/dashboardv2/package.json b/dashboardv2/package.json
index 8924f71..59889d3 100644
--- a/dashboardv2/package.json
+++ b/dashboardv2/package.json
@@ -28,10 +28,12 @@
     "backgrid-select-all": "^0.3.5",
     "backgrid-sizeable-columns": "^0.1.1",
     "bootstrap": "^3.3.5",
+    "bootstrap-daterangepicker": "^2.1.25",
     "d3": "^3.5.16",
     "d3-tip": "^0.6.7",
     "dagre-d3": "^0.4.17",
     "font-awesome": "^4.6.3",
+    "jQuery-QueryBuilder": "^2.4.3",
     "jquery": "^2.2.2",
     "jquery-asBreadcrumbs": "^0.2.1",
     "jquery-placeholder": "^2.3.1",
@@ -55,7 +57,10 @@
     "grunt-contrib-compress": "^0.13.0",
     "grunt-contrib-connect": "^1.0.0",
     "grunt-contrib-copy": "~0.8.0",
+    "grunt-contrib-cssmin": "^2.0.0",
+    "grunt-contrib-htmlmin": "^2.2.0",
     "grunt-contrib-jshint": "^0.11.0",
+    "grunt-contrib-uglify": "^2.1.0",
     "grunt-contrib-watch": "^0.6.1",
     "grunt-dev-update": "^1.0.2",
     "grunt-jsbeautifier": "^0.2.6",

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/css/scss/form.scss
----------------------------------------------------------------------
diff --git a/dashboardv2/public/css/scss/form.scss b/dashboardv2/public/css/scss/form.scss
index 3ee94f5..9989e7c 100644
--- a/dashboardv2/public/css/scss/form.scss
+++ b/dashboardv2/public/css/scss/form.scss
@@ -166,6 +166,10 @@ button:focus {
     margin-bottom: 10px;
     border-radius: 4px;
     padding: 5px 10px;
+    &.active {
+        background-color: $color_keppel_approx;
+        color: $white;
+    }
     &:hover {
         background-color: $color_keppel_approx;
         color: $white;

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/css/scss/override.scss
----------------------------------------------------------------------
diff --git a/dashboardv2/public/css/scss/override.scss b/dashboardv2/public/css/scss/override.scss
index 2852f80..6821d77 100644
--- a/dashboardv2/public/css/scss/override.scss
+++ b/dashboardv2/public/css/scss/override.scss
@@ -46,6 +46,7 @@
     position: relative;
     padding: 15px;
     max-height: 400px;
+    min-height: 70px;
     overflow: auto;
 }
 
@@ -265,3 +266,23 @@
     margin-left: 5px;
     cursor: pointer;
 }
+
+.query-builder .rule-container .rule-value-container {
+    display: inline-block !important;
+}
+
+.rules-group-header .btn-group.pull-right.group-actions {
+    float: left !important;
+}
+
+.query-builder .rules-list>:first-child::before {
+    top: -8px;
+}
+
+.query-builder .rules-group-header .btn-group.group-conditions label {
+    display: none;
+}
+
+.query-builder .rule-value-container input {
+    padding: 6px 12px !important;
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/css/scss/search.scss
----------------------------------------------------------------------
diff --git a/dashboardv2/public/css/scss/search.scss b/dashboardv2/public/css/scss/search.scss
index 8c3dbec..2bce210 100644
--- a/dashboardv2/public/css/scss/search.scss
+++ b/dashboardv2/public/css/scss/search.scss
@@ -114,3 +114,20 @@ $switchTransition: .4s ease-out;
         color: $gray
     }
 }
+
+.filterQuery {
+    word-break: break-all;
+    span {
+        font-size: 15px;
+        font-weight: bold;
+    }
+    .key {
+        color: $color_jungle_green_approx;
+    }
+    .operator {
+        color: $action_gray;
+    }
+    .value {
+        color: $tag_color;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/index.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/index.html b/dashboardv2/public/index.html
index 5fff304..b460686 100644
--- a/dashboardv2/public/index.html
+++ b/dashboardv2/public/index.html
@@ -45,12 +45,13 @@
     <link rel="stylesheet" href="js/libs/select2/css/select2.min.css">
     <link rel="stylesheet" href="js/libs/bootstrap/css/bootstrap.min.css">
     <link rel="stylesheet" href="js/libs/jquery-asBreadcrumbs/css/asBreadcrumbs.min.css">
-    <link rel="stylesheet" href="js/external_lib/datetimepicker/bootstrap-datetimepicker.min.css">
     <link rel="stylesheet" href="css/googlefonts.css" type='text/css'>
     <link rel="stylesheet" type="text/css" href="js/external_lib/jquery-ui/jquery-ui.min.css">
     <link href="css/bootstrap-sidebar.css" rel="stylesheet">
     <link href="js/libs/font-awesome/css/font-awesome.min.css" rel="stylesheet">
     <link href="js/external_lib/pnotify/pnotify.custom.min.css" rel="stylesheet">
+    <link href="js/libs/jQueryQueryBuilder/css/query-builder.default.min.css" rel="stylesheet">
+    <link href="js/libs/bootstrap-daterangepicker/css/daterangepicker.css" rel="stylesheet">
     <link href="css/style.css" rel="stylesheet">
 </head>
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/collection/VSearchList.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/collection/VSearchList.js b/dashboardv2/public/js/collection/VSearchList.js
index 6a84943..6291838 100644
--- a/dashboardv2/public/js/collection/VSearchList.js
+++ b/dashboardv2/public/js/collection/VSearchList.js
@@ -39,6 +39,17 @@ define(['require',
                 this.queryText = resp.queryText;
                 return resp.entities ? resp.entities : [];
             },
+            getBasicRearchResult: function(options) {
+                var url = UrlLinks.searchApiUrl('basic');
+
+                options = _.extend({
+                    contentType: 'application/json',
+                    dataType: 'json',
+                }, options);
+                options.data = JSON.stringify(options.data);
+
+                return this.constructor.nonCrudOperation.call(this, url, 'POST', options);
+            }
         },
         //Static Class Members
         {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/collection/VTagList.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/collection/VTagList.js b/dashboardv2/public/js/collection/VTagList.js
index 9304858..146fd10 100644
--- a/dashboardv2/public/js/collection/VTagList.js
+++ b/dashboardv2/public/js/collection/VTagList.js
@@ -26,7 +26,7 @@ define(['require',
     var VTagList = BaseCollection.extend(
         //Prototypal attributes
         {
-            url: UrlLinks.typesClassicationApiUrl(),
+            url: UrlLinks.classificationDefApiUrl(),
             model: VTag,
             initialize: function() {
                 this.modelName = 'VTag';


[3/5] incubator-atlas git commit: ATLAS-1894: Search using entity/tag attribute filters

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.min.css
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.min.css b/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.min.css
deleted file mode 100644
index e9ec816..0000000
--- a/dashboardv2/public/js/external_lib/datetimepicker/bootstrap-datetimepicker.min.css
+++ /dev/null
@@ -1,5 +0,0 @@
-/*!
- * Datetimepicker for Bootstrap 3
- * version : 4.14.30
- * https://github.com/Eonasdan/bootstrap-datetimepicker/
- */.bootstrap-datetimepicker-widget{list-style:none}.bootstrap-datetimepicker-widget.dropdown-menu{margin:2px 0;padding:4px;width:19em}@media (min-width:768px){.bootstrap-datetimepicker-widget.dropdown-menu.timepicker-sbs{width:38em}}@media (min-width:992px){.bootstrap-datetimepicker-widget.dropdown-menu.timepicker-sbs{width:38em}}@media (min-width:1200px){.bootstrap-datetimepicker-widget.dropdown-menu.timepicker-sbs{width:38em}}.bootstrap-datetimepicker-widget.dropdown-menu:after,.bootstrap-datetimepicker-widget.dropdown-menu:before{content:'';display:inline-block;position:absolute}.bootstrap-datetimepicker-widget.dropdown-menu.bottom:before{border-left:7px solid transparent;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-bottom-color:rgba(0,0,0,.2);top:-7px;left:7px}.bootstrap-datetimepicker-widget.dropdown-menu.bottom:after{border-left:6px solid transparent;border-right:6px solid transparent;border-bottom:6px solid #fff;top:-6px;left:8px}.bootstrap-datetime
 picker-widget.dropdown-menu.top:before{border-left:7px solid transparent;border-right:7px solid transparent;border-top:7px solid #ccc;border-top-color:rgba(0,0,0,.2);bottom:-7px;left:6px}.bootstrap-datetimepicker-widget.dropdown-menu.top:after{border-left:6px solid transparent;border-right:6px solid transparent;border-top:6px solid #fff;bottom:-6px;left:7px}.bootstrap-datetimepicker-widget.dropdown-menu.pull-right:before{left:auto;right:6px}.bootstrap-datetimepicker-widget.dropdown-menu.pull-right:after{left:auto;right:7px}.bootstrap-datetimepicker-widget .list-unstyled{margin:0}.bootstrap-datetimepicker-widget a[data-action]{padding:6px 0}.bootstrap-datetimepicker-widget a[data-action]:active{box-shadow:none}.bootstrap-datetimepicker-widget .timepicker-hour,.bootstrap-datetimepicker-widget .timepicker-minute,.bootstrap-datetimepicker-widget .timepicker-second{width:54px;font-weight:700;font-size:1.2em;margin:0}.bootstrap-datetimepicker-widget button[data-action]{padding:6px}.bootst
 rap-datetimepicker-widget .btn[data-action=incrementHours]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Increment Hours"}.bootstrap-datetimepicker-widget .btn[data-action=incrementMinutes]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Increment Minutes"}.bootstrap-datetimepicker-widget .btn[data-action=decrementHours]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Decrement Hours"}.bootstrap-datetimepicker-widget .btn[data-action=decrementMinutes]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Decrement Minutes"}.bootstrap-datetimepicker-widget .btn[data-action=showHours]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content
 :"Show Hours"}.bootstrap-datetimepicker-widget .btn[data-action=showMinutes]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Show Minutes"}.bootstrap-datetimepicker-widget .btn[data-action=togglePeriod]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Toggle AM/PM"}.bootstrap-datetimepicker-widget .btn[data-action=clear]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Clear the picker"}.bootstrap-datetimepicker-widget .btn[data-action=today]::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Set the date to today"}.bootstrap-datetimepicker-widget .picker-switch{text-align:center}.bootstrap-datetimepicker-widget .picker-switch::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflo
 w:hidden;clip:rect(0,0,0,0);border:0;content:"Toggle Date and Time Screens"}.bootstrap-datetimepicker-widget .picker-switch td{padding:0;margin:0;height:auto;width:auto;line-height:inherit}.bootstrap-datetimepicker-widget .picker-switch td span{line-height:2.5;height:2.5em;width:100%}.bootstrap-datetimepicker-widget table{width:100%;margin:0}.bootstrap-datetimepicker-widget table td,.bootstrap-datetimepicker-widget table th{text-align:center;border-radius:4px}.bootstrap-datetimepicker-widget table th{height:20px;line-height:20px;width:20px}.bootstrap-datetimepicker-widget table th.picker-switch{width:145px}.bootstrap-datetimepicker-widget table th.disabled,.bootstrap-datetimepicker-widget table th.disabled:hover{background:0 0;color:#777;cursor:not-allowed}.bootstrap-datetimepicker-widget table th.prev::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Previous Month"}.bootstrap-datetimepicker-widget table th.next
 ::after{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0;content:"Next Month"}.bootstrap-datetimepicker-widget table thead tr:first-child th{cursor:pointer}.bootstrap-datetimepicker-widget table thead tr:first-child th:hover{background:#eee}.bootstrap-datetimepicker-widget table td{height:54px;line-height:54px;width:54px}.bootstrap-datetimepicker-widget table td.cw{font-size:.8em;height:20px;line-height:20px;color:#777}.bootstrap-datetimepicker-widget table td.day{height:20px;line-height:20px;width:20px}.bootstrap-datetimepicker-widget table td.day:hover,.bootstrap-datetimepicker-widget table td.hour:hover,.bootstrap-datetimepicker-widget table td.minute:hover,.bootstrap-datetimepicker-widget table td.second:hover{background:#eee;cursor:pointer}.bootstrap-datetimepicker-widget table td.new,.bootstrap-datetimepicker-widget table td.old{color:#777}.bootstrap-datetimepicker-widget table td.today{position:relative}.bootstrap-dateti
 mepicker-widget table td.today:before{content:'';display:inline-block;border:0 solid transparent;border-bottom-color:#337ab7;border-top-color:rgba(0,0,0,.2);position:absolute;bottom:4px;right:4px}.bootstrap-datetimepicker-widget table td.active,.bootstrap-datetimepicker-widget table td.active:hover{background-color:#337ab7;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,.25)}.bootstrap-datetimepicker-widget table td.active.today:before{border-bottom-color:#fff}.bootstrap-datetimepicker-widget table td.disabled,.bootstrap-datetimepicker-widget table td.disabled:hover{background:0 0;color:#777;cursor:not-allowed}.bootstrap-datetimepicker-widget table td span{display:inline-block;width:54px;height:54px;line-height:54px;margin:2px 1.5px;cursor:pointer;border-radius:4px}.bootstrap-datetimepicker-widget table td span:hover{background:#eee}.bootstrap-datetimepicker-widget table td span.active{background-color:#337ab7;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,.25)}.bootstrap-datetimepicker-widg
 et table td span.old{color:#777}.bootstrap-datetimepicker-widget table td span.disabled,.bootstrap-datetimepicker-widget table td span.disabled:hover{background:0 0;color:#777;cursor:not-allowed}.bootstrap-datetimepicker-widget.usetwentyfour td.hour{height:27px;line-height:27px}.bootstrap-datetimepicker-widget.wider{width:21em}.bootstrap-datetimepicker-widget .datepicker-decades .decade{line-height:1.8em!important}.input-group.date .input-group-addon{cursor:pointer}.sr-only{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/main.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/main.js b/dashboardv2/public/js/main.js
index 3b40451..9235d28 100644
--- a/dashboardv2/public/js/main.js
+++ b/dashboardv2/public/js/main.js
@@ -94,10 +94,6 @@ require.config({
             'deps': ['d3'],
             'exports': ['d3-tip']
         },
-        'datetimepicker': {
-            'deps': ['jquery'],
-            'exports': 'datetimepicker'
-        },
         'dagreD3': {
             'deps': ['d3'],
             'exports': ['dagreD3']
@@ -107,6 +103,12 @@ require.config({
         },
         'jquery-placeholder': {
             'deps': ['jquery']
+        },
+        'query-builder': {
+            'deps': ['jquery']
+        },
+        'daterangepicker': {
+            'deps': ['jquery', 'moment']
         }
     },
 
@@ -137,12 +139,13 @@ require.config({
         'backgrid-select-all': 'libs/backgrid-select-all/backgrid-select-all.min',
         'moment': 'libs/moment/js/moment.min',
         'jquery-ui': 'external_lib/jquery-ui/jquery-ui.min',
-        'datetimepicker': 'external_lib/datetimepicker/bootstrap-datetimepicker',
         'pnotify': 'external_lib/pnotify/pnotify.custom.min',
         'pnotify.buttons': 'external_lib/pnotify/pnotify.custom.min',
         'pnotify.confirm': 'external_lib/pnotify/pnotify.custom.min',
         'jquery-placeholder': 'libs/jquery-placeholder/js/jquery.placeholder',
-        'platform': 'libs/platform/platform'
+        'platform': 'libs/platform/platform',
+        'query-builder': 'libs/jQueryQueryBuilder/js/query-builder.standalone.min',
+        'daterangepicker': 'libs/bootstrap-daterangepicker/js/daterangepicker'
     },
 
     /**
@@ -167,7 +170,7 @@ require(['App',
     'select2'
 ], function(App, Router, CommonViewFunction, Globals, UrlLinks, VEntityList, VTagList) {
     var that = this;
-    this.asyncFetchCounter = 4;
+    this.asyncFetchCounter = 5;
     this.entityDefCollection = new VEntityList();
     this.entityDefCollection.url = UrlLinks.entitiesDefApiUrl();
     this.typeHeaders = new VTagList();
@@ -175,11 +178,13 @@ require(['App',
     this.enumDefCollection = new VTagList();
     this.enumDefCollection.url = UrlLinks.enumDefApiUrl();
     this.enumDefCollection.modelAttrName = "enumDefs";
+    this.classificationDefCollection = new VTagList();
 
     App.appRouter = new Router({
         entityDefCollection: this.entityDefCollection,
         typeHeaders: this.typeHeaders,
-        enumDefCollection: this.enumDefCollection
+        enumDefCollection: this.enumDefCollection,
+        classificationDefCollection: this.classificationDefCollection
     });
 
     var startApp = function() {
@@ -238,4 +243,11 @@ require(['App',
             startApp();
         }
     });
+    this.classificationDefCollection.fetch({
+        skipDefaultError: true,
+        complete: function() {
+            --that.asyncFetchCounter;
+            startApp();
+        }
+    });
 });

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/models/VSearch.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/models/VSearch.js b/dashboardv2/public/js/models/VSearch.js
index 5c3df97..700a909 100644
--- a/dashboardv2/public/js/models/VSearch.js
+++ b/dashboardv2/public/js/models/VSearch.js
@@ -49,7 +49,7 @@ define(['require',
             }, options);
 
             return this.constructor.nonCrudOperation.call(this, url, 'GET', options);
-        },
+        }
     }, {});
     return VSearch;
 });

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/models/VTag.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/models/VTag.js b/dashboardv2/public/js/models/VTag.js
index c875d5a..65010cf 100644
--- a/dashboardv2/public/js/models/VTag.js
+++ b/dashboardv2/public/js/models/VTag.js
@@ -23,7 +23,7 @@ define(['require',
 ], function(require, Globals, vBaseModel, UrlLinks) {
     'use strict';
     var VTag = vBaseModel.extend({
-        urlRoot: UrlLinks.typesClassicationApiUrl(),
+        urlRoot: UrlLinks.classificationDefApiUrl(),
 
         defaults: {},
 
@@ -49,7 +49,7 @@ define(['require',
             return this.constructor.nonCrudOperation.call(this, url, 'DELETE', options);
         },
         deleteTag: function(options) {
-            var url = UrlLinks.typesClassicationApiUrl();
+            var url = UrlLinks.classificationDefApiUrl();
             options = _.extend({
                 contentType: 'application/json',
                 dataType: 'json'
@@ -57,7 +57,7 @@ define(['require',
             return this.constructor.nonCrudOperation.call(this, url, 'DELETE', options);
         },
         saveTagAttribute: function(options) {
-            var url = UrlLinks.typesClassicationApiUrl();
+            var url = UrlLinks.classificationDefApiUrl();
             options = _.extend({
                 contentType: 'application/json',
                 dataType: 'json'

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/router/Router.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/router/Router.js b/dashboardv2/public/js/router/Router.js
index f1250e7..8a1d241 100644
--- a/dashboardv2/public/js/router/Router.js
+++ b/dashboardv2/public/js/router/Router.js
@@ -42,16 +42,16 @@ define([
             '*actions': 'defaultAction'
         },
         initialize: function(options) {
-            _.extend(this, _.pick(options, 'entityDefCollection', 'typeHeaders', 'enumDefCollection'));
+            _.extend(this, _.pick(options, 'entityDefCollection', 'typeHeaders', 'enumDefCollection', 'classificationDefCollection'));
             this.showRegions();
             this.bindCommonEvents();
             this.listenTo(this, 'route', this.postRouteExecute, this);
-            this.tagCollection = new VTagList();
             this.searchVent = new Backbone.Wreqr.EventAggregator();
             this.preFetchedCollectionLists = {
                 'entityDefCollection': this.entityDefCollection,
                 'typeHeaders': this.typeHeaders,
-                'enumDefCollection': this.enumDefCollection
+                'enumDefCollection': this.enumDefCollection,
+                'classificationDefCollection': this.classificationDefCollection
             }
         },
         bindCommonEvents: function() {
@@ -125,7 +125,6 @@ define([
                         App.rSideNav.show(new SideNavLayoutView(
                             _.extend({
                                 'url': url,
-                                'collection': that.tagCollection
                             }, that.preFetchedCollectionLists)
                         ));
                     } else {
@@ -157,9 +156,7 @@ define([
                     App.rNHeader.show(new Header());
                     if (!App.rSideNav.currentView) {
                         App.rSideNav.show(new SideNavLayoutView(
-                            _.extend({
-                                'collection': that.tagCollection,
-                            }, that.preFetchedCollectionLists)
+                            _.extend({}, that.preFetchedCollectionLists)
                         ));
                     } else {
                         App.rSideNav.currentView.selectTab();
@@ -196,8 +193,7 @@ define([
                     }
                     App.rSideNav.show(new SideNavLayoutView(
                         _.extend({
-                            'tag': tagName,
-                            'collection': that.tagCollection
+                            'tag': tagName
                         }, that.preFetchedCollectionLists)
                     ));
                 } else {
@@ -221,8 +217,7 @@ define([
                     }
                     App.rNContent.show(new TagDetailLayoutView(
                         _.extend({
-                            'tag': tagName,
-                            'collection': that.tagCollection,
+                            'tag': tagName
                         }, that.preFetchedCollectionLists)
                     ));
                 }
@@ -236,12 +231,17 @@ define([
                 'views/business_catalog/SideNavLayoutView',
                 'views/search/SearchDetailLayoutView',
             ], function(Header, BusinessCatalogLayoutView, SideNavLayoutView, SearchDetailLayoutView) {
-                var paramObj = Utils.getUrlState.getQueryParams();
+                var paramObj = Utils.getUrlState.getQueryParams(),
+                    filterObj = {
+                        'tagFilters': JSON.parse(Utils.localStorage.getValue('tagFilters')),
+                        'entityFilters': JSON.parse(Utils.localStorage.getValue('entityFilters'))
+                    }
                 App.rNHeader.show(new Header());
                 if (!App.rSideNav.currentView) {
                     App.rSideNav.show(new SideNavLayoutView(
                         _.extend({
-                            'collection': that.tagCollection
+                            'searchVent': that.searchVent,
+                            'filterObj': filterObj
                         }, that.preFetchedCollectionLists)
                     ));
                 } else {
@@ -256,7 +256,9 @@ define([
                     App.rNContent.show(new SearchDetailLayoutView(
                         _.extend({
                             'value': paramObj,
-                            'initialView': true
+                            'initialView': true,
+                            'filterObj': filterObj,
+                            'searchVent': that.searchVent
                         }, that.preFetchedCollectionLists)
                     ));
                 } else {
@@ -273,14 +275,42 @@ define([
                 'views/business_catalog/SideNavLayoutView',
                 'views/search/SearchDetailLayoutView'
             ], function(Header, BusinessCatalogLayoutView, SideNavLayoutView, SearchDetailLayoutView) {
-                var paramObj = Utils.getUrlState.getQueryParams();
+                var paramObj = Utils.getUrlState.getQueryParams(),
+                    filterObj = {
+                        'tagFilters': JSON.parse(Utils.localStorage.getValue('tagFilters')),
+                        'entityFilters': JSON.parse(Utils.localStorage.getValue('entityFilters'))
+                    }
+                if (paramObj && paramObj.searchType === "basic") {
+                    if (paramObj.type) {
+                        if (_.has(filterObj.entityFilters, paramObj.type)) {
+                            _.extend(paramObj, {
+                                'entityFilters': +new Date()
+                            })
+                        }
+                    }
+                    if (paramObj.tag) {
+                        if (_.has(filterObj.entityFilters, paramObj.type)) {
+                            _.extend(paramObj, {
+                                'tagFilters': +new Date()
+                            })
+                        }
+                    }
+                    Utils.setUrl({
+                        url: '#!/search/searchResult',
+                        trigger: false,
+                        urlParams: paramObj,
+                        updateTabState: function() {
+                            return { searchUrl: this.url, stateChanged: true };
+                        },
+                    });
+                }
                 App.rNHeader.show(new Header());
                 if (!App.rSideNav.currentView) {
                     App.rSideNav.show(new SideNavLayoutView(
                         _.extend({
                             'value': paramObj,
-                            'collection': that.tagCollection,
-                            'searchVent': that.searchVent
+                            'searchVent': that.searchVent,
+                            'filterObj': filterObj
                         }, that.preFetchedCollectionLists)
                     ));
                 } else {
@@ -291,6 +321,7 @@ define([
                     _.extend({
                         'value': paramObj,
                         'searchVent': that.searchVent,
+                        'filterObj': filterObj,
                         'initialView': (paramObj.type || (paramObj.dslChecked == "true" ? "" : paramObj.tag) || (paramObj.query ? paramObj.query.trim() : "")).length === 0
                     }, that.preFetchedCollectionLists)
                 ));

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/search/QueryBuilder_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/search/QueryBuilder_tmpl.html b/dashboardv2/public/js/templates/search/QueryBuilder_tmpl.html
new file mode 100644
index 0000000..f204390
--- /dev/null
+++ b/dashboardv2/public/js/templates/search/QueryBuilder_tmpl.html
@@ -0,0 +1,17 @@
+<!--
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+-->
+<div id="builder"></div>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/search/SearchLayoutView_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/search/SearchLayoutView_tmpl.html b/dashboardv2/public/js/templates/search/SearchLayoutView_tmpl.html
index a766cdf..a23057f 100644
--- a/dashboardv2/public/js/templates/search/SearchLayoutView_tmpl.html
+++ b/dashboardv2/public/js/templates/search/SearchLayoutView_tmpl.html
@@ -33,16 +33,26 @@
     </div>
     <div class="col-sm-12">
         <div class="form-group">
-            <div class="srchType">
+            <div class="srchType clearfix">
                 <span class="srchTitle">Search By Type</span>
-                <div class="typeLOV">
-                    <select data-id="typeLOV"></select>
+                <div class="">
+                    <div class="col-sm-10 no-padding temFilter">
+                        <select data-id="typeLOV"></select>
+                    </div>
+                    <div class="col-sm-2 no-padding temFilterBtn">
+                        <button type="button" class="btn btn-atlasAction btn-atlas pull-right typeLOV" title="Entity Attribute Filter" data-id="typeAttrFilter"><i class="fa fa-filter"></i></button>
+                    </div>
                 </div>
             </div>
-            <div class="srchType tagBox">
+            <div class="srchType tagBox clearfix">
                 <span class="srchTitle">Search By Tag</span>
-                <div class="typeLOV">
-                    <select data-id="tagLOV"></select>
+                <div>
+                    <div class="col-sm-10 no-padding">
+                        <select data-id="tagLOV"></select>
+                    </div>
+                    <div class="col-sm-2 no-padding">
+                        <button type="button" class="btn btn-atlasAction btn-atlas pull-right" title="Tag Attribute Filter" data-id="tagAttrFilter"><i class="fa fa-filter"></i></button>
+                    </div>
                 </div>
             </div>
             <div class="srchType">

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/search/SearchQuery_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/search/SearchQuery_tmpl.html b/dashboardv2/public/js/templates/search/SearchQuery_tmpl.html
new file mode 100644
index 0000000..05c2a61
--- /dev/null
+++ b/dashboardv2/public/js/templates/search/SearchQuery_tmpl.html
@@ -0,0 +1,21 @@
+<!--
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+-->
+<div id="r_queryBuilder">
+    <div class="fontLoader">
+        <i class="fa fa-refresh fa-spin-custom"></i>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/search/SearchResultLayoutView_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/search/SearchResultLayoutView_tmpl.html b/dashboardv2/public/js/templates/search/SearchResultLayoutView_tmpl.html
index 91f1a67..fc2fe8c 100644
--- a/dashboardv2/public/js/templates/search/SearchResultLayoutView_tmpl.html
+++ b/dashboardv2/public/js/templates/search/SearchResultLayoutView_tmpl.html
@@ -20,6 +20,21 @@
     <div class="fontLoader">
         <i class="fa fa-refresh fa-spin-custom"></i>
     </div>
+    <div>
+        <div id="r_searchQuery">
+            <div class="panel panel-default" id="filterPanel">
+                <div class="panel-heading clearfix">
+                    <h4 class="panel-title pull-left">Type,Tag filter/s</h4>
+                    <div class="btn-group pull-right">
+                        <button type="button" id="expand_collapse_panel" class="expand_collapse_panel" title="Collapse"><i class="fa fa-chevron-up" aria-hidden="true"></i></button>
+                    </div>
+                </div>
+                <div id="panel_body" class="panel-body collapse in" align="center">
+                    <p class="filterQuery" id="filterQuery"></p>
+                </div>
+            </div>
+        </div>
+    </div>
     <div class="ellipsis" style="display: none;"><span class="searchResult" style=" font-size: 16px;"></span>
         <a href="javascript:void(0)" class="inputAssignTag multiSelectTerm btnAssign" style="display:none" data-id="addTerm"><i class="fa fa-folder-o"></i> Assign Term</a>
         <a href="javascript:void(0)" class="inputAssignTag multiSelectTag assignTag btnAssign" style="display:none" data-id="addAssignTag"><i class="fa fa-plus"></i> Assign Tag</a>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/tag/AddTagAttributeView_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/tag/AddTagAttributeView_tmpl.html b/dashboardv2/public/js/templates/tag/AddTagAttributeView_tmpl.html
index a7f9146..26d283b 100644
--- a/dashboardv2/public/js/templates/tag/AddTagAttributeView_tmpl.html
+++ b/dashboardv2/public/js/templates/tag/AddTagAttributeView_tmpl.html
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
 -->
-<div class="fontLoader" style="margin-top:-10px">
+<div class="fontLoader">
     <i class="fa fa-refresh fa-spin-custom"></i>
 </div>
 <div class="row row-margin-bottom hide">

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/templates/tag/createTagLayoutView_tmpl.html
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/templates/tag/createTagLayoutView_tmpl.html b/dashboardv2/public/js/templates/tag/createTagLayoutView_tmpl.html
index 0491c78..8b0b63d 100644
--- a/dashboardv2/public/js/templates/tag/createTagLayoutView_tmpl.html
+++ b/dashboardv2/public/js/templates/tag/createTagLayoutView_tmpl.html
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
 -->
-<div class="fontLoader" style="margin-top:-10px">
+<div class="fontLoader">
     <i class="fa fa-refresh fa-spin-custom"></i>
 </div>
 <form name="tagDefinitionform" class="css-form hide" data-id="createTagForm">

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/utils/Overrides.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/utils/Overrides.js b/dashboardv2/public/js/utils/Overrides.js
index b6fc17c..615e3b1 100644
--- a/dashboardv2/public/js/utils/Overrides.js
+++ b/dashboardv2/public/js/utils/Overrides.js
@@ -49,7 +49,6 @@ define(['require', 'utils/Utils', 'marionette', 'backgrid', 'asBreadcrumbs', 'jq
             }
         }
     });
-
     // For placeholder support 
     if (!('placeholder' in HTMLInputElement.prototype)) {
         var originalRender = Backbone.Marionette.LayoutView.prototype.render;
@@ -141,4 +140,8 @@ define(['require', 'utils/Utils', 'marionette', 'backgrid', 'asBreadcrumbs', 'jq
             return this;
         }
     });
+
+    String.prototype.capitalize = function() {
+        return this.charAt(0).toUpperCase() + this.slice(1);
+    }
 });

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/utils/UrlLinks.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/utils/UrlLinks.js b/dashboardv2/public/js/utils/UrlLinks.js
index c692d8d..df20cf1 100644
--- a/dashboardv2/public/js/utils/UrlLinks.js
+++ b/dashboardv2/public/js/utils/UrlLinks.js
@@ -28,6 +28,23 @@ define(['require', 'utils/Enums'], function(require, Enums) {
                 def: this.baseUrlV2 + '/types/typedef'
             };
         },
+        entitiesDefApiUrl: function(name) {
+            return this.getDefApiUrl('entity', name);
+        },
+        classificationDefApiUrl: function(name) {
+            return this.getDefApiUrl('classification', name);
+        },
+        enumDefApiUrl: function(name) {
+            return this.getDefApiUrl('enum', name);
+        },
+        getDefApiUrl: function(type, name) {
+            var defApiUrl = this.typedefsUrl();
+            if (name) {
+                return defApiUrl.def + '/name/' + name + '?type=' + type;
+            } else {
+                return defApiUrl.defs + '?type=' + type;
+            }
+        },
         taxonomiesApiUrl: function() {
             return this.baseUrl + '/v1/taxonomies';
         },
@@ -44,22 +61,6 @@ define(['require', 'utils/Enums'], function(require, Enums) {
                 return entitiesUrl;
             }
         },
-        entitiesDefApiUrl: function(name) {
-            var entitieDefUrl = this.typedefsUrl();
-            if (name) {
-                return entitieDefUrl.def + '/name/' + name + '?type=entity';
-            } else {
-                return entitieDefUrl.defs + '?type=entity';
-            }
-        },
-        enumDefApiUrl: function(name) {
-            var enumDefApiUrl = this.typedefsUrl();
-            if (name) {
-                return enumDefApiUrl.def + '/name/' + name + '?type=enum';
-            } else {
-                return enumDefApiUrl.defs + '?type=enum';
-            }
-        },
         entitiesTraitsApiUrl: function(token) {
             if (token) {
                 return this.baseUrlV2 + '/entity/guid/' + token + '/classifications';
@@ -67,19 +68,16 @@ define(['require', 'utils/Enums'], function(require, Enums) {
                 // For Multiple Assignment
                 return this.baseUrlV2 + '/entity/bulk/classification';
             }
-
         },
         entityCollectionaudit: function(guid) {
             return this.baseUrl + '/entities/' + guid + '/audit';
         },
-        typesClassicationApiUrl: function(name, guid) {
+        classicationApiUrl: function(name, guid) {
             var typeUrl = this.typedefsUrl();
             if (name) {
                 return typeUrl.def + '/name/' + name + '?type=classification';
             } else if (guid) {
                 return typeUrl.def + '/guid/' + guid + '?type=classification';
-            } else {
-                return typeUrl.defs + '?type=classification';
             }
         },
         typesApiUrl: function() {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/utils/Utils.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/utils/Utils.js b/dashboardv2/public/js/utils/Utils.js
index 1d614a8..9071df3 100644
--- a/dashboardv2/public/js/utils/Utils.js
+++ b/dashboardv2/public/js/utils/Utils.js
@@ -166,37 +166,11 @@ define(['require', 'utils/Globals', 'pnotify', 'utils/Messages', 'pnotify.button
             });
         }
     };
-    Utils.localStorage = {
-        checkLocalStorage: function(key, value) {
-            if (typeof(Storage) !== "undefined") {
-                return this.getLocalStorage(key, value);
-            } else {
-                console.log('Sorry! No Web Storage support');
-                Utils.cookie.checkCookie(key, value);
-            }
-        },
-        setLocalStorage: function(key, value) {
-            localStorage.setItem(key, value);
-            return { found: false, 'value': value };
-        },
-        getLocalStorage: function(key, value) {
-            var keyValue = localStorage.getItem(key);
-            if (!keyValue || keyValue == "undefined") {
-                return this.setLocalStorage(key, value);
-            } else {
-                return { found: true, 'value': keyValue };
-            }
-        }
-    };
     Utils.cookie = {
-        setCookie: function(cname, cvalue) {
-            //var d = new Date();
-            //d.setTime(d.getTime() + (exdays*24*60*60*1000));
-            //var expires = "expires=" + d.toGMTString();
+        setValue: function(cname, cvalue) {
             document.cookie = cname + "=" + cvalue + "; ";
-            return { found: false, 'value': cvalue };
         },
-        getCookie: function(findString) {
+        getValue: function(findString) {
             var search = findString + "=";
             var ca = document.cookie.split(';');
             for (var i = 0; i < ca.length; i++) {
@@ -207,16 +181,34 @@ define(['require', 'utils/Globals', 'pnotify', 'utils/Messages', 'pnotify.button
                 }
             }
             return "";
-        },
-        checkCookie: function(key, value) {
-            var findString = getCookie(key);
-            if (findString != "" || keyValue != "undefined") {
-                return { found: true, 'value': ((findString == "undefined") ? (undefined) : (findString)) };
+        }
+    };
+    Utils.localStorage = function() {
+        this.setValue = function() {
+            localStorage.setItem(arguments[0], arguments[1]);
+        }
+        this.getValue = function(key, value) {
+            var keyValue = localStorage.getItem(key);
+            if ((!keyValue || keyValue == "undefined") && (value != undefined)) {
+                return this.setLocalStorage(key, value);
             } else {
-                return setCookie(key, value);
+                if (keyValue === "" || keyValue === "undefined" || keyValue === "null") {
+                    return null;
+                } else {
+                    return keyValue;
+                }
+
             }
         }
-    };
+        this.removeValue = function() {
+            localStorage.removeItem(arguments[0]);
+        }
+        if (typeof(Storage) === "undefined") {
+            _.extend(this, Utils.cookie);
+            console.log('Sorry! No Web Storage support');
+        }
+    }
+    Utils.localStorage = new Utils.localStorage();
 
     Utils.setUrl = function(options) {
         if (options) {
@@ -509,16 +501,16 @@ define(['require', 'utils/Globals', 'pnotify', 'utils/Messages', 'pnotify.button
     }
     $('body').on('click', '.expand_collapse_panel', function() {
         var icon = $(this).find('i'),
-            panel = $(this).parents('.panel'),
+            panel = $(this).parents('.panel').first(),
             panelBody = panel.find('.panel-body');
         icon.toggleClass('fa-chevron-up fa-chevron-down');
         $(this).toggleAttribute('title', 'Collapse', 'Expand');
-        panelBody.toggle('0.5', 'linear');
+        panelBody.toggle();
         $(this).trigger('expand_collapse_panel', [$(this).parents('.panel')]);
     });
     $('body').on('click', '.fullscreen_panel', function() {
         var icon = $(this).find('i'),
-            panel = $(this).parents('.panel'),
+            panel = $(this).parents('.panel').first(),
             panelBody = panel.find('.panel-body');
         icon.toggleClass('fa-expand fa-compress');
         $(this).toggleAttribute('title', 'Fullscreen', 'Exit Fullscreen');

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/business_catalog/SideNavLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/business_catalog/SideNavLayoutView.js b/dashboardv2/public/js/views/business_catalog/SideNavLayoutView.js
index af18540..013c72e 100644
--- a/dashboardv2/public/js/views/business_catalog/SideNavLayoutView.js
+++ b/dashboardv2/public/js/views/business_catalog/SideNavLayoutView.js
@@ -66,7 +66,7 @@ define(['require',
             return events;
         },
         initialize: function(options) {
-            _.extend(this, _.pick(options, 'url', 'value', 'tag', 'selectFirst', 'collection', 'typeHeaders', 'searchVent'));
+            _.extend(this, _.pick(options, 'url', 'value', 'tag', 'selectFirst', 'classificationDefCollection', 'typeHeaders', 'searchVent', 'entityDefCollection', 'enumDefCollection', 'filterObj'));
             if (Globals.taxonomy) {
                 this.tabClass = "tab col-sm-4";
             } else {
@@ -94,7 +94,7 @@ define(['require',
             var that = this;
             require(['views/tag/TagLayoutView'], function(TagLayoutView) {
                 that.RTagLayoutView.show(new TagLayoutView({
-                    collection: that.collection,
+                    collection: that.classificationDefCollection,
                     tag: that.tag,
                     typeHeaders: that.typeHeaders
                 }));
@@ -105,8 +105,12 @@ define(['require',
             require(['views/search/SearchLayoutView'], function(SearchLayoutView) {
                 that.RSearchLayoutView.show(new SearchLayoutView({
                     value: that.value,
+                    filterObj: that.filterObj,
                     searchVent: that.searchVent,
-                    typeHeaders: that.typeHeaders
+                    typeHeaders: that.typeHeaders,
+                    entityDefCollection: that.entityDefCollection,
+                    enumDefCollection: that.enumDefCollection,
+                    classificationDefCollection: that.classificationDefCollection
                 }));
             });
         },

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/entity/CreateEntityLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/entity/CreateEntityLayoutView.js b/dashboardv2/public/js/views/entity/CreateEntityLayoutView.js
index 704062f..d461d09 100644
--- a/dashboardv2/public/js/views/entity/CreateEntityLayoutView.js
+++ b/dashboardv2/public/js/views/entity/CreateEntityLayoutView.js
@@ -25,13 +25,13 @@ define(['require',
     'models/VEntity',
     'modules/Modal',
     'utils/Messages',
-    'datetimepicker',
     'moment',
     'utils/UrlLinks',
     'collection/VSearchList',
     'utils/Enums',
-    'utils/Globals'
-], function(require, Backbone, CreateEntityLayoutViewTmpl, Utils, VTagList, VEntityList, VEntity, Modal, Messages, datepicker, moment, UrlLinks, VSearchList, Enums, Globals) {
+    'utils/Globals',
+    'daterangepicker'
+], function(require, Backbone, CreateEntityLayoutViewTmpl, Utils, VTagList, VEntityList, VEntity, Modal, Messages, moment, UrlLinks, VSearchList, Enums, Globals) {
 
     var CreateEntityLayoutView = Backbone.Marionette.LayoutView.extend(
         /** @lends CreateEntityLayoutView */
@@ -145,7 +145,7 @@ define(['require',
                     }
                 });
 
-                this.ui.entityInputData.on('keyup change dp.change', 'input.true,select.true', function(e) {
+                this.ui.entityInputData.on('keyup change', 'input.true,select.true', function(e) {
                     if (this.value !== "") {
                         if ($(this).data('select2')) {
                             $(this).data('select2').$container.find('.select2-selection').removeClass("errorClass");
@@ -171,7 +171,7 @@ define(['require',
             },
             bindNonRequiredField: function() {
                 var that = this;
-                this.ui.entityInputData.off('keyup change dp.change', 'input.false,select.false').on('keyup change dp.change', 'input.false,select.false', function(e) {
+                this.ui.entityInputData.off('keyup change', 'input.false,select.false').on('keyup change', 'input.false,select.false', function(e) {
                     if (that.modal.$el.find('button.ok').prop('disabled') && that.ui.entityInputData.find('.errorClass').length === 0) {
                         that.modal.$el.find('button.ok').prop("disabled", false);
                     }
@@ -331,11 +331,12 @@ define(['require',
             initilizeElements: function() {
                 var that = this;
                 this.$('input[data-type="date"]').each(function() {
-                    if (!$(this).data('datepicker')) {
-                        $(this).datetimepicker({
-                            format: 'DD MMMM YYYY',
-                            keepInvalid: true
-                        });
+                    if (!$(this).data('daterangepicker')) {
+                        var dateObj = { "singleDatePicker": true, "showDropdowns": true };
+                        if (that.guid) {
+                            dateObj["startDate"] = this.value
+                        }
+                        $(this).daterangepicker(dateObj);
                     }
                 });
                 this.initializeValidation();
@@ -387,9 +388,9 @@ define(['require',
                     removeText(e, e.currentTarget.value);
                 });
 
-                this.$('input[data-type="date"]').on('dp.hide keydown', function(event) {
+                this.$('input[data-type="date"]').on('hide.daterangepicker keydown', function(event) {
                     if (event.type) {
-                        if (event.type == 'dp') {
+                        if (event.type == 'hide') {
                             this.blur();
                         } else if (event.type == 'keydown') {
                             return false;
@@ -462,7 +463,7 @@ define(['require',
                             entityValue = dataValue;
                         }
                         if (value.typeName === "date" && dataValue) {
-                            entityValue = moment(dataValue).format("DD MMMM YYYY");
+                            entityValue = moment(dataValue).format("MM/DD/YYYY");
                         }
                     }
                 }
@@ -538,7 +539,7 @@ define(['require',
                         if (dataTypeEnitity && datakeyEntity) {
                             if (that.entityDefCollection.fullCollection.find({ name: dataTypeEnitity })) {
                                 entity[datakeyEntity] = extractValue(value, typeName);
-                            } else if (typeof dataTypeEnitity === 'string' && datakeyEntity.indexOf("Time") > -1) {
+                            } else if (dataTypeEnitity === 'date' || dataTypeEnitity === 'time') {
                                 entity[datakeyEntity] = Date.parse(value);
                             } else if (dataTypeEnitity.indexOf("map") > -1 || (typeNameCategory && typeNameCategory.get('category') === 'STRUCT')) {
                                 try {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/search/QueryBuilderView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/search/QueryBuilderView.js b/dashboardv2/public/js/views/search/QueryBuilderView.js
new file mode 100644
index 0000000..468a2d7
--- /dev/null
+++ b/dashboardv2/public/js/views/search/QueryBuilderView.js
@@ -0,0 +1,181 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+define(['require',
+    'backbone',
+    'hbs!tmpl/search/QueryBuilder_tmpl',
+    'utils/Utils',
+    'query-builder',
+    'daterangepicker'
+], function(require, Backbone, QueryBuilder_Tmpl, Utils) {
+
+    var QueryBuilderView = Backbone.Marionette.LayoutView.extend(
+        /** @lends QueryBuilderView */
+        {
+            _viewName: 'QueryBuilderView',
+
+            template: QueryBuilder_Tmpl,
+
+
+
+            /** Layout sub regions */
+            regions: {},
+
+
+            /** ui selector cache */
+            ui: {
+                "builder": "#builder"
+            },
+            /** ui events hash */
+            events: function() {
+                var events = {};
+                return events;
+            },
+            /**
+             * intialize a new QueryBuilderView Layout
+             * @constructs
+             */
+            initialize: function(options) {
+                _.extend(this, _.pick(options, 'attrObj', 'value', 'typeHeaders', 'filterObj', 'entityDefCollection', 'enumDefCollection', 'tag'));
+                this.attrObj = _.sortBy(this.attrObj, 'name');
+            },
+            bindEvents: function() {},
+            getOperator: function(type) {
+                var obj = {
+                    operators: null
+                }
+                if (type === "string") {
+                    obj.operators = ['=', '!=', 'contains', 'begins_with', 'ends_with'];
+                }
+                if (type === "enum" || type === "date" || type === "int" || type === "byte" || type === "short" || type === "long" || type === "float" || type === "double") {
+                    obj.operators = ['=', '!=', '>', '<', '>=', '<='];
+                }
+                return obj;
+            },
+            isPrimitive: function(type) {
+                if (type === "int" || type === "byte" || type === "short" || type === "long" || type === "float" || type === "double" || type === "string" || type === "boolean" || type === "date") {
+                    return true;
+                }
+                return false;
+            },
+            getObjDef: function(attrObj, rules) {
+                var obj = {
+                    id: attrObj.name,
+                    label: attrObj.name.capitalize(),
+                    type: attrObj.typeName
+                };
+                if (obj.type === "date") {
+                    obj['plugin'] = 'daterangepicker';
+                    obj['plugin_config'] = {
+                        "singleDatePicker": true,
+                        "showDropdowns": true,
+                        "timePicker": true,
+                        locale: {
+                            format: 'MM/DD/YYYY h:mm A'
+                        }
+                    };
+                    if (rules && rules.rules) {
+                        var valueObj = _.find(rules.rules, { id: obj.id });
+                        if (valueObj) {
+                            obj.plugin_config["startDate"] = valueObj.value;
+                        }
+                    }
+                    _.extend(obj, this.getOperator(obj.type));
+                    return obj;
+                }
+                if (this.isPrimitive(obj.type)) {
+                    if (obj.type === "long" || obj.type === "float") {
+                        obj.type = "double";
+                    }
+                    if (obj.type === "int" || obj.type === "byte" || obj.type === "short") {
+                        obj.type = "integer";
+                    }
+                    if (obj.type === "boolean") {
+                        obj['input'] = 'select';
+                        obj['values'] = ['true', 'false'];
+                    }
+                    _.extend(obj, this.getOperator(obj.type));
+                    return obj;
+                }
+                var enumObj = this.enumDefCollection.fullCollection.find({ name: obj.type });
+                if (enumObj) {
+                    obj.type = "string";
+                    obj['input'] = 'select';
+                    var value = [];
+                    _.each(enumObj.get('elementDefs'), function(o) {
+                        value.push(o.value)
+                    })
+                    obj['values'] = value;
+                    _.extend(obj, this.getOperator('enum'));
+                    return obj;
+                }
+            },
+            onRender: function() {
+                var that = this,
+                    filters = [];
+                if (this.filterObj) {
+                    var filter = this.filterObj[(this.tag ? 'tagFilters' : 'entityFilters')],
+                        tagTermName = this.tag ? this.value.tag : this.value.type;
+                    if (filter) {
+                        ruleObj = filter[tagTermName];
+                        if (ruleObj) {
+                            var rules_widgets = ruleObj.rule;
+                        }
+                    }
+                }
+
+                _.each(this.attrObj, function(obj) {
+                    var returnObj = that.getObjDef(obj, rules_widgets);
+                    if (returnObj) {
+                        filters.push(returnObj);
+                    }
+                });
+                if (filters && !_.isEmpty(filters)) {
+                    this.ui.builder.queryBuilder({
+                        plugins: ['bt-tooltip-errors'],
+                        filters: filters,
+                        select_placeholder: '--Select Attribute--',
+                        allow_empty: true,
+                        conditions: ['AND'],
+                        allow_groups: false,
+                        allow_empty: true,
+                        operators: [
+                            { type: '=', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: '!=', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: '>', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: '<', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: '>=', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: '<=', nb_inputs: 1, multiple: false, apply_to: ['number', 'string', 'boolean'] },
+                            { type: 'contains', nb_inputs: 1, multiple: false, apply_to: ['string'] },
+                            { type: 'begins_with', nb_inputs: 1, multiple: false, apply_to: ['string'] },
+                            { type: 'ends_with', nb_inputs: 1, multiple: false, apply_to: ['string'] }
+                        ],
+                        lang: {
+                            add_rule: 'Add filter',
+                            add_group: 'Add filter group'
+                        },
+                        rules: rules_widgets
+                    });
+                    this.$('.rules-group-header .btn-group.pull-right.group-actions').toggleClass('pull-right pull-left');
+                } else {
+                    this.ui.builder.html('<h4>No Attributes are available !</h4>')
+                }
+            }
+        });
+    return QueryBuilderView;
+});

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/search/SearchDetailLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/search/SearchDetailLayoutView.js b/dashboardv2/public/js/views/search/SearchDetailLayoutView.js
index b22fae8..32ad542 100644
--- a/dashboardv2/public/js/views/search/SearchDetailLayoutView.js
+++ b/dashboardv2/public/js/views/search/SearchDetailLayoutView.js
@@ -43,7 +43,7 @@ define(['require',
              * @constructs
              */
             initialize: function(options) {
-                _.extend(this, _.pick(options, 'value', 'initialView', 'entityDefCollection', 'typeHeaders', 'searchVent', 'enumDefCollection'));
+                _.extend(this, _.pick(options, 'value', 'initialView', 'entityDefCollection', 'typeHeaders', 'searchVent', 'enumDefCollection', 'filterObj'));
             },
             bindEvents: function() {},
             onRender: function() {
@@ -62,6 +62,7 @@ define(['require',
                             entityDefCollection: that.entityDefCollection,
                             typeHeaders: that.typeHeaders,
                             searchVent: that.searchVent,
+                            filterObj: that.filterObj,
                             enumDefCollection: that.enumDefCollection
                         }));
                     }

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/search/SearchLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/search/SearchLayoutView.js b/dashboardv2/public/js/views/search/SearchLayoutView.js
index 32b82d7..7ac394c 100644
--- a/dashboardv2/public/js/views/search/SearchLayoutView.js
+++ b/dashboardv2/public/js/views/search/SearchLayoutView.js
@@ -44,7 +44,9 @@ define(['require',
                 typeLov: '[data-id="typeLOV"]',
                 tagLov: '[data-id="tagLOV"]',
                 refreshBtn: '[data-id="refreshBtn"]',
-                advancedInfoBtn: '[data-id="advancedInfo"]'
+                advancedInfoBtn: '[data-id="advancedInfo"]',
+                typeAttrFilter: '[data-id="typeAttrFilter"]',
+                tagAttrFilter: '[data-id="tagAttrFilter"]'
             },
 
             /** ui events hash */
@@ -65,6 +67,12 @@ define(['require',
                 events["change " + this.ui.tagLov] = 'checkForButtonVisiblity';
                 events["click " + this.ui.refreshBtn] = 'onRefreshButton';
                 events["click " + this.ui.advancedInfoBtn] = 'advancedInfo';
+                events["click " + this.ui.typeAttrFilter] = function() {
+                    this.openAttrFilter('type');
+                };
+                events["click " + this.ui.tagAttrFilter] = function() {
+                    this.openAttrFilter('tag');
+                };
                 return events;
             },
             /**
@@ -72,7 +80,7 @@ define(['require',
              * @constructs
              */
             initialize: function(options) {
-                _.extend(this, _.pick(options, 'value', 'typeHeaders', 'searchVent'));
+                _.extend(this, _.pick(options, 'value', 'typeHeaders', 'searchVent', 'entityDefCollection', 'enumDefCollection', 'classificationDefCollection', 'filterObj'));
                 this.type = "basic";
                 var param = Utils.getUrlState.getQueryParams();
                 this.query = {
@@ -83,9 +91,14 @@ define(['require',
                     basic: {
                         query: null,
                         type: null,
+                        typeFilter: null,
+                        tagFilter: null,
                         tag: null
                     }
                 };
+                if (!this.value) {
+                    this.value = {};
+                }
                 this.dsl = false;
                 if (param && param.searchType) {
                     this.type = param.searchType;
@@ -108,6 +121,51 @@ define(['require',
                     this.checkForButtonVisiblity();
                 }, this);
             },
+            bindSelect2Events: function(argument) {
+                var that = this;
+                this.ui.typeLov.on('select2:select', function(argument) {
+                    // this function calles after checkForButtonVisiblity that is why disabled flter here
+                    that.ui.typeAttrFilter.prop('disabled', false);
+                    _.extend(that.value, { 'type': this.value });
+                    that.makeFilterButtonActive('type');
+                });
+                this.ui.tagLov.on('select2:select', function(argument) {
+                    // this function calles after checkForButtonVisiblity that is why disabled flter here
+                    that.ui.tagAttrFilter.prop('disabled', false);
+                    _.extend(that.value, { 'tag': this.value });
+                    that.makeFilterButtonActive('tag');
+                });
+                this.ui.typeLov.on('select2:unselect', function(argument) {
+                    _.extend(that.value, { 'type': null });
+                });
+                this.ui.tagLov.on('select2:unselect', function(argument) {
+                    _.extend(that.value, { 'tag': null });
+                });
+            },
+            makeFilterButtonActive: function(type) {
+                if (this.filterObj) {
+                    var tagFilters = this.filterObj.tagFilters,
+                        entityFilters = this.filterObj.entityFilters;
+                    if (type == "type") {
+                        if (_.has(entityFilters, this.value[type])) {
+                            this.query[this.type]['entityFilters'] = +new Date();
+                            this.ui.typeAttrFilter.addClass('active');
+                        } else {
+                            this.query[this.type]['entityFilters'] = null;
+                            this.ui.typeAttrFilter.removeClass('active');
+                        }
+                    }
+                    if (type == "tag") {
+                        if (_.has(tagFilters, this.value[type])) {
+                            this.query[this.type]['tagFilters'] = +new Date();
+                            this.ui.tagAttrFilter.addClass('active');
+                        } else {
+                            this.query[this.type]['tagFilters'] = null;
+                            this.ui.tagAttrFilter.removeClass('active');
+                        }
+                    }
+                }
+            },
             checkForButtonVisiblity: function() {
                 var that = this,
                     value = this.ui.searchInput.val() || this.ui.typeLov.val();
@@ -122,6 +180,24 @@ define(['require',
                 } else {
                     this.ui.searchBtn.attr("disabled", "true");
                 }
+                if (this.value) {
+                    if (this.value.tag) {
+                        this.ui.tagAttrFilter.prop('disabled', false);
+                    } else {
+                        this.ui.tagAttrFilter.prop('disabled', true);
+                    }
+                    if (this.value.type) {
+                        this.ui.typeAttrFilter.prop('disabled', false);
+                    } else {
+                        this.ui.typeAttrFilter.prop('disabled', true);
+                    }
+                    this.makeFilterButtonActive('type');
+                    this.makeFilterButtonActive('tag');
+                } else {
+                    this.ui.tagAttrFilter.prop('disabled', true);
+                    this.ui.typeAttrFilter.prop('disabled', true);
+                }
+
             },
             onRender: function() {
                 // array of tags which is coming from url
@@ -135,19 +211,24 @@ define(['require',
                     placeholder: "Select",
                     allowClear: true
                 });
-                this.bindEvents();
+                this.bindSelect2Events();
                 this.checkForButtonVisiblity();
             },
             updateQueryObject: function(param) {
                 if (param && param.searchType) {
                     this.type = param.searchType;
                 }
-                _.extend(this.query[this.type], {
+                _.extend(this.query[this.type],
+                    (this.type == "dsl" ? {
+                        query: null,
+                        type: null
+                    } : {
                         query: null,
                         type: null,
-                        tag: null
-                    },
-                    param);
+                        tag: null,
+                        entityFilters: null,
+                        tagFilters: null
+                    }), param);
             },
             fetchCollection: function(value) {
                 this.typeHeaders.fetch({ reset: true });
@@ -176,6 +257,86 @@ define(['require',
                     });
                 });
             },
+            openAttrFilter: function(filterType) {
+                var that = this;
+                require(['views/search/SearchQueryView'], function(SearchQueryView) {
+                    that.attrModal = new SearchQueryView({
+                        value: that.value,
+                        tag: (filterType === "tag" ? true : false),
+                        type: (filterType === "type" ? true : false),
+                        searchVent: that.searchVent,
+                        typeHeaders: that.typeHeaders,
+                        entityDefCollection: that.entityDefCollection,
+                        enumDefCollection: that.enumDefCollection,
+                        filterObj: that.filterObj,
+                        classificationDefCollection: that.classificationDefCollection
+                    });
+                    that.attrModal.on('ok', function(e) {
+                        that.okAttrFilterButton();
+                    });
+                });
+            },
+            okAttrFilterButton: function() {
+                var filtertype = this.attrModal.tag ? 'tagFilters' : 'entityFilters',
+                    rule = this.attrModal.RQueryBuilder.currentView.ui.builder.queryBuilder('getRules'),
+                    result = this.getQueryBuilderParsData(rule);
+                if (result && !_.isEmpty(result.criterion)) {
+                    this.query[this.type][filtertype] = +new Date();
+                    if (result) {
+                        var filterObj = this.filterObj ? this.filterObj[filtertype] : null;
+                        if (!filterObj) {
+                            filterObj = {};
+                        }
+                        var temp = {}; // IE fix
+                        temp[(this.attrModal.tag ? this.value.tag : this.value.type)] = { 'result': result, 'rule': rule };
+                        _.extend(filterObj, temp);
+                        this.filterObj[filtertype] = filterObj;
+                        this.makeFilterButtonActive(this.attrModal.tag ? 'tag' : 'type');
+                        Utils.localStorage.setValue((filtertype), JSON.stringify(filterObj));
+                    } else {
+                        this.filterObj[filtertype] = null;
+                        this.query[this.type][filtertype] = null;
+                        this.makeFilterButtonActive(this.attrModal.tag ? 'tag' : 'type');
+                        Utils.localStorage.removeValue(filtertype);
+                    }
+                } else {
+                    this.filterObj[filtertype] = null;
+                    this.query[this.type][filtertype] = null;
+                    this.makeFilterButtonActive(this.attrModal.tag ? 'tag' : 'type');
+                    Utils.localStorage.removeValue(filtertype);
+                }
+                this.attrModal.modal.close();
+            },
+            getQueryBuilderParsData: function(obj) {
+                if (obj) {
+                    var parsObj = {
+                        "condition": obj.condition,
+                        "criterion": convertKeyAndExtractObj(obj.rules)
+                    }
+                }
+
+                function convertKeyAndExtractObj(rules) {
+                    var convertObj = [];
+                    _.each(rules, function(rulObj) {
+                        var tempObj = {}
+                        if (rulObj.rules) {
+                            tempObj = {
+                                "condition": rulObj.condition,
+                                "criterion": convertKeyAndExtractObj(rulObj.rules)
+                            }
+                        } else {
+                            tempObj = {
+                                "attributeName": rulObj.id,
+                                "operator": rulObj.operator,
+                                "attributeValue": (rulObj.type === "date" ? Date.parse(rulObj.value) : rulObj.value)
+                            }
+                        }
+                        convertObj.push(tempObj);
+                    });
+                    return convertObj;
+                }
+                return parsObj;
+            },
             manualRender: function(paramObj) {
                 this.updateQueryObject(paramObj);
                 this.setValues(paramObj);
@@ -232,6 +393,7 @@ define(['require',
                     setTimeout(function() {
                         that.ui.searchInput.focus();
                     }, 0);
+                    //this.searchVent.trigger('searchAttribute', this.value);
                 }
             },
             findSearchResult: function() {
@@ -243,7 +405,6 @@ define(['require',
                 if (!this.dsl) {
                     this.query[this.type].tag = this.ui.tagLov.select2('val') || null;
                 }
-
                 Utils.setUrl({
                     url: '#!/search/searchResult',
                     urlParams: _.extend(this.query[this.type], {
@@ -263,7 +424,11 @@ define(['require',
                     this.type = "dsl";
                     this.dsl = true;
                     this.$('.tagBox').hide();
+                    this.$('.temFilterBtn').hide();
+                    this.$('.temFilter').toggleClass('col-sm-10 col-sm-12');
                 } else {
+                    this.$('.temFilter').toggleClass('col-sm-10 col-sm-12');
+                    this.$('.temFilterBtn').show();
                     this.$('.tagBox').show();
                     this.dsl = false;
                     this.type = "basic";

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/search/SearchQueryView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/search/SearchQueryView.js b/dashboardv2/public/js/views/search/SearchQueryView.js
new file mode 100644
index 0000000..37ccf90
--- /dev/null
+++ b/dashboardv2/public/js/views/search/SearchQueryView.js
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+define(['require',
+    'backbone',
+    'modules/Modal',
+    'utils/Utils',
+    'hbs!tmpl/search/SearchQuery_tmpl',
+], function(require, Backbone, Modal, Utils, SearchQuery_Tmpl) {
+
+    var SearchQueryView = Backbone.Marionette.LayoutView.extend(
+        /** @lends SearchQueryView */
+        {
+            _viewName: 'SearchQueryView',
+
+            template: SearchQuery_Tmpl,
+
+
+
+            /** Layout sub regions */
+            regions: {
+                RQueryBuilder: '#r_queryBuilder',
+            },
+
+
+            /** ui selector cache */
+            ui: {},
+            /** ui events hash */
+            events: function() {
+                var events = {};
+                return events;
+            },
+            /**
+             * intialize a new SearchQueryView Layout
+             * @constructs
+             */
+            initialize: function(options) {
+                _.extend(this, _.pick(options, 'value', 'entityDefCollection', 'typeHeaders', 'searchVent', 'enumDefCollection', 'classificationDefCollection', 'tag', 'filterObj'));
+                this.bindEvents();
+                var that = this;
+                this.modal = new Modal({
+                    title: 'Attribute Filter',
+                    content: this,
+                    okText: 'Apply',
+                    cancelText: "Cancel",
+                    allowCancel: true,
+                    okCloses: false,
+                    width: '50%'
+                }).open();
+                this.modal.on('closeModal', function() {
+                    that.modal.trigger('cancel');
+                });
+            },
+            onRender: function() {
+                this.$('.fontLoader').show();
+                var obj = {
+                    value: this.value,
+                    searchVent: this.searchVent,
+                    entityDefCollection: this.entityDefCollection,
+                    enumDefCollection: this.enumDefCollection,
+                    filterObj: this.filterObj,
+                    classificationDefCollection: this.classificationDefCollection
+                }
+
+                if (this.tag) {
+                    obj['tag'] = true;
+                    obj['attrObj'] = this.classificationDefCollection.fullCollection.find({ name: this.value.tag });
+                    if (obj.attrObj) {
+                        obj.attrObj = Utils.getNestedSuperTypeObj({
+                            data: obj.attrObj.toJSON(),
+                            collection: this.classificationDefCollection,
+                            attrMerge: true,
+                        });
+                    }
+                } else {
+                    obj['type'] = true;
+                    obj['attrObj'] = this.entityDefCollection.fullCollection.find({ name: this.value.type });
+                    if (obj.attrObj) {
+                        obj.attrObj = Utils.getNestedSuperTypeObj({
+                            data: obj.attrObj.toJSON(),
+                            collection: this.entityDefCollection,
+                            attrMerge: true
+                        });
+                    }
+                }
+                this.renderQueryBuilder(obj);
+                // this.showHideFilter(this.value);
+            },
+            bindEvents: function() {},
+            renderQueryBuilder: function(obj) {
+                var that = this;
+                require(['views/search/QueryBuilderView'], function(QueryBuilderView) {
+                    that.RQueryBuilder.show(new QueryBuilderView(obj));
+                });
+            }
+        });
+    return SearchQueryView;
+});

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/search/SearchResultLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/search/SearchResultLayoutView.js b/dashboardv2/public/js/views/search/SearchResultLayoutView.js
index 1198809..5236751 100644
--- a/dashboardv2/public/js/views/search/SearchResultLayoutView.js
+++ b/dashboardv2/public/js/views/search/SearchResultLayoutView.js
@@ -44,6 +44,7 @@ define(['require',
                 RTagLayoutView: "#r_tagLayoutView",
                 RSearchLayoutView: "#r_searchLayoutView",
                 REntityTableLayoutView: "#r_searchResultTableLayoutView",
+                RSearchQuery: '#r_searchQuery'
             },
 
             /** ui selector cache */
@@ -61,7 +62,9 @@ define(['require',
                 editEntityButton: "[data-id='editEntityButton']",
                 createEntity: "[data-id='createEntity']",
                 checkDeletedEntity: "[data-id='checkDeletedEntity']",
-                containerCheckBox: "[data-id='containerCheckBox']"
+                containerCheckBox: "[data-id='containerCheckBox']",
+                filterPanel: "#filterPanel",
+                filterQuery: "#filterQuery"
             },
             templateHelpers: function() {
                 return {
@@ -126,7 +129,7 @@ define(['require',
              * @constructs
              */
             initialize: function(options) {
-                _.extend(this, _.pick(options, 'value', 'initialView', 'entityDefCollection', 'typeHeaders', 'searchVent', 'enumDefCollection', 'tagCollection'));
+                _.extend(this, _.pick(options, 'value', 'initialView', 'entityDefCollection', 'typeHeaders', 'searchVent', 'enumDefCollection', 'tagCollection', 'filterObj'));
                 var pagination = "";
                 this.entityModel = new VEntity();
                 this.searchCollection = new VSearchList();
@@ -198,7 +201,7 @@ define(['require',
                     } else {
                         if (response.statusText !== "abort") {
                             Utils.notifyError({
-                                content: "Invalid Expression : " + model.queryParams.query
+                                content: "Invalid Expression"
                             });
                         }
                     }
@@ -206,6 +209,9 @@ define(['require',
                 this.listenTo(this.searchVent, "search:refresh", function(model, response) {
                     this.fetchCollection();
                 }, this);
+                this.listenTo(this.searchVent, "searchAttribute", function(obj) {
+                    this.showHideFilter(obj);
+                }, this);
             },
             onRender: function() {
                 if (!this.initialView) {
@@ -234,27 +240,74 @@ define(['require',
                         this.$(".entityLink").show();
                     }
                 }
+                this.showHideFilter();
+            },
+            generateQueryOfFilter: function() {
+                var value = this.value,
+                    entityFilters = this.filterObj && this.filterObj.entityFilters ? this.filterObj.entityFilters[value.type] : null,
+                    tagFilters = this.filterObj && this.filterObj.tagFilters ? this.filterObj.tagFilters[value.tag] : null,
+                    queryArray = [],
+                    objToString = function(filterObj) {
+                        var tempObj = [];
+                        _.each(filterObj.rules, function(obj) {
+                            tempObj.push('<span class="key">' + obj.field + '</span>&nbsp<span class="operator">' + obj.operator + '</span>&nbsp<span class="value">' + obj.value + "</span>")
+                        });
+                        return tempObj.join('&nbsp<span class="operator">AND</span>&nbsp');
+                    }
+                if (entityFilters) {
+                    var typeKeyValue = '<span class="key">Type:</span>&nbsp<span class="value">' + value.type + '</span>&nbsp<span class="operator">AND</span>&nbsp';
+                    queryArray = queryArray.concat(typeKeyValue + objToString(entityFilters.rule));
+                }
+                if (tagFilters) {
+                    var tagKeyValue = '<span class="key">Tag:</span>&nbsp<span class="value">' + value.tag + '</span>&nbsp<span class="operator">AND</span>&nbsp';
+                    queryArray = queryArray.concat(tagKeyValue + objToString(tagFilters.rule));
+
+                }
+                if (queryArray.length == 2) {
+                    return "<span>(</span>&nbsp" + queryArray.join('<span>&nbsp)</span>&nbsp<span>AND</span>&nbsp<span>(</span>&nbsp') + "&nbsp<span>)</span>";
+                } else {
+                    return queryArray.join();
+                }
+            },
+            showHideFilter: function() {
+                if (this.value) {
+                    if (Utils.getUrlState.isSearchTab() && this.value.searchType == "basic") {
+                        var query = this.generateQueryOfFilter();
+                        if (query) {
+                            this.ui.filterQuery.html(query);
+                            this.ui.filterPanel.show();
+                        } else
+                            this.ui.filterPanel.hide();
+                    } else {
+                        this.ui.filterPanel.hide();
+                    }
+                } else {
+                    this.ui.filterPanel.hide();
+                }
             },
             fetchCollection: function(value, clickObj) {
-                var that = this;
+                var that = this,
+                    isPostMethod = this.value.searchType === "basic" && Utils.getUrlState.isSearchTab(),
+                    tagFilters = this.filterObj && this.filterObj.tagFilters ? this.filterObj.tagFilters[this.value.tag] : null,
+                    entityFilters = this.filterObj && this.filterObj.entityFilters ? this.filterObj.entityFilters[this.value.type] : null,
+                    filterObj = {
+                        'entityFilters': entityFilters ? entityFilters.result : null,
+                        'tagFilters': tagFilters ? tagFilters.result : null
+                    }
                 this.showLoader();
                 if (Globals.searchApiCallRef && Globals.searchApiCallRef.readyState === 1) {
                     Globals.searchApiCallRef.abort();
                 }
-                if (value) {
-                    $.extend(this.searchCollection.queryParams, { limit: this.limit, excludeDeletedEntities: true });
-                    if (value.searchType) {
-                        this.searchCollection.url = UrlLinks.searchApiUrl(value.searchType);
-                    }
-                    _.extend(this.searchCollection.queryParams, { 'query': (value.query ? value.query.trim() : null), 'typeName': value.type || null, 'classification': value.tag || null });
-                }
-                Globals.searchApiCallRef = this.searchCollection.fetch({
+                var apiObj = {
                     skipDefaultError: true,
-                    success: function() {
+                    success: function(model, response) {
                         Globals.searchApiCallRef = undefined;
                         if (!(that.ui.pageRecordText instanceof jQuery)) {
                             return;
                         }
+                        if (isPostMethod) {
+                            that.searchCollection.reset(model.entities);
+                        }
                         if (that.searchCollection.models.length === 0 && that.offset > that.limit) {
                             that.ui.nextData.attr('disabled', true);
                             that.offset = that.offset - that.limit;
@@ -302,6 +355,38 @@ define(['require',
                     },
                     silent: true,
                     reset: true
+                }
+                if (value) {
+                    $.extend(this.searchCollection.queryParams, { limit: this.limit, excludeDeletedEntities: true });
+                    if (value.searchType) {
+                        this.searchCollection.url = UrlLinks.searchApiUrl(value.searchType);
+                    }
+                    _.extend(this.searchCollection.queryParams, { 'query': (value.query ? value.query.trim() : null), 'typeName': value.type || null, 'classification': value.tag || null });
+                    if (isPostMethod) {
+                        apiObj['data'] = _.extend({}, filterObj, _.pick(this.searchCollection.queryParams, 'query', 'excludeDeletedEntities', 'limit', 'offset', 'typeName', 'classification'))
+                        Globals.searchApiCallRef = this.searchCollection.getBasicRearchResult(apiObj);
+                        this.showHideFilter();
+                    } else {
+                        apiObj.data = null;
+                        Globals.searchApiCallRef = this.searchCollection.fetch(apiObj);
+                    }
+                } else {
+                    if (isPostMethod) {
+                        apiObj['data'] = _.extend({}, filterObj, _.pick(this.searchCollection.queryParams, 'query', 'excludeDeletedEntities', 'limit', 'offset', 'typeName', 'classification'));
+                        Globals.searchApiCallRef = this.searchCollection.getBasicRearchResult(apiObj);
+                    } else {
+                        apiObj.data = null;
+                        Globals.searchApiCallRef = this.searchCollection.fetch(apiObj);
+                    }
+                }
+            },
+            renderSearchQueryView: function() {
+                var that = this;
+                require(['views/search/SearchQueryView'], function(SearchQueryView) {
+                    that.RSearchQuery.show(new SearchQueryView({
+                        value: that.value,
+                        searchVent: that.searchVent
+                    }));
                 });
             },
             renderTableLayoutView: function(col) {
@@ -343,7 +428,7 @@ define(['require',
                     col = {};
                 col['Check'] = {
                     name: "selected",
-                    label: "",
+                    label: "Select",
                     cell: "select-row",
                     headerCell: "select-all"
                 };

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/tag/TagDetailLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/tag/TagDetailLayoutView.js b/dashboardv2/public/js/views/tag/TagDetailLayoutView.js
index 3bb27c3..410c629 100644
--- a/dashboardv2/public/js/views/tag/TagDetailLayoutView.js
+++ b/dashboardv2/public/js/views/tag/TagDetailLayoutView.js
@@ -44,7 +44,8 @@ define(['require',
              * @constructs
              */
             initialize: function(options) {
-                _.extend(this, _.pick(options, 'tag', 'collection', 'entityDefCollection', 'typeHeaders', 'enumDefCollection'));
+                _.extend(this, _.pick(options, 'tag', 'classificationDefCollection', 'entityDefCollection', 'typeHeaders', 'enumDefCollection'));
+                this.collection = this.classificationDefCollection;
             },
             bindEvents: function() {},
             onRender: function() {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/9a4ed469/dashboardv2/public/js/views/tag/TagLayoutView.js
----------------------------------------------------------------------
diff --git a/dashboardv2/public/js/views/tag/TagLayoutView.js b/dashboardv2/public/js/views/tag/TagLayoutView.js
index a5b5665..b8a1cd1 100644
--- a/dashboardv2/public/js/views/tag/TagLayoutView.js
+++ b/dashboardv2/public/js/views/tag/TagLayoutView.js
@@ -64,7 +64,7 @@ define(['require',
             bindEvents: function() {
                 var that = this;
                 this.listenTo(this.collection, "reset add remove", function() {
-                    this.tagsAndTypeGenerator('collection');
+                    this.tagsGenerator();
                 }, this);
                 this.ui.tagsParent.on('click', 'li.parent-node a', function() {
                     that.setUrl(this.getAttribute("href"));
@@ -147,7 +147,7 @@ define(['require',
                     }
                 }
             },
-            tagsAndTypeGenerator: function(collection, searchString) {
+            tagsGenerator: function(searchString) {
                 var that = this,
                     str = '';
                 that.collection.fullCollection.comparator = function(model) {
@@ -340,7 +340,7 @@ define(['require',
             },
             offlineSearchTag: function(e) {
                 var type = $(e.currentTarget).data('type');
-                this.tagsAndTypeGenerator('collection', $(e.currentTarget).val());
+                this.tagsGenerator($(e.currentTarget).val());
             },
             createTagAction: function() {
                 var that = this;