You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ol...@apache.org on 2016/09/28 09:42:14 UTC

[47/52] [abbrv] ambari git commit: AMBARI-18310. Logsearch - Refactor solr query layer (oleewere)

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonAuditLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonAuditLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonAuditLogRequestConverter.java
deleted file mode 100644
index d40c5a1..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonAuditLogRequestConverter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.BaseAuditLogRequest;
-import org.apache.ambari.logsearch.query.model.CommonSearchCriteria;
-import org.apache.commons.lang.StringEscapeUtils;
-import org.springframework.stereotype.Component;
-
-@Component
-public abstract class AbstractCommonAuditLogRequestConverter<SOURCE extends BaseAuditLogRequest, RESULT extends CommonSearchCriteria>
-  extends AbstractCommonSearchRequestConverter<SOURCE, RESULT> {
-
-  @Override
-  public RESULT convertToSearchCriteria(SOURCE request) {
-    RESULT criteria = createCriteria(request);
-    criteria.addParam("q", request.getQuery());
-    criteria.setMustBe(request.getMustBe());
-    criteria.setMustNot(request.getMustNot());
-    criteria.setExcludeQuery(StringEscapeUtils.unescapeXml(request.getExcludeQuery()));
-    criteria.setIncludeQuery(StringEscapeUtils.unescapeXml(request.getIncludeQuery()));
-    criteria.setStartTime(request.getFrom());
-    criteria.setEndTime(request.getTo());
-    return criteria;
-  }
-
-  public abstract RESULT createCriteria(SOURCE request);
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonSearchRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonSearchRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonSearchRequestConverter.java
deleted file mode 100644
index ea2c28a..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonSearchRequestConverter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.CommonSearchRequest;
-import org.apache.ambari.logsearch.query.model.CommonSearchCriteria;
-import org.apache.commons.lang.StringUtils;
-
-import static org.apache.ambari.logsearch.query.SearchCriteriaConstants.PARAM_GLOBAL_END_TIME;
-import static org.apache.ambari.logsearch.query.SearchCriteriaConstants.PARAM_GLOBAL_START_TIME;
-
-public abstract class AbstractCommonSearchRequestConverter<SOURCE extends CommonSearchRequest, RESULT extends CommonSearchCriteria>
-  extends AbstractConverterAware<SOURCE, RESULT> {
-
-  @Override
-  public RESULT convert(SOURCE source) {
-    RESULT criteria = convertToSearchCriteria(source);
-    addDefaultParams(source, criteria);
-    return criteria;
-  }
-
-  public abstract RESULT convertToSearchCriteria(SOURCE source);
-
-  private void addDefaultParams(SOURCE request, RESULT criteria) {
-    criteria.setStartIndex(StringUtils.isNumeric(request.getStartIndex()) ? new Integer(request.getStartIndex()) : 0);
-    criteria.setPage(StringUtils.isNumeric(request.getPage()) ? new Integer(request.getPage()) : 0);
-    criteria.setMaxRows(StringUtils.isNumeric(request.getPageSize()) ? new Integer(request.getPageSize()) : 50);
-    criteria.setSortBy(request.getSortBy());
-    criteria.setSortType(request.getSortType());
-    if (StringUtils.isNotEmpty(request.getStartTime())){
-      criteria.setGlobalStartTime(request.getStartTime());
-      criteria.getUrlParamMap().put(PARAM_GLOBAL_START_TIME, request.getStartTime());
-    }
-    if (StringUtils.isNotEmpty(request.getEndTime())){
-      criteria.setGlobalEndTime(request.getEndTime());
-      criteria.getUrlParamMap().put(PARAM_GLOBAL_END_TIME, request.getEndTime());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonServiceLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonServiceLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonServiceLogRequestConverter.java
deleted file mode 100644
index 8e91584..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractCommonServiceLogRequestConverter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.BaseServiceLogRequest;
-import org.apache.ambari.logsearch.query.model.CommonServiceLogSearchCriteria;
-import org.apache.commons.lang.StringEscapeUtils;
-
-public abstract class AbstractCommonServiceLogRequestConverter<SOURCE extends BaseServiceLogRequest, RESULT extends CommonServiceLogSearchCriteria>
-  extends AbstractCommonSearchRequestConverter<SOURCE, RESULT> {
-
-  @Override
-  public RESULT convertToSearchCriteria(SOURCE request) {
-    RESULT criteria = createCriteria(request);
-    // TODO: check are these used from the UI or not?
-    criteria.addParam("q", request.getQuery());
-    criteria.addParam("unselectComp", request.getMustNot());
-
-    criteria.setLevel(request.getLevel());
-    criteria.setFrom(request.getFrom());
-    criteria.setTo(request.getTo());
-    criteria.setSelectComp(request.getMustBe());
-    criteria.setBundleId(request.getBundleId());
-    criteria.setHostName(request.getHostName());
-    criteria.setComponentName(request.getComponentName());
-    criteria.setFileName(request.getFileName());
-    criteria.setStartTime(request.getStartTime());
-    criteria.setEndTime(request.getEndTime());
-    criteria.setExcludeQuery(StringEscapeUtils.unescapeXml(request.getExcludeQuery()));
-    criteria.setIncludeQuery(StringEscapeUtils.unescapeXml(request.getIncludeQuery()));
-    return criteria;
-  }
-
-  public abstract RESULT createCriteria(SOURCE request);
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractConverterAware.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractConverterAware.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractConverterAware.java
index 18a71c1..d8470ea 100644
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractConverterAware.java
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractConverterAware.java
@@ -18,18 +18,18 @@
  */
 package org.apache.ambari.logsearch.query.converter;
 
-import org.springframework.beans.factory.annotation.Qualifier;
 import org.springframework.core.convert.ConversionService;
 import org.springframework.core.convert.converter.Converter;
 import org.springframework.core.convert.converter.ConverterRegistry;
 
 import javax.annotation.PostConstruct;
 import javax.inject.Inject;
+import javax.inject.Named;
 
 public abstract class AbstractConverterAware<SOURCE, RESULT> implements Converter<SOURCE, RESULT> {
 
   @Inject
-  @Qualifier("conversionService")
+  @Named("conversionService")
   private ConversionService conversionService;
 
   public ConversionService getConversionService() {

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractDateRangeFacetQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractDateRangeFacetQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractDateRangeFacetQueryConverter.java
new file mode 100644
index 0000000..12d19d2
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractDateRangeFacetQueryConverter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.common.LogSearchConstants;
+import org.apache.ambari.logsearch.model.request.DateRangeParamDefinition;
+import org.apache.ambari.logsearch.model.request.UnitParamDefinition;
+import org.apache.commons.lang.StringUtils;
+import org.apache.solr.client.solrj.SolrQuery;
+
+import java.util.Locale;
+
+public abstract class AbstractDateRangeFacetQueryConverter<SOURCE extends DateRangeParamDefinition & UnitParamDefinition>
+  extends AbstractConverterAware<SOURCE , SolrQuery> {
+
+  @Override
+  public SolrQuery convert(SOURCE request) {
+    SolrQuery solrQuery = new SolrQuery();
+    String unit = StringUtils.isBlank(request.getUnit()) ? request.getUnit() : "+1HOUR";
+    solrQuery.setQuery("*:*");
+    solrQuery.setFacet(true);
+    solrQuery.addFacetPivotField("{!range=r1}" + getTypeFieldName());
+    solrQuery.setFacetMinCount(1);
+    solrQuery.setFacetLimit(-1);
+    solrQuery.setFacetSort(LogSearchConstants.FACET_INDEX);
+    solrQuery.add("facet.range", "{!tag=r1}" + getDateFieldName());
+    solrQuery.add(String.format(Locale.ROOT, "f.%s.%s", new Object[]{getDateFieldName(), "facet.range.start"}), request.getFrom());
+    solrQuery.add(String.format(Locale.ROOT, "f.%s.%s", new Object[]{getDateFieldName(), "facet.range.end"}), request.getTo());
+    solrQuery.add(String.format(Locale.ROOT, "f.%s.%s", new Object[]{getDateFieldName(), "facet.range.gap"}), unit);
+    solrQuery.remove("sort");
+    solrQuery.setRows(0);
+    solrQuery.setStart(0);
+    return solrQuery;
+  }
+
+  public abstract String getDateFieldName();
+
+  public abstract String getTypeFieldName();
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestFacetQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestFacetQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestFacetQueryConverter.java
new file mode 100644
index 0000000..17c4e01
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestFacetQueryConverter.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.BaseLogRequest;
+import org.apache.commons.lang.StringUtils;
+import org.springframework.data.solr.core.query.Criteria;
+import org.springframework.data.solr.core.query.FacetOptions;
+import org.springframework.data.solr.core.query.SimpleFacetQuery;
+import org.springframework.data.solr.core.query.SimpleFilterQuery;
+import org.springframework.data.solr.core.query.SimpleStringCriteria;
+
+public abstract class AbstractLogRequestFacetQueryConverter<SOURCE extends BaseLogRequest> extends AbstractConverterAware<SOURCE, SimpleFacetQuery>{
+
+  @Override
+  public SimpleFacetQuery convert(SOURCE request) {
+    String fromValue = StringUtils.isNotEmpty(request.getFrom()) ? request.getFrom() : "*";
+    String toValue = StringUtils.isNotEmpty(request.getTo()) ? request.getTo() : "*";
+    Criteria criteria = new SimpleStringCriteria("*:*");
+    SimpleFacetQuery facetQuery = new SimpleFacetQuery();
+    facetQuery.addCriteria(criteria);
+    SimpleFilterQuery simpleFilterQuery = new SimpleFilterQuery();
+    simpleFilterQuery.addCriteria(new SimpleStringCriteria(getDateTimeField() + ":[" + fromValue +" TO "+ toValue+ "]" ));
+    facetQuery.addFilterQuery(simpleFilterQuery);
+    FacetOptions facetOptions = new FacetOptions();
+    facetOptions.setFacetMinCount(1);
+    facetOptions.setFacetSort(getFacetSort());
+    appendFacetOptions(facetOptions, request);
+    facetQuery.setFacetOptions(facetOptions);
+    facetQuery.setRows(0);
+    appendFacetQuery(facetQuery, request);
+    return facetQuery;
+  }
+
+  public abstract FacetOptions.FacetSort getFacetSort();
+
+  public abstract String getDateTimeField();
+
+  public void appendFacetQuery(SimpleFacetQuery facetQuery, SOURCE request) {
+  }
+
+  public void appendFacetOptions(FacetOptions facetOptions, SOURCE request) {
+    facetOptions.setFacetLimit(-1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestQueryConverter.java
new file mode 100644
index 0000000..a4ca73d
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractLogRequestQueryConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.BaseLogRequest;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.springframework.data.solr.core.query.Query;
+
+import java.util.List;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.COMPONENT;
+
+public abstract class AbstractLogRequestQueryConverter<REQUEST_TYPE extends BaseLogRequest, QUERY_TYPE extends Query>
+  extends AbstractSearchRequestQueryConverter<REQUEST_TYPE, QUERY_TYPE> {
+
+  @Override
+  public QUERY_TYPE extendSolrQuery(REQUEST_TYPE request, QUERY_TYPE query) {
+    List<String> includeTypes = splitValueAsList(request.getMustBe(), ",");
+    List<String> excludeTypes = splitValueAsList(request.getMustNot(), ",");
+    addInFilterQuery(query, COMPONENT, includeTypes);
+    addInFilterQuery(query, COMPONENT, excludeTypes, true);
+    addIncludeFieldValues(query, StringEscapeUtils.unescapeXml(request.getIncludeQuery()));
+    addExcludeFieldValues(query, StringEscapeUtils.unescapeXml(request.getExcludeQuery()));
+    return extendLogQuery(request, query);
+  }
+
+  public abstract QUERY_TYPE extendLogQuery(REQUEST_TYPE request, QUERY_TYPE query);
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractSearchRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractSearchRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractSearchRequestQueryConverter.java
new file mode 100644
index 0000000..b8e351c
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AbstractSearchRequestQueryConverter.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import com.google.common.base.Splitter;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+import org.apache.ambari.logsearch.common.LogSearchConstants;
+import org.apache.ambari.logsearch.common.LogType;
+import org.apache.ambari.logsearch.dao.SolrSchemaFieldDao;
+import org.apache.ambari.logsearch.model.request.impl.CommonSearchRequest;
+import org.apache.ambari.logsearch.util.SolrUtil;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.springframework.data.domain.PageRequest;
+import org.springframework.data.domain.Sort;
+import org.springframework.data.solr.core.query.Criteria;
+import org.springframework.data.solr.core.query.Query;
+import org.springframework.data.solr.core.query.SimpleFilterQuery;
+import org.springframework.data.solr.core.query.SimpleStringCriteria;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOG_MESSAGE;
+
+public abstract class AbstractSearchRequestQueryConverter<REQUEST_TYPE extends CommonSearchRequest, QUERY_TYPE extends Query>
+  extends AbstractConverterAware<REQUEST_TYPE, QUERY_TYPE> {
+
+  @Inject
+  @Named("serviceSolrFieldDao")
+  private SolrSchemaFieldDao serviceSolrSchemaFieldDao;
+
+  @Inject
+  @Named("auditSolrFieldDao")
+  private SolrSchemaFieldDao auditSolrSchemaFieldDao;
+
+  @Override
+  public QUERY_TYPE convert(REQUEST_TYPE request) {
+    QUERY_TYPE query = createQuery();
+    int page = StringUtils.isNumeric(request.getPage()) ? new Integer(request.getPage()) : 0;
+    int pageSize = StringUtils.isNumeric(request.getPageSize()) ? new Integer(request.getPageSize()) : Integer.MAX_VALUE;
+    PageRequest pageRequest = new PageRequest(page, pageSize, sort(request));
+    query.setPageRequest(pageRequest);
+    Criteria criteria = new SimpleStringCriteria("*:*");
+    query.addCriteria(criteria);
+    return extendSolrQuery(request, query);
+  }
+
+  public abstract QUERY_TYPE extendSolrQuery(REQUEST_TYPE request, QUERY_TYPE query);
+
+  public abstract Sort sort(REQUEST_TYPE request);
+
+  public abstract QUERY_TYPE createQuery();
+
+  public abstract LogType getLogType();
+
+  public List<String> splitValueAsList(String value, String separator) {
+    return StringUtils.isNotEmpty(value) ? Splitter.on(separator).omitEmptyStrings().splitToList(value) : null;
+  }
+
+  public Query addEqualsFilterQuery(Query query, String field, String value) {
+    return this.addEqualsFilterQuery(query, field, value, false);
+  }
+
+  public Query addEqualsFilterQuery(Query query, String field, String value, boolean negate) {
+    if (StringUtils.isNotEmpty(value)) {
+      addFilterQuery(query, new Criteria(field).is(value), negate);
+    }
+    return query;
+  }
+
+  public Query addContainsFilterQuery(Query query, String field, String value) {
+    return this.addContainsFilterQuery(query, field, value, false);
+  }
+
+  public Query addContainsFilterQuery(Query query, String field, String value, boolean negate) {
+    if (StringUtils.isNotEmpty(value)) {
+      addFilterQuery(query, new Criteria(field).contains(value), negate);
+    }
+    return query;
+  }
+
+  public Query addInFilterQuery(Query query, String field, List<String> values) {
+    return this.addInFilterQuery(query, field, values, false);
+  }
+
+  public Query addInFilterQuery(Query query, String field, List<String> values, boolean negate) {
+    if (CollectionUtils.isNotEmpty(values)) {
+      addFilterQuery(query, new Criteria(field).is(values), negate);
+    }
+    return query;
+  }
+
+  public Query addRangeFilter(Query query, String field, String from, String to) {
+    return this.addRangeFilter(query, field, from, to, false);
+  }
+
+  public Query addRangeFilter(Query query, String field, String from, String to, boolean negate) { // TODO use criteria.between without escaping
+    String fromValue = StringUtils.isNotEmpty(from) ? from : "*";
+    String toValue = StringUtils.isNotEmpty(to) ? to : "*";
+    addFilterQuery(query, new SimpleStringCriteria(field + ":[" + fromValue +" TO "+ toValue + "]" ), negate);
+    return query;
+  }
+
+  public Query addIncludeFieldValues(Query query, String fieldValuesMapStr) {
+    if (StringUtils.isNotEmpty(fieldValuesMapStr)) {
+      List<Map<String, String>> criterias = new Gson().fromJson(fieldValuesMapStr,
+        new TypeToken<List<HashMap<String, String>>>(){}.getType());
+      for (Map<String, String> criteriaMap : criterias) {
+        for (Map.Entry<String, String> fieldEntry : criteriaMap.entrySet()) {
+          escapeFieldValueByType(fieldEntry);
+          addFilterQuery(query, new Criteria(fieldEntry.getKey()).is(escapeFieldValueByType(fieldEntry)), false);
+        }
+      }
+    }
+    return query;
+  }
+
+  public Query addExcludeFieldValues(Query query, String fieldValuesMapStr) {
+    if (StringUtils.isNotEmpty(fieldValuesMapStr)) {
+      List<Map<String, String>> criterias = new Gson().fromJson(fieldValuesMapStr,
+        new TypeToken<List<HashMap<String, String>>>(){}.getType());
+      for (Map<String, String> criteriaMap : criterias) {
+        for (Map.Entry<String, String> fieldEntry : criteriaMap.entrySet()) {
+          addFilterQuery(query, new Criteria(fieldEntry.getKey()).is(escapeFieldValueByType(fieldEntry)), true);
+        }
+      }
+    }
+    return query;
+  }
+
+  private String escapeFieldValueByType(Map.Entry<String, String> fieldEntry) {
+    String escapedFieldValue;
+    if (fieldEntry.getKey().equalsIgnoreCase(LOG_MESSAGE)) {
+      escapedFieldValue = SolrUtil.escapeForLogMessage(fieldEntry.getValue());
+    } else {
+      escapedFieldValue = SolrUtil.putWildCardByType(fieldEntry.getValue(), fieldEntry.getKey(), getSchemaFieldsTypeMapByLogType(getLogType()));
+    }
+    return escapedFieldValue;
+  }
+
+  private void addFilterQuery(Query query, Criteria criteria, boolean negate) {
+    if (negate) {
+      criteria.not();
+    }
+    query.addFilterQuery(new SimpleFilterQuery(criteria));
+  }
+
+  private Map<String, String> getSchemaFieldsTypeMapByLogType(LogType logType) {
+    return LogType.AUDIT.equals(logType) ? auditSolrSchemaFieldDao.getSchemaFieldTypeMap() : serviceSolrSchemaFieldDao.getSchemaFieldTypeMap();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AnyGraphRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AnyGraphRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AnyGraphRequestConverter.java
deleted file mode 100644
index 1639563..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AnyGraphRequestConverter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.AnyGraphRequest;
-import org.apache.ambari.logsearch.query.model.AnyGraphSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class AnyGraphRequestConverter extends AbstractCommonSearchRequestConverter<AnyGraphRequest, AnyGraphSearchCriteria> {
-
-  @Override
-  public AnyGraphSearchCriteria convertToSearchCriteria(AnyGraphRequest anyGraphRequest) {
-    AnyGraphSearchCriteria criteria = new AnyGraphSearchCriteria();
-    criteria.setxAxis(anyGraphRequest.getxAxis());
-    criteria.setyAxis(anyGraphRequest.getyAxis());
-    criteria.setStackBy(anyGraphRequest.getStackBy());
-    criteria.setUnit(anyGraphRequest.getUnit());
-    criteria.setFrom(anyGraphRequest.getFrom());
-    criteria.setTo(anyGraphRequest.getTo());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestConverter.java
deleted file mode 100644
index ac74287..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.AuditBarGraphRequest;
-import org.apache.ambari.logsearch.query.model.AuditBarGraphSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class AuditBarGraphRequestConverter extends AbstractCommonAuditLogRequestConverter<AuditBarGraphRequest, AuditBarGraphSearchCriteria>{
-
-  @Override
-  public AuditBarGraphSearchCriteria createCriteria(AuditBarGraphRequest request) {
-    AuditBarGraphSearchCriteria criteria = new AuditBarGraphSearchCriteria();
-    criteria.setUnit(request.getUnit());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestQueryConverter.java
new file mode 100644
index 0000000..eca88c2
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditBarGraphRequestQueryConverter.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.AuditBarGraphRequest;
+
+import javax.inject.Named;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_COMPONENT;
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_EVTTIME;
+
+@Named
+public class AuditBarGraphRequestQueryConverter extends AbstractDateRangeFacetQueryConverter<AuditBarGraphRequest> {
+
+  @Override
+  public String getDateFieldName() {
+    return AUDIT_EVTTIME;
+  }
+
+  @Override
+  public String getTypeFieldName() {
+    return AUDIT_COMPONENT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditComponentsRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditComponentsRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditComponentsRequestQueryConverter.java
new file mode 100644
index 0000000..645c0c3
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditComponentsRequestQueryConverter.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.common.LogSearchConstants;
+import org.apache.ambari.logsearch.common.LogType;
+import org.apache.ambari.logsearch.model.request.impl.AuditComponentRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.data.domain.Sort;
+import org.springframework.data.solr.core.query.FacetOptions;
+import org.springframework.data.solr.core.query.SimpleFacetQuery;
+
+import javax.inject.Named;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_COMPONENT;
+
+@Named
+public class AuditComponentsRequestQueryConverter extends AbstractSearchRequestQueryConverter<AuditComponentRequest, SimpleFacetQuery> {
+
+  @Override
+  public SimpleFacetQuery extendSolrQuery(AuditComponentRequest request, SimpleFacetQuery query) {
+    FacetOptions facetOptions = new FacetOptions();
+    facetOptions.addFacetOnField(AUDIT_COMPONENT);
+    facetOptions.setFacetSort(FacetOptions.FacetSort.INDEX);
+    facetOptions.setFacetLimit(-1);
+    query.setFacetOptions(facetOptions);
+    return query;
+  }
+
+  @Override
+  public Sort sort(AuditComponentRequest request) {
+    Sort.Direction direction = StringUtils.equals(request.getSortType(), LogSearchConstants.DESCENDING_ORDER)
+      ? Sort.Direction.DESC : Sort.Direction.ASC;
+    return new Sort(new Sort.Order(direction, AUDIT_COMPONENT));
+  }
+
+  @Override
+  public SimpleFacetQuery createQuery() {
+    return new SimpleFacetQuery();
+  }
+
+  @Override
+  public LogType getLogType() {
+    return LogType.AUDIT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestConverter.java
deleted file mode 100644
index 5ec7632..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.AuditLogRequest;
-import org.apache.ambari.logsearch.query.model.AuditLogSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class AuditLogRequestConverter extends AbstractCommonAuditLogRequestConverter<AuditLogRequest, AuditLogSearchCriteria> {
-
-  @Override
-  public AuditLogSearchCriteria createCriteria(AuditLogRequest request) {
-    AuditLogSearchCriteria criteria = new AuditLogSearchCriteria();
-    criteria.setLastPage(request.isLastPage());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestQueryConverter.java
new file mode 100644
index 0000000..317b1fa
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditLogRequestQueryConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.common.LogType;
+import org.apache.ambari.logsearch.model.request.impl.AuditLogRequest;
+import org.springframework.data.solr.core.query.SimpleQuery;
+
+import javax.inject.Named;
+
+@Named
+public class AuditLogRequestQueryConverter extends AbstractAuditLogRequestQueryConverter<AuditLogRequest, SimpleQuery> {
+
+  @Override
+  public SimpleQuery extendLogQuery(AuditLogRequest request, SimpleQuery query) {
+    return query;
+  }
+
+  @Override
+  public SimpleQuery createQuery() {
+    return new SimpleQuery();
+  }
+
+  @Override
+  public LogType getLogType() {
+    return LogType.AUDIT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditServiceLoadRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditServiceLoadRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditServiceLoadRequestQueryConverter.java
new file mode 100644
index 0000000..1008191
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/AuditServiceLoadRequestQueryConverter.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.AuditServiceLoadRequest;
+import org.apache.ambari.logsearch.model.request.impl.BaseLogRequest;
+import org.springframework.data.solr.core.query.FacetOptions;
+
+import javax.inject.Named;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_EVTTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_COMPONENT;
+
+@Named
+public class AuditServiceLoadRequestQueryConverter extends AbstractLogRequestFacetQueryConverter<AuditServiceLoadRequest> {
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.COUNT;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return AUDIT_EVTTIME;
+  }
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, AuditServiceLoadRequest request) {
+    facetOptions.addFacetOnField(AUDIT_COMPONENT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseAuditLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseAuditLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseAuditLogRequestConverter.java
deleted file mode 100644
index 35aceb2..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseAuditLogRequestConverter.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.BaseAuditLogRequest;
-import org.apache.ambari.logsearch.query.model.CommonSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class BaseAuditLogRequestConverter extends AbstractCommonAuditLogRequestConverter<BaseAuditLogRequest, CommonSearchCriteria> {
-
-  @Override
-  public CommonSearchCriteria createCriteria(BaseAuditLogRequest request) {
-    return new CommonSearchCriteria();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestConverter.java
deleted file mode 100644
index efc9bc9..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestConverter.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.BaseServiceLogRequest;
-import org.apache.ambari.logsearch.query.model.CommonSearchCriteria;
-import org.apache.ambari.logsearch.query.model.CommonServiceLogSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class BaseServiceLogRequestConverter extends AbstractCommonServiceLogRequestConverter<BaseServiceLogRequest, CommonServiceLogSearchCriteria> {
-
-  @Override
-  public CommonServiceLogSearchCriteria createCriteria(BaseServiceLogRequest request) {
-    return new CommonServiceLogSearchCriteria();
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestQueryConverter.java
new file mode 100644
index 0000000..1d86844
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/BaseServiceLogRequestQueryConverter.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.common.LogSearchConstants;
+import org.apache.ambari.logsearch.common.LogType;
+import org.apache.ambari.logsearch.model.request.impl.BaseServiceLogRequest;
+import org.apache.ambari.logsearch.util.SolrUtil;
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.data.domain.Sort;
+import org.springframework.data.solr.core.query.SimpleQuery;
+import javax.inject.Named;
+import java.util.List;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.KEY_LOG_MESSAGE;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.HOST;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.PATH;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.COMPONENT;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.BUNDLE_ID;
+import static org.apache.ambari.logsearch.solr.SolrConstants.CommonLogConstants.SEQUENCE_ID;
+
+@Named
+public class BaseServiceLogRequestQueryConverter extends AbstractLogRequestQueryConverter<BaseServiceLogRequest, SimpleQuery> {
+
+  @Override
+  public SimpleQuery extendLogQuery(BaseServiceLogRequest request, SimpleQuery query) {
+    List<String> levels = splitValueAsList(request.getLevel(), ",");
+    addContainsFilterQuery(query, KEY_LOG_MESSAGE, SolrUtil.escapeForStandardTokenizer(request.getiMessage()));
+    addContainsFilterQuery(query, KEY_LOG_MESSAGE, SolrUtil.escapeForStandardTokenizer(request.geteMessage()), true);
+    addEqualsFilterQuery(query, HOST, SolrUtil.escapeQueryChars(request.getHostName()));
+    addEqualsFilterQuery(query, PATH, SolrUtil.escapeQueryChars(request.getFileName()));
+    addEqualsFilterQuery(query, COMPONENT, SolrUtil.escapeQueryChars(request.getComponentName()));
+    addEqualsFilterQuery(query, BUNDLE_ID, request.getBundleId());
+    addInFilterQuery(query, LEVEL, levels);
+    addRangeFilter(query, LOGTIME, request.getFrom(), request.getTo());
+    return query;
+  }
+
+  @Override
+  public Sort sort(BaseServiceLogRequest request) {
+    String sortBy = request.getSortBy();
+    String sortType = request.getSortType();
+    Sort.Order defaultSortOrder;
+    if (!StringUtils.isBlank(sortBy)) {
+      Sort.Direction direction = StringUtils.equals(sortType, LogSearchConstants.ASCENDING_ORDER) ? Sort.Direction.ASC : Sort.Direction.DESC;
+      defaultSortOrder = new Sort.Order(direction, sortBy);
+    } else {
+      defaultSortOrder = new Sort.Order(Sort.Direction.DESC, LOGTIME);
+    }
+    Sort.Order secuqnceIdOrder = new Sort.Order(Sort.Direction.DESC, SEQUENCE_ID);
+    return new Sort(defaultSortOrder, secuqnceIdOrder);
+  }
+
+  @Override
+  public SimpleQuery createQuery() {
+    return new SimpleQuery();
+  }
+
+  @Override
+  public LogType getLogType() {
+    return LogType.SERVICE;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestConverter.java
deleted file mode 100644
index 6197d48..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.FieldAuditLogRequest;
-import org.apache.ambari.logsearch.query.model.FieldAuditLogSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class FieldAuditLogRequestConverter extends AbstractCommonAuditLogRequestConverter<FieldAuditLogRequest, FieldAuditLogSearchCriteria> {
-
-  @Override
-  public FieldAuditLogSearchCriteria createCriteria(FieldAuditLogRequest request) {
-    FieldAuditLogSearchCriteria criteria = new FieldAuditLogSearchCriteria();
-    criteria.setField(request.getField());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestQueryConverter.java
new file mode 100644
index 0000000..02c1557
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldAuditLogRequestQueryConverter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.FieldAuditLogRequest;
+import org.springframework.data.solr.core.query.FacetOptions;
+
+import javax.inject.Named;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_EVTTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_COMPONENT;
+
+@Named
+public class FieldAuditLogRequestQueryConverter extends AbstractLogRequestFacetQueryConverter<FieldAuditLogRequest> {
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, FieldAuditLogRequest request) {
+    facetOptions.addFacetOnPivot(request.getField(), AUDIT_COMPONENT);
+    facetOptions.setFacetLimit(request.getTop());
+  }
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.COUNT;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return AUDIT_EVTTIME;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldBarGraphRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldBarGraphRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldBarGraphRequestConverter.java
deleted file mode 100644
index 74b0dac..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/FieldBarGraphRequestConverter.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.FieldAuditBarGraphRequest;
-import org.apache.ambari.logsearch.query.model.FieldAuditBarGraphSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class FieldBarGraphRequestConverter extends AbstractCommonAuditLogRequestConverter<FieldAuditBarGraphRequest, FieldAuditBarGraphSearchCriteria> {
-
-  @Override
-  public FieldAuditBarGraphSearchCriteria createCriteria(FieldAuditBarGraphRequest request) {
-    FieldAuditBarGraphSearchCriteria criteria = new FieldAuditBarGraphSearchCriteria();
-    criteria.setUnit(request.getUnit());
-    criteria.setField(request.getField());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceAnyGraphRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceAnyGraphRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceAnyGraphRequestConverter.java
deleted file mode 100644
index 8f1aaf0..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceAnyGraphRequestConverter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.ServiceAnyGraphRequest;
-import org.apache.ambari.logsearch.query.model.ServiceAnyGraphSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class ServiceAnyGraphRequestConverter extends AbstractCommonServiceLogRequestConverter<ServiceAnyGraphRequest, ServiceAnyGraphSearchCriteria> {
-
-  @Override
-  public ServiceAnyGraphSearchCriteria createCriteria(ServiceAnyGraphRequest anyGraphRequest) {
-    ServiceAnyGraphSearchCriteria criteria = new ServiceAnyGraphSearchCriteria();
-    criteria.setxAxis(anyGraphRequest.getxAxis());
-    criteria.setyAxis(anyGraphRequest.getyAxis());
-    criteria.setStackBy(anyGraphRequest.getStackBy());
-    criteria.setUnit(anyGraphRequest.getUnit());
-    criteria.setFrom(anyGraphRequest.getFrom());
-    criteria.setTo(anyGraphRequest.getTo());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceExtremeDatesRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceExtremeDatesRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceExtremeDatesRequestConverter.java
deleted file mode 100644
index 489e879..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceExtremeDatesRequestConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.ServiceExtremeDatesRequest;
-import org.apache.ambari.logsearch.query.model.ServiceExtremeDatesCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class ServiceExtremeDatesRequestConverter extends AbstractCommonSearchRequestConverter<ServiceExtremeDatesRequest, ServiceExtremeDatesCriteria> {
-
-  @Override
-  public ServiceExtremeDatesCriteria convertToSearchCriteria(ServiceExtremeDatesRequest request) {
-    ServiceExtremeDatesCriteria criteria = new ServiceExtremeDatesCriteria();
-    criteria.setBundleId(request.getBundleId());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceGraphRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceGraphRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceGraphRequestConverter.java
deleted file mode 100644
index 37ec7dc..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceGraphRequestConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.ServiceGraphRequest;
-import org.apache.ambari.logsearch.query.model.ServiceGraphSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class ServiceGraphRequestConverter extends AbstractCommonServiceLogRequestConverter<ServiceGraphRequest, ServiceGraphSearchCriteria> {
-
-  @Override
-  public ServiceGraphSearchCriteria createCriteria(ServiceGraphRequest request) {
-    ServiceGraphSearchCriteria criteria = new ServiceGraphSearchCriteria();
-    criteria.setUnit(request.getUnit());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogAnyGraphRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogAnyGraphRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogAnyGraphRequestQueryConverter.java
new file mode 100644
index 0000000..4c549a0
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogAnyGraphRequestQueryConverter.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.ServiceAnyGraphRequest;
+import org.springframework.data.solr.core.query.FacetOptions;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+
+import javax.inject.Named;
+
+@Named
+public class ServiceLogAnyGraphRequestQueryConverter extends AbstractLogRequestFacetQueryConverter<ServiceAnyGraphRequest>{
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, ServiceAnyGraphRequest request) {
+    facetOptions.addFacetOnField(LEVEL);
+  }
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.COUNT;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return LOGTIME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentLevelRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentLevelRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentLevelRequestQueryConverter.java
new file mode 100644
index 0000000..0154128
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentLevelRequestQueryConverter.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import com.google.common.base.Splitter;
+import org.apache.ambari.logsearch.model.request.impl.ServiceLogComponentLevelRequest;
+import org.springframework.data.solr.core.query.Criteria;
+import org.springframework.data.solr.core.query.FacetOptions;
+import org.springframework.data.solr.core.query.SimpleFacetQuery;
+import org.springframework.data.solr.core.query.SimpleFilterQuery;
+
+import javax.inject.Named;
+import java.util.List;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.COMPONENT;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+
+@Named
+public class ServiceLogComponentLevelRequestQueryConverter extends AbstractLogRequestFacetQueryConverter <ServiceLogComponentLevelRequest> {
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.INDEX;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return LOGTIME;
+  }
+
+  @Override
+  public void appendFacetQuery(SimpleFacetQuery facetQuery, ServiceLogComponentLevelRequest request) {
+    List<String> levels = Splitter.on(",").splitToList(request.getLevel());
+    SimpleFilterQuery filterQuery = new SimpleFilterQuery();
+    filterQuery.addCriteria(new Criteria(LEVEL).in(levels));
+    facetQuery.addFilterQuery(filterQuery);
+  }
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, ServiceLogComponentLevelRequest request) {
+    facetOptions.addFacetOnPivot(COMPONENT, LEVEL);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentRequestFacetQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentRequestFacetQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentRequestFacetQueryConverter.java
new file mode 100644
index 0000000..6aa8e2d
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogComponentRequestFacetQueryConverter.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import com.google.common.base.Splitter;
+import org.apache.ambari.logsearch.model.request.impl.ServiceLogComponentHostRequest;
+import org.springframework.data.solr.core.query.Criteria;
+import org.springframework.data.solr.core.query.FacetOptions;
+import org.springframework.data.solr.core.query.SimpleFacetQuery;
+import org.springframework.data.solr.core.query.SimpleFilterQuery;
+
+import javax.inject.Named;
+import java.util.List;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.COMPONENT;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.HOST;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+
+@Named
+public class ServiceLogComponentRequestFacetQueryConverter extends AbstractLogRequestFacetQueryConverter<ServiceLogComponentHostRequest> {
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.INDEX;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return LOGTIME;
+  }
+
+  @Override
+  public void appendFacetQuery(SimpleFacetQuery facetQuery, ServiceLogComponentHostRequest request) {
+    List<String> levels = Splitter.on(",").splitToList(request.getLevel());
+    SimpleFilterQuery filterQuery = new SimpleFilterQuery();
+    filterQuery.addCriteria(new Criteria(LEVEL).in(levels));
+    facetQuery.addFilterQuery(filterQuery);
+  }
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, ServiceLogComponentHostRequest request) {
+    facetOptions.addFacetOnPivot(COMPONENT, HOST, LEVEL);
+    facetOptions.addFacetOnPivot(COMPONENT, LEVEL);
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogExportRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogExportRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogExportRequestConverter.java
deleted file mode 100644
index 783b0e0..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogExportRequestConverter.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.ServiceLogExportRequest;
-import org.apache.ambari.logsearch.query.model.ServiceLogExportSearchCriteria;
-import org.springframework.stereotype.Component;
-
-@Component
-public class ServiceLogExportRequestConverter extends AbstractCommonServiceLogRequestConverter<ServiceLogExportRequest, ServiceLogExportSearchCriteria> {
-
-  @Override
-  public ServiceLogExportSearchCriteria createCriteria(ServiceLogExportRequest request) {
-    ServiceLogExportSearchCriteria criteria = new ServiceLogExportSearchCriteria();
-    criteria.setFormat(request.getFormat());
-    criteria.setUtcOffset(request.getUtcOffset());
-    return criteria;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelCountRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelCountRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelCountRequestQueryConverter.java
new file mode 100644
index 0000000..30cb05b
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelCountRequestQueryConverter.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import org.apache.ambari.logsearch.model.request.impl.ServiceLogLevelCountRequest;
+import org.springframework.data.solr.core.query.FacetOptions;
+
+import javax.inject.Named;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+
+@Named
+public class ServiceLogLevelCountRequestQueryConverter extends AbstractLogRequestFacetQueryConverter<ServiceLogLevelCountRequest> {
+
+  @Override
+  public FacetOptions.FacetSort getFacetSort() {
+    return FacetOptions.FacetSort.COUNT;
+  }
+
+  @Override
+  public String getDateTimeField() {
+    return LOGTIME;
+  }
+
+  @Override
+  public void appendFacetOptions(FacetOptions facetOptions, ServiceLogLevelCountRequest request) {
+    facetOptions.addFacetOnField(LEVEL);
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelDateRangeRequestQueryConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelDateRangeRequestQueryConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelDateRangeRequestQueryConverter.java
new file mode 100644
index 0000000..2b7a521
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogLevelDateRangeRequestQueryConverter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.ambari.logsearch.query.converter;
+
+import com.google.common.base.Splitter;
+import org.apache.ambari.logsearch.model.request.impl.ServiceGraphRequest;
+import org.apache.commons.lang.StringUtils;
+import org.apache.solr.client.solrj.SolrQuery;
+
+import javax.inject.Named;
+
+import java.util.List;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+
+@Named
+public class ServiceLogLevelDateRangeRequestQueryConverter extends AbstractDateRangeFacetQueryConverter<ServiceGraphRequest>{
+
+  @Override
+  public String getDateFieldName() {
+    return LOGTIME;
+  }
+
+  @Override
+  public String getTypeFieldName() {
+    return LEVEL;
+  }
+
+  @Override
+  public SolrQuery convert(ServiceGraphRequest request) {
+    SolrQuery solrQuery = super.convert(request);
+    List<String> levels = Splitter.on(",").splitToList(request.getLevel());
+    if (!levels.isEmpty()) {
+      solrQuery.addFilterQuery(String.format("%s:(%s)", LEVEL, StringUtils.join(levels, " OR ")));
+    }
+    return solrQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/44644cd6/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogRequestConverter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogRequestConverter.java b/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogRequestConverter.java
deleted file mode 100644
index 86d055d..0000000
--- a/ambari-logsearch/ambari-logsearch-portal/src/main/java/org/apache/ambari/logsearch/query/converter/ServiceLogRequestConverter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ambari.logsearch.query.converter;
-
-import org.apache.ambari.logsearch.model.request.impl.ServiceLogRequest;
-import org.apache.ambari.logsearch.query.model.ServiceLogSearchCriteria;
-import org.apache.commons.lang.StringEscapeUtils;
-import org.springframework.stereotype.Component;
-
-@Component
-public class ServiceLogRequestConverter extends AbstractCommonServiceLogRequestConverter<ServiceLogRequest, ServiceLogSearchCriteria> {
-
-  @Override
-  public ServiceLogSearchCriteria createCriteria(ServiceLogRequest request) {
-    ServiceLogSearchCriteria criteria = new ServiceLogSearchCriteria();
-    criteria.setKeyword(StringEscapeUtils.unescapeXml(request.getKeyWord()));
-    criteria.setKeywordType(request.getKeywordType());
-    criteria.setSourceLogId(request.getSourceLogId());
-    criteria.setToken(request.getToken());
-    criteria.setLastPage(request.isLastPage());
-    return criteria;
-  }
-}