You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by GitBox <gi...@apache.org> on 2018/11/12 13:45:15 UTC

[GitHub] kasakrisz closed pull request #18: AMBARI-24847 - Logsearch: Cannot search for a term that includes spaces or dashes

kasakrisz closed pull request #18: AMBARI-24847 - Logsearch: Cannot search for a term that includes spaces or dashes
URL: https://github.com/apache/ambari-logsearch/pull/18
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/ambari-logsearch-it/pom.xml b/ambari-logsearch-it/pom.xml
index eda11d82d6..889525e190 100644
--- a/ambari-logsearch-it/pom.xml
+++ b/ambari-logsearch-it/pom.xml
@@ -59,6 +59,16 @@
       <artifactId>solr-solrj</artifactId>
       <version>${solr.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.solr</groupId>
+      <artifactId>solr-core</artifactId>
+      <version>${solr.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.solr</groupId>
+      <artifactId>solr-dataimporthandler</artifactId>
+      <version>${solr.version}</version>
+    </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
diff --git a/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/Solr.java b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/Solr.java
new file mode 100644
index 0000000000..ee0bc8c4ef
--- /dev/null
+++ b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/Solr.java
@@ -0,0 +1,115 @@
+/*
+ * 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.solr;
+
+import static org.apache.ambari.logsearch.solr.SolrConstants.AuditLogConstants.AUDIT_EVTTIME;
+import static org.apache.ambari.logsearch.solr.SolrConstants.CommonLogConstants.ID;
+import static org.apache.ambari.logsearch.solr.SolrConstants.CommonLogConstants.LOG_MESSAGE;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LEVEL;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOGTIME;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.response.PivotField;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.SolrResourceLoader;
+
+public class Solr {
+  public static final DateTimeFormatter SOLR_DATETIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX");
+
+  public static Solr core(String coreName) throws IOException, SolrServerException {
+    assert coreName != null;
+
+    String targetLocation = Solr.class.getProtectionDomain().getCodeSource().getLocation().getFile() + "..";
+    String logsearchConfigSetDir = targetLocation + "/../../ambari-logsearch-server/src/main/configsets";
+    File targetConfigSetDir = new File(targetLocation + "/configsets");
+    if (targetConfigSetDir.exists())
+      FileUtils.deleteDirectory(targetConfigSetDir);
+    FileUtils.copyDirectory(new File(logsearchConfigSetDir), targetConfigSetDir);
+    String solrHome = targetLocation + "/solr";
+    File solrHomeDir = new File(solrHome);
+    if (solrHomeDir.exists())
+      FileUtils.deleteDirectory(solrHomeDir);
+    solrHomeDir.mkdirs();
+
+    SolrResourceLoader solrResourceLoader = new SolrResourceLoader(solrHomeDir.toPath());
+
+    NodeConfig config = new NodeConfig.NodeConfigBuilder("embeddedSolrServerNode", solrResourceLoader)
+            .setConfigSetBaseDirectory(targetConfigSetDir.getAbsolutePath())
+            .build();
+
+    EmbeddedSolrServer embeddedSolrServer = new EmbeddedSolrServer(config, coreName);
+
+    CoreAdminRequest.Create createRequest = new CoreAdminRequest.Create();
+    createRequest.setCoreName(coreName);
+    createRequest.setConfigSet(coreName);
+    embeddedSolrServer.request(createRequest);
+
+    return new Solr(embeddedSolrServer);
+  }
+
+  private final EmbeddedSolrServer server;
+
+  private Solr(EmbeddedSolrServer solrServer) {
+    server = solrServer;
+  }
+
+  public void addDoc(String id, String logMessage) throws SolrServerException, IOException {
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.addField(ID, id);
+    doc.addField(LOGTIME, new Date(OffsetDateTime.now().toInstant().toEpochMilli()));
+    doc.addField(AUDIT_EVTTIME, new Date(OffsetDateTime.now().toInstant().toEpochMilli()));
+    doc.addField(LOG_MESSAGE, logMessage);
+    doc.addField(LEVEL, "INFO");
+    server.add(doc);
+    server.commit();
+  }
+
+  public SolrDocumentList executeQuery(SolrQuery solrQuery) throws SolrServerException, IOException {
+    return server.query(solrQuery).getResults();
+  }
+
+  public NamedList<List<PivotField>> executeFacetQuery(SolrQuery solrQuery) throws SolrServerException, IOException {
+    QueryResponse qResp = server.query(solrQuery);
+    return qResp.getFacetPivot();
+  }
+
+  public void clear() throws Exception {
+    server.deleteByQuery("*:*");
+    server.commit();
+  }
+
+  public void close() throws IOException {
+    server.close();
+  }
+}
diff --git a/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/SolrDocumentMatcher.java b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/SolrDocumentMatcher.java
new file mode 100644
index 0000000000..540e117c54
--- /dev/null
+++ b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/SolrDocumentMatcher.java
@@ -0,0 +1,60 @@
+/*
+ * 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.solr;
+
+import org.apache.solr.common.SolrDocument;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeDiagnosingMatcher;
+
+public class SolrDocumentMatcher extends TypeSafeDiagnosingMatcher<SolrDocument> {
+
+  public static SolrDocumentMatcher solrDoc(String expectedId) {
+    return new SolrDocumentMatcher(expectedId);
+  }
+
+  private final String expectedId;
+
+  private SolrDocumentMatcher(String expectedId) {
+    this.expectedId = expectedId;
+  }
+
+  @Override
+  protected boolean matchesSafely(SolrDocument item, Description mismatchDescription) {
+    if (!item.containsKey("id")) {
+      mismatchDescription.appendText("SolrDocument without 'id' field.");
+      return false;
+    }
+    if (!expectedId.equals(item.get("id"))) {
+      describe(mismatchDescription, item.get("id"));
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void describeTo(Description description) {
+    describe(description, expectedId);
+  }
+
+  private void describe(Description description, Object id) {
+    description.appendText("SolrDocument[id=");
+    description.appendValue(id);
+    description.appendText("]");
+  }
+}
diff --git a/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/search/SearchServiceLogsTest.java b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/search/SearchServiceLogsTest.java
new file mode 100644
index 0000000000..b7766c91f5
--- /dev/null
+++ b/ambari-logsearch-it/src/test/java/org/apache/ambari/logsearch/solr/search/SearchServiceLogsTest.java
@@ -0,0 +1,376 @@
+/*
+ * 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.solr.search;
+
+import static org.apache.ambari.logsearch.solr.Solr.SOLR_DATETIME_FORMATTER;
+import static org.apache.ambari.logsearch.solr.SolrDocumentMatcher.solrDoc;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.core.Is.is;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.ambari.logsearch.converter.BaseServiceLogRequestQueryConverter;
+import org.apache.ambari.logsearch.converter.ServiceLogLevelDateRangeRequestQueryConverter;
+import org.apache.ambari.logsearch.model.request.impl.BaseServiceLogRequest;
+import org.apache.ambari.logsearch.model.request.impl.ServiceGraphRequest;
+import org.apache.ambari.logsearch.model.request.impl.query.BaseServiceLogQueryRequest;
+import org.apache.ambari.logsearch.model.request.impl.query.ServiceGraphQueryRequest;
+import org.apache.ambari.logsearch.solr.Solr;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.PivotField;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.util.NamedList;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.data.solr.core.DefaultQueryParser;
+import org.springframework.data.solr.core.query.SimpleQuery;
+
+public class SearchServiceLogsTest {
+  private static Solr solr;
+
+  private final BaseServiceLogRequestQueryConverter requestQueryConverter = new BaseServiceLogRequestQueryConverter();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    solr = Solr.core("hadoop_logs");
+  }
+
+  @AfterClass
+  public static void stopSolr() throws Exception {
+    solr.close();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    solr.clear();
+  }
+
+  @Test
+  public void testIncludeLogMessageFilter() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"exception\"}]");
+    ArrayList<SolrDocument> found = executeQuery(request);
+
+    assertThat(found, hasSize(3));
+    assertThat(found, hasItem(solrDoc("0")));
+    assertThat(found, hasItem(solrDoc("1")));
+    assertThat(found, hasItem(solrDoc("2")));
+  }
+
+  @Test
+  public void testIncludeBlankLogMessageFilter() throws Exception {
+    solr.addDoc("0", "");
+    solr.addDoc("1", "exception occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"\\\"\"}]");
+    ArrayList<SolrDocument> found = executeQuery(request);
+
+    assertThat(found, hasSize(0));
+  }
+
+  @Test
+  public void testIncludeQuotedSpaceLogMessageFilter() throws Exception {
+    solr.addDoc("0", " ");
+    solr.addDoc("1", "exception occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\" \\\"\"}]");
+    ArrayList<SolrDocument> found = executeQuery(request);
+
+    assertThat(found, hasSize(0));
+  }
+
+  @Test
+  public void testIncludeSpaceLogMessageFilter() throws Exception {
+    solr.addDoc("0", " ");
+    solr.addDoc("1", "exception occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\" \"}]");
+    ArrayList<SolrDocument> found = executeQuery(request);
+
+    assertThat(found, hasSize(2));
+  }
+
+  private SolrDocumentList executeQuery(BaseServiceLogRequest request) throws IOException, SolrServerException {
+    SimpleQuery simpleQuery = requestQueryConverter.convert(request);
+    assert simpleQuery != null;
+    return solr.executeQuery(new DefaultQueryParser().doConstructSolrQuery(simpleQuery));
+  }
+
+
+  @Test
+  public void testIncludeMultipleLogMessageFilterContainsWildcard() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"exceptio*\"},{\"log_message\":\"unha*\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("0")));
+  }
+
+  @Test
+  public void testIncludeMultipleTerms() throws Exception {
+    solr.addDoc("0", "At line 112 an unhandled exception occurred when");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"Unhandled exception\\\"\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("0")));
+  }
+
+  @Test
+  public void testIncludeMultipleTermsDelimitedByDash() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "At line 112 an unhandled-exception occurred when");
+    solr.addDoc("2", "exception occurred");
+    solr.addDoc("3", "exception");
+    solr.addDoc("4", "Unhandled error");
+    solr.addDoc("5", "Error occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"Unhandled-exception\\\"\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("1")));
+  }
+
+  @Test
+  public void testIncludeMultipleTermsDelimitedByPlusSign() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "Unhandled+exception");
+    solr.addDoc("2", "Unhandled-exception");
+    solr.addDoc("3", "exception occurred");
+    solr.addDoc("4", "exception");
+    solr.addDoc("5", "Unhandled error");
+    solr.addDoc("6", "Error occurred");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"Unhandled+exception\\\"\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("1")));
+  }
+
+  @Test
+  public void testIncludeALongMessage() throws Exception {
+    solr.addDoc("6", "[   ] org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:400) - Error in trigger 'index_size_trigger' configuration, trigger config ignored: {\\r\\n  \\\"aboveBytes\\\":53687091200,\\r\\n  \\\"aboveOp\\\":\\\"SPLITSHARD\\\",\\r\\n  \\\"event\\\":\\\"indexSize\\\",\\r\\n  \\\"waitFor\\\":1,\\r\\n  \\\"actions\\\":[\\r\\n    {\\r\\n      \\\"name\\\":\\\"compute_plan\\\",\\r\\n      \\\"class\\\":\\\"solr.ComputePlanAction\\\"},\\r\\n    {\\r\\n      \\\"name\\\":\\\"execute_plan\\\",\\r\\n      \\\"class\\\":\\\"solr.ExecutePlanAction\\\"}],\\r\\n  \\\"enabled\\\":true}\\r\\norg.apache.solr.cloud.autoscaling.TriggerValidationException: null\\r\\n\\tat org.apache.solr.cloud.autoscaling.TriggerBase.configure(TriggerBase.java:118) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.IndexSizeTrigger.configure(IndexSizeTrigger.java:87) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.AutoScaling$TriggerFactoryImpl.create(AutoScaling.java:189) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.loadTriggers(OverseerTriggerThread.java:398) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.refreshAutoScalingConf(OverseerTriggerThread.java:335) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.run(OverseerTriggerThread.java:161) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat java.lang.Thread.run(Thread.java:745) [?:1.8.0_112]");
+    solr.addDoc("7", "[   ] org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:422) - Something else");
+    solr.addDoc("8", "[   ] org.apache.solr.cloud.OverseerTriggerThread (OverseerTriggerThread.java:400) - Different package");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:400)\\\"\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("6")));
+  }
+
+  @Test
+  public void testIncludeMultipleWhitespaceDelimitedWords() throws Exception {
+    solr.addDoc("0", "At line 112 an unhandled exception occurred when");
+    solr.addDoc("1", "At line 112 an unhandled  exception occurred when");
+    solr.addDoc("2", "At line 112 an unhandled   \texception occurred when");
+    solr.addDoc("3", "At line 112 an exception occurred when");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"\\\"Unhandled  exception\\\"\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(3));
+    assertThat(found, hasItem(solrDoc("0")));
+    assertThat(found, hasItem(solrDoc("1")));
+    assertThat(found, hasItem(solrDoc("2")));
+  }
+
+  @Test
+  public void testIncludeEMailAddress() throws Exception {
+    solr.addDoc("0", "Email address: john@hortonworks.com");
+    solr.addDoc("1", "Another document");
+
+    BaseServiceLogRequest request = new BaseServiceLogQueryRequest();
+    request.setIncludeQuery("[{\"log_message\":\"john@hortonworks.com\"}]");
+    SolrDocumentList found = executeQuery(request);
+
+    assertThat(found, hasSize(1));
+    assertThat(found, hasItem(solrDoc("0")));
+  }
+
+
+  private final ServiceLogLevelDateRangeRequestQueryConverter dateRangeRequestQueryConverter = new ServiceLogLevelDateRangeRequestQueryConverter();
+
+  @Test
+  public void testDateRangeRequestIncludeEMailAddress() throws Exception {
+    solr.addDoc("0", "Email address: john@hortonworks.com");
+    solr.addDoc("1", "Another document doe@hortonworks.com");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"john@hortonworks.com\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  private ServiceGraphRequest serviceGraphRequest(String includeQuery) {
+    ServiceGraphRequest request = new ServiceGraphQueryRequest();
+    request.setFrom(SOLR_DATETIME_FORMATTER.format(OffsetDateTime.now(ZoneId.of("UTC")).minusDays(1)));
+    request.setTo(SOLR_DATETIME_FORMATTER.format(OffsetDateTime.now(ZoneId.of("UTC")).plusDays(1)));
+    request.setIncludeQuery(includeQuery);
+    return request;
+  }
+
+  private NamedList<List<PivotField>> executeQuery(ServiceGraphRequest request) throws IOException, SolrServerException {
+    return solr.executeFacetQuery(dateRangeRequestQueryConverter.convert(request));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeLogMessageFilter() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"exception\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(3));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeMultipleLogMessageFilterContainsWildcard() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"exceptio*\"},{\"log_message\":\"unha*\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeMultipleTerms() throws Exception {
+    solr.addDoc("0", "At line 112 an unhandled exception occurred when");
+    solr.addDoc("1", "exception occurred");
+    solr.addDoc("2", "exception");
+    solr.addDoc("3", "Unhandled error");
+    solr.addDoc("4", "Error occurred");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"\\\"Unhandled exception\\\"\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeMultipleTermsDelimitedByDash() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "At line 112 an unhandled-exception occurred when");
+    solr.addDoc("2", "exception occurred");
+    solr.addDoc("3", "exception");
+    solr.addDoc("4", "Unhandled error");
+    solr.addDoc("5", "Error occurred");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"\\\"Unhandled-exception\\\"\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeMultipleTermsDelimitedByPlusSign() throws Exception {
+    solr.addDoc("0", "Unhandled exception");
+    solr.addDoc("1", "Unhandled+exception");
+    solr.addDoc("2", "Unhandled-exception");
+    solr.addDoc("3", "exception occurred");
+    solr.addDoc("4", "exception");
+    solr.addDoc("5", "Unhandled error");
+    solr.addDoc("6", "Error occurred");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"\\\"Unhandled+exception\\\"\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeALongMessage() throws Exception {
+    solr.addDoc("6", "[   ] org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:400) - Error in trigger 'index_size_trigger' configuration, trigger config ignored: {\\r\\n  \\\"aboveBytes\\\":53687091200,\\r\\n  \\\"aboveOp\\\":\\\"SPLITSHARD\\\",\\r\\n  \\\"event\\\":\\\"indexSize\\\",\\r\\n  \\\"waitFor\\\":1,\\r\\n  \\\"actions\\\":[\\r\\n    {\\r\\n      \\\"name\\\":\\\"compute_plan\\\",\\r\\n      \\\"class\\\":\\\"solr.ComputePlanAction\\\"},\\r\\n    {\\r\\n      \\\"name\\\":\\\"execute_plan\\\",\\r\\n      \\\"class\\\":\\\"solr.ExecutePlanAction\\\"}],\\r\\n  \\\"enabled\\\":true}\\r\\norg.apache.solr.cloud.autoscaling.TriggerValidationException: null\\r\\n\\tat org.apache.solr.cloud.autoscaling.TriggerBase.configure(TriggerBase.java:118) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.IndexSizeTrigger.configure(IndexSizeTrigger.java:87) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.AutoScaling$TriggerFactoryImpl.create(AutoScaling.java:189) ~[solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.loadTriggers(OverseerTriggerThread.java:398) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.refreshAutoScalingConf(OverseerTriggerThread.java:335) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat org.apache.solr.cloud.autoscaling.OverseerTriggerThread.run(OverseerTriggerThread.java:161) [solr-core-7.5.0.jar:7.5.0 b5bf70b7e32d7ddd9742cc821d471c5fabd4e3df - jimczi - 2018-09-18 13:07:55]\\r\\n\\tat java.lang.Thread.run(Thread.java:745) [?:1.8.0_112]");
+    solr.addDoc("7", "[   ] org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:422) - Something else");
+    solr.addDoc("8", "[   ] org.apache.solr.cloud.OverseerTriggerThread (OverseerTriggerThread.java:400) - Different package");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"\\\"org.apache.solr.cloud.autoscaling.OverseerTriggerThread (OverseerTriggerThread.java:400)\\\"\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(1));
+  }
+
+  @Test
+  public void testDateRangeRequestIncludeMultipleWhitespaceDelimitedWords() throws Exception {
+    solr.addDoc("0", "At line 112 an unhandled exception occurred when");
+    solr.addDoc("1", "At line 112 an unhandled  exception occurred when");
+    solr.addDoc("2", "At line 112 an unhandled   \texception occurred when");
+    solr.addDoc("3", "At line 112 an exception occurred when");
+
+    ServiceGraphRequest request = serviceGraphRequest("[{\"log_message\":\"\\\"Unhandled  exception\\\"\"}]");
+    NamedList<List<PivotField>> found = executeQuery(request);
+
+    assertThat(found.size(), is(1));
+    assertThat(found.get("level").get(0).getCount(), is(3));
+  }
+}
diff --git a/ambari-logsearch-server/src/main/configsets/audit_logs/conf/managed-schema b/ambari-logsearch-server/src/main/configsets/audit_logs/conf/managed-schema
index fd90adab26..689691887a 100644
--- a/ambari-logsearch-server/src/main/configsets/audit_logs/conf/managed-schema
+++ b/ambari-logsearch-server/src/main/configsets/audit_logs/conf/managed-schema
@@ -91,7 +91,7 @@
   <field name="ip" type="key_lower_case"/>
   <field name="level" type="key_lower_case"/>
   <field name="logType" type="key_lower_case" multiValued="false"/>
-  <field name="log_message" type="key_lower_case" multiValued="false" omitNorms="false"/>
+  <field name="log_message" type="text_ws" multiValued="false" omitNorms="false"/>
   <field name="logfile_line_number" type="tint" omitNorms="false"/>
   <field name="logger_name" type="key_lower_case"/>
   <field name="message_md5" type="string" multiValued="false"/>
diff --git a/ambari-logsearch-server/src/main/configsets/hadoop_logs/conf/managed-schema b/ambari-logsearch-server/src/main/configsets/hadoop_logs/conf/managed-schema
index b1fda991cf..168427eb75 100644
--- a/ambari-logsearch-server/src/main/configsets/hadoop_logs/conf/managed-schema
+++ b/ambari-logsearch-server/src/main/configsets/hadoop_logs/conf/managed-schema
@@ -96,7 +96,7 @@
   <field name="ip" type="string" multiValued="false"/>
   <field name="level" type="lowercase" multiValued="false"/>
   <field name="line_number" type="tint" omitNorms="false"/>
-  <field name="log_message" type="text_general" multiValued="false" omitNorms="false"/>
+  <field name="log_message" type="text_ws" multiValued="false" omitNorms="false"/>
   <field name="logfile_line_number" type="tint" omitNorms="false"/>
   <field name="logger_name" type="string" multiValued="false"/>
   <field name="logtime" type="tdate" multiValued="false"  docValues="true"/>
diff --git a/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractDateRangeFacetQueryConverter.java b/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractDateRangeFacetQueryConverter.java
index 2143f56022..02a58725fc 100644
--- a/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractDateRangeFacetQueryConverter.java
+++ b/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractDateRangeFacetQueryConverter.java
@@ -18,14 +18,14 @@
  */
 package org.apache.ambari.logsearch.converter;
 
+import java.util.Locale;
+
 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 AbstractOperationHolderConverter<SOURCE , SolrQuery> {
 
@@ -40,9 +40,9 @@ public SolrQuery convert(SOURCE request) {
     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.add(String.format(Locale.ROOT, "f.%s.%s", getDateFieldName(), "facet.range.start"), request.getFrom());
+    solrQuery.add(String.format(Locale.ROOT, "f.%s.%s", getDateFieldName(), "facet.range.end"), request.getTo());
+    solrQuery.add(String.format(Locale.ROOT, "f.%s.%s", getDateFieldName(), "facet.range.gap"), unit);
     solrQuery.remove("sort");
     solrQuery.setRows(0);
     solrQuery.setStart(0);
diff --git a/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractOperationHolderConverter.java b/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractOperationHolderConverter.java
index 5e7f440d38..54ae154c37 100644
--- a/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractOperationHolderConverter.java
+++ b/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/converter/AbstractOperationHolderConverter.java
@@ -18,28 +18,30 @@
  */
 package org.apache.ambari.logsearch.converter;
 
-import com.google.common.base.Splitter;
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
+import static java.util.Collections.singletonList;
+import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOG_MESSAGE;
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
+
 import org.apache.ambari.logsearch.common.LogType;
 import org.apache.ambari.logsearch.dao.SolrSchemaFieldDao;
 import org.apache.ambari.logsearch.util.SolrUtil;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.text.StrTokenizer;
 import org.apache.solr.client.solrj.SolrQuery;
 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 java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.ambari.logsearch.solr.SolrConstants.ServiceLogConstants.LOG_MESSAGE;
+import com.google.common.base.Splitter;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
 
 public abstract class AbstractOperationHolderConverter <REQUEST_TYPE, QUERY_TYPE>
   extends AbstractConverterAware<REQUEST_TYPE, QUERY_TYPE> {
@@ -79,7 +81,7 @@ public Query addInFilterQuery(Query query, String field, List<String> values) {
 
   public Query addInFiltersIfNotNullAndEnabled(Query query, String value, String field, boolean condition) {
     if (value != null && condition) {
-      List<String> values = value.length() == 0 ? Arrays.asList("-1") : splitValueAsList(value, ",");
+      List<String> values = value.length() == 0 ? singletonList("-1") : splitValueAsList(value, ",");
       addInFilterQuery(query, field, values);
     }
     return query;
@@ -87,7 +89,7 @@ public Query addInFiltersIfNotNullAndEnabled(Query query, String value, String f
 
   public SolrQuery addInFiltersIfNotNullAndEnabled(SolrQuery query, String value, String field, boolean condition) {
     if (condition) {
-      List<String> valuesList = value.length() == 0 ? Arrays.asList("\\-1") : splitValueAsList(value, ",");
+      List<String> valuesList = value.length() == 0 ? singletonList("\\-1") : splitValueAsList(value, ",");
       if (valuesList.size() > 1) {
         query.addFilterQuery(String.format("%s:(%s)", field, StringUtils.join(valuesList, " OR ")));
       } else {
@@ -198,41 +200,25 @@ public SolrQuery addListFilterToSolrQuery(SolrQuery solrQuery, String fieldName,
   public abstract LogType getLogType();
 
   private void addLogMessageFilter(Query query, String value, boolean negate) {
-    StrTokenizer tokenizer = new StrTokenizer(value, ' ', '"');
-    for (String token : tokenizer.getTokenArray()) {
-      token = token.trim();
-      if (token.contains(" ") || !token.startsWith("*") && !token.endsWith("*")) {
-        addFilterQuery(query, new Criteria(LOG_MESSAGE).is(SolrUtil.escapeQueryChars(token)), negate);
-      } else if (token.startsWith("*") && token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 1, -1);
-        addFilterQuery(query, new Criteria(LOG_MESSAGE).contains(SolrUtil.escapeQueryChars(plainToken)), negate);
-      } else if (token.startsWith("*") && !token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 1);
-        addFilterQuery(query, new Criteria(LOG_MESSAGE).endsWith(SolrUtil.escapeQueryChars(plainToken)), negate);
-      } else if (!token.startsWith("*") && token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 0, -1);
-        addFilterQuery(query, new Criteria(LOG_MESSAGE).startsWith(SolrUtil.escapeQueryChars(plainToken)), negate);
-      }
+    value = value.trim();
+    if (StringUtils.startsWith(value, "\"") && StringUtils.endsWith(value,"\"")) {
+      value = String.format("\"%s\"", SolrUtil.escapeQueryChars(StringUtils.substring(value, 1, -1)));
+      addFilterQuery(query, new Criteria(LOG_MESSAGE).expression(value), negate);
+    }
+    else if (isNotBlank(value)){
+      addFilterQuery(query, new Criteria(LOG_MESSAGE).expression(SolrUtil.escapeQueryChars(value)), negate);
     }
   }
 
   private void addLogMessageFilter(SolrQuery query, String value, boolean negate) {
-    StrTokenizer tokenizer = new StrTokenizer(value, ' ', '"');
     String negateToken = negate ? "-" : "";
-    for (String token : tokenizer.getTokenArray()) {
-      token = token.trim();
-      if (token.contains(" ") || !token.startsWith("*") && !token.endsWith("*")) {
-        query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, SolrUtil.escapeQueryChars(token)));
-      } else if (token.startsWith("*") && token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 1, -1);
-        query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, SolrUtil.escapeQueryChars(plainToken)));
-      } else if (token.startsWith("*") && !token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 1);
-        query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, SolrUtil.escapeQueryChars(plainToken)));
-      } else if (!token.startsWith("*") && token.endsWith("*")) {
-        String plainToken = StringUtils.substring(token, 0, -1);
-        query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, SolrUtil.escapeQueryChars(plainToken)));
-      }
+    value = value.trim();
+    if (StringUtils.startsWith(value, "\"") && StringUtils.endsWith(value,"\"")) {
+      value = String.format("\"%s\"", SolrUtil.escapeQueryChars(StringUtils.substring(value, 1, -1)));
+      query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, value));
+    }
+    else {
+      query.addFilterQuery(String.format("%s%s:%s", negateToken, LOG_MESSAGE, SolrUtil.escapeQueryChars(value)));
     }
   }
 
diff --git a/ambari-logsearch-web/src/app/services/logs-filtering-utils.service.ts b/ambari-logsearch-web/src/app/services/logs-filtering-utils.service.ts
index 89f8a6db2d..9fddade6db 100644
--- a/ambari-logsearch-web/src/app/services/logs-filtering-utils.service.ts
+++ b/ambari-logsearch-web/src/app/services/logs-filtering-utils.service.ts
@@ -355,7 +355,7 @@ export class LogsFilteringUtilsService {
           return item.isExclude === isExclude;
         }).map((parameter: SearchBoxParameter): HomogeneousObject<string> => {
           return {
-            [parameter.name]: parameter.value.replace(/\s/g, '+')
+            [parameter.name]: parameter.value
           };
         });
       }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services