You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@metron.apache.org by nickwallen <gi...@git.apache.org> on 2017/11/06 19:37:48 UTC

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

GitHub user nickwallen opened a pull request:

    https://github.com/apache/metron/pull/832

    METRON-1301 Sorting on Triage Score Unexpectedly Filters Some Recods

    This PR makes the following changes.
    
    * Adds logic to sort fields that are not present in all of the underlying indices.  For example, threat triage score is only present in the records that have been triaged.  Without this PR, sorting on the threat triage score would unexpectedly filter the search results.
        * When sorted descending, missing fields are sorted last.
        * When sorted ascending, missing fields are sorted first.
    
    * Adds validation and error checking of the response received from Elasticsearch.  Previously there was little to no validation of the response.  This also handles partial shard failures; errors that occur only on a subset of the data.  Without this change some errors that occurred within Elasticsearch were not visible.
    
    * Adds additional logging in the ElasticsearchDAO so that errors can be traced and debugged.
    
    * Refactored the ElasticsearchDAO and extracted some logic into two new interfaces.  This was done so that I could write tests for the new logic.  It was not possible to specifically test this change without refactoring due to limitations with mocking.
        * The column metadata logic is now contained in `ColumnMetadataDao` and `ElasticsearchColumnMetadataDao`.  Without pulling this logic out, it was not possible to test the new sorting logic.
        * The logic for submitting a request to Elasticsearch and validating the response is now contained within `ElasticsearchSearchSubmitter` class.  Without refactoring this it was not possible to test the  new response handling logic.
    
    ## Testing
    
    1. Launch Full Dev
    2. Allow some alerts to accumulate so that when you open the Alerts UI there are a fair number of alerts to work with.
    3. By default it will sort by timestamp.  Take note of the number of records returned.  
    4. Click on the "Score" field to sort the results by threat triage score.
    5. Note the total number of records returned after sorting.  This should be equal to or greater than what you saw before.
    6. The highest scores should be sorted first.  Records that do not contain a threat triage score should be sorted last.
    7. Click on the "Score" field to reverse the sort order.  Records without a threat triage score should now be sorted first.
    
    ## Pull Request Checklist
    
    - [ ] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel). 
    - [ ] Does your PR title start with METRON-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    - [ ] Have you included steps to reproduce the behavior or problem that is being changed or addressed?
    - [ ] Have you included steps or a guide to how the change may be verified and tested manually?
    - [ ] Have you ensured that the full suite of tests and checks have been executed in the root metron folder via:
    - [ ] Have you written or updated unit tests and or integration tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent?
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`:
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/nickwallen/metron METRON-1301

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/metron/pull/832.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #832
    
----
commit de7eba97331442fac293c5c3ca72993f7a431200
Author: Nick Allen <ni...@nickallen.org>
Date:   2017-10-30T16:02:46Z

    METRON-1301 Alerts UI - Sorting on Triage Score Unexpectedly Filters Some Records

----


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on the issue:

    https://github.com/apache/metron/pull/832
  
    I just merged in the last bits from master.  This is ready for review.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149182580
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java ---
    @@ -0,0 +1,192 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.FieldType;
    +import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
    +import org.elasticsearch.client.AdminClient;
    +import org.elasticsearch.cluster.metadata.MappingMetaData;
    +import org.elasticsearch.common.collect.ImmutableOpenMap;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER;
    +
    +/**
    + * Responsible for retrieving column-level metadata for Elasticsearch search indices.
    + */
    +public class ElasticsearchColumnMetadataDao implements ColumnMetadataDao {
    --- End diff --
    
    Extracting this logic into its own implementation class made it possible for me to test the new sort logic.  Just using mocks was not possible due to some of the key methods in the Elasticsearch API being declared final.


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on the issue:

    https://github.com/apache/metron/pull/832
  
    I want to get all of @merrimanr 's fixes in (#824, #825 #827 ) before we consider merging this one.  There will be some merge conflicts and I can take care of sorting through those.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149471327
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java ---
    @@ -0,0 +1,192 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.FieldType;
    +import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
    +import org.elasticsearch.client.AdminClient;
    +import org.elasticsearch.cluster.metadata.MappingMetaData;
    +import org.elasticsearch.common.collect.ImmutableOpenMap;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER;
    +
    +/**
    + * Responsible for retrieving column-level metadata for Elasticsearch search indices.
    + */
    +public class ElasticsearchColumnMetadataDao implements ColumnMetadataDao {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  private static Map<String, FieldType> elasticsearchTypeMap;
    +  static {
    +    Map<String, FieldType> fieldTypeMap = new HashMap<>();
    +    fieldTypeMap.put("string", FieldType.STRING);
    +    fieldTypeMap.put("ip", FieldType.IP);
    +    fieldTypeMap.put("integer", FieldType.INTEGER);
    +    fieldTypeMap.put("long", FieldType.LONG);
    +    fieldTypeMap.put("date", FieldType.DATE);
    +    fieldTypeMap.put("float", FieldType.FLOAT);
    +    fieldTypeMap.put("double", FieldType.DOUBLE);
    +    fieldTypeMap.put("boolean", FieldType.BOOLEAN);
    +    elasticsearchTypeMap = Collections.unmodifiableMap(fieldTypeMap);
    +  }
    +
    +  private transient AdminClient adminClient;
    +  private List<String> ignoredIndices;
    +
    +  public ElasticsearchColumnMetadataDao(AdminClient adminClient, List<String> ignoredIndices) {
    --- End diff --
    
    Done.  See latest.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149411916
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    While I agree with the inclusion of `.kibana` here, what are the consequences for those users with other, non-metron, indices?  Should we allow users to specify this list in the REST application.yml?


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149182933
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -98,78 +121,153 @@ public ElasticsearchDao() {
         //uninitialized.
       }
     
    -  private static Map<String, FieldType> elasticsearchSearchTypeMap;
    -
    -  static {
    -    Map<String, FieldType> fieldTypeMap = new HashMap<>();
    -    fieldTypeMap.put("string", FieldType.STRING);
    -    fieldTypeMap.put("ip", FieldType.IP);
    -    fieldTypeMap.put("integer", FieldType.INTEGER);
    -    fieldTypeMap.put("long", FieldType.LONG);
    -    fieldTypeMap.put("date", FieldType.DATE);
    -    fieldTypeMap.put("float", FieldType.FLOAT);
    -    fieldTypeMap.put("double", FieldType.DOUBLE);
    -    fieldTypeMap.put("boolean", FieldType.BOOLEAN);
    -    elasticsearchSearchTypeMap = Collections.unmodifiableMap(fieldTypeMap);
    -  }
    -
       @Override
       public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
         return search(searchRequest, new QueryStringQueryBuilder(searchRequest.getQuery()));
       }
     
       /**
        * Defers to a provided {@link org.elasticsearch.index.query.QueryBuilder} for the query.
    -   * @param searchRequest The request defining the parameters of the search
    +   * @param request The request defining the parameters of the search
        * @param queryBuilder The actual query to be run. Intended for if the SearchRequest requires wrapping
        * @return The results of the query
        * @throws InvalidSearchException When the query is malformed or the current state doesn't allow search
        */
    -  protected SearchResponse search(SearchRequest searchRequest, QueryBuilder queryBuilder) throws InvalidSearchException {
    +  protected SearchResponse search(SearchRequest request, QueryBuilder queryBuilder) throws InvalidSearchException {
    +    org.elasticsearch.action.search.SearchRequest esRequest;
    +    org.elasticsearch.action.search.SearchResponse esResponse;
    +
         if(client == null) {
           throw new InvalidSearchException("Uninitialized Dao!  You must call init() prior to use.");
         }
    -    if (searchRequest.getSize() > accessConfig.getMaxSearchResults()) {
    +
    +    if (request.getSize() > accessConfig.getMaxSearchResults()) {
           throw new InvalidSearchException("Search result size must be less than " + accessConfig.getMaxSearchResults());
         }
    -    SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder()
    +
    +    esRequest = buildSearchRequest(request, queryBuilder);
    +    esResponse = searchSubmitter.submitSearch(esRequest);
    +    return buildSearchResponse(request, esResponse);
    +  }
    +
    +  /**
    +   * Builds an Elasticsearch search request.
    +   * @param searchRequest The Metron search request.
    +   * @param queryBuilder
    +   * @return An Elasticsearch search request.
    +   */
    +  private org.elasticsearch.action.search.SearchRequest buildSearchRequest(
    +          SearchRequest searchRequest,
    +          QueryBuilder queryBuilder) throws InvalidSearchException {
    +
    +    LOG.debug("Got search request; request={}", ElasticsearchUtils.toJSON(searchRequest));
    +    SearchSourceBuilder searchBuilder = new SearchSourceBuilder()
                 .size(searchRequest.getSize())
                 .from(searchRequest.getFrom())
                 .query(queryBuilder)
                 .trackScores(true);
    -    searchRequest.getSort().forEach(sortField -> searchSourceBuilder.sort(sortField.getField(), getElasticsearchSortOrder(sortField.getSortOrder())));
    -    Optional<List<String>> fields = searchRequest.getFields();
    -    if (fields.isPresent()) {
    -      searchSourceBuilder.fields(fields.get());
    -    } else {
    -      searchSourceBuilder.fetchSource(true);
    +
    +    // column metadata needed to understand the type of each sort field
    +    Map<String, Map<String, FieldType>> meta;
    +    try {
    +      meta = getColumnMetadata(searchRequest.getIndices());
    +    } catch(IOException e) {
    +      throw new InvalidSearchException("Unable to get column metadata", e);
         }
    -    Optional<List<String>> facetFields = searchRequest.getFacetFields();
    -    if (facetFields.isPresent()) {
    -      facetFields.get().forEach(field -> searchSourceBuilder.aggregation(new TermsBuilder(getFacentAggregationName(field)).field(field)));
    +
    +    // handle sort fields
    +    for(SortField sortField : searchRequest.getSort()) {
    +
    +      // what type is the sort field?
    +      FieldType sortFieldType = meta
    +              .values()
    +              .stream()
    +              .filter(e -> e.containsKey(sortField.getField()))
    +              .map(m -> m.get(sortField.getField()))
    +              .findFirst()
    +              .orElse(FieldType.OTHER);
    +
    +      // sort order - if ASC, then missing values sorted last.  Otherwise, missing values sorted first
    +      org.elasticsearch.search.sort.SortOrder sortOrder = getElasticsearchSortOrder(sortField.getSortOrder());
    +      String missingSortOrder;
    +      if(sortOrder == org.elasticsearch.search.sort.SortOrder.DESC) {
    +        missingSortOrder = SORT_MISSING_LAST;
    +      } else {
    +        missingSortOrder = SORT_MISSING_FIRST;
    +      }
    --- End diff --
    
    Here is the additional logic to handle sorting of 'missing' fields.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149184792
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchSubmitter.java ---
    @@ -0,0 +1,138 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.commons.lang3.ArrayUtils;
    +import org.apache.commons.lang3.exception.ExceptionUtils;
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.InvalidSearchException;
    +import org.elasticsearch.action.search.SearchPhaseExecutionException;
    +import org.elasticsearch.action.search.SearchRequest;
    +import org.elasticsearch.action.search.SearchResponse;
    +import org.elasticsearch.action.search.ShardSearchFailure;
    +import org.elasticsearch.client.transport.TransportClient;
    +import org.elasticsearch.rest.RestStatus;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.lang.invoke.MethodHandles;
    +
    +/**
    + * Responsible for submitting searches to Elasticsearch.
    + */
    +public class ElasticsearchSearchSubmitter {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * The Elasticsearch client.
    +   */
    +  private TransportClient client;
    +
    +  public ElasticsearchSearchSubmitter(TransportClient client) {
    +    this.client = client;
    +  }
    +
    +  /**
    +   * Submit a search to Elasticsearch.
    +   * @param request A search request.
    +   * @return The search response.
    +   */
    +  public SearchResponse submitSearch(SearchRequest request) throws InvalidSearchException {
    +    LOG.debug("About to submit a search; request={}", ElasticsearchUtils.toJSON(request));
    +
    +    // submit the search request
    +    org.elasticsearch.action.search.SearchResponse esResponse;
    +    try {
    +      esResponse = client
    +              .search(request)
    +              .actionGet();
    +      LOG.debug("Got Elasticsearch response; response={}", esResponse.toString());
    +
    +    } catch (SearchPhaseExecutionException e) {
    +      String msg = String.format(
    +              "Failed to execute search; error='%s', search='%s'",
    +              ExceptionUtils.getRootCauseMessage(e),
    +              ElasticsearchUtils.toJSON(request));
    +      LOG.error(msg, e);
    +      throw new InvalidSearchException(msg, e);
    +    }
    +
    +    // check for shard failures
    +    if(esResponse.getFailedShards() > 0) {
    --- End diff --
    
    This is the logic that will ensure similar, subtle errors that occur in Elasticsearch will not be masked going forward.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149181570
  
    --- Diff: metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/bro_index.template ---
    @@ -98,7 +98,7 @@
               "mapping": {
                 "type": "float"
               },
    -          "match": "threat.triage.rules:*:score",
    +          "match": "threat:triage:*score",
    --- End diff --
    
    Prior to this change, the overall threat triage score field `threat:triage:score` was not mapped to a `float` as we had assumed.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149197184
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchSubmitter.java ---
    @@ -0,0 +1,138 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.commons.lang3.ArrayUtils;
    +import org.apache.commons.lang3.exception.ExceptionUtils;
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.InvalidSearchException;
    +import org.elasticsearch.action.search.SearchPhaseExecutionException;
    +import org.elasticsearch.action.search.SearchRequest;
    +import org.elasticsearch.action.search.SearchResponse;
    +import org.elasticsearch.action.search.ShardSearchFailure;
    +import org.elasticsearch.client.transport.TransportClient;
    +import org.elasticsearch.rest.RestStatus;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.lang.invoke.MethodHandles;
    +
    +/**
    + * Responsible for submitting searches to Elasticsearch.
    + */
    +public class ElasticsearchSearchSubmitter {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * The Elasticsearch client.
    +   */
    +  private TransportClient client;
    +
    +  public ElasticsearchSearchSubmitter(TransportClient client) {
    +    this.client = client;
    +  }
    +
    +  /**
    +   * Submit a search to Elasticsearch.
    +   * @param request A search request.
    +   * @return The search response.
    +   */
    +  public SearchResponse submitSearch(SearchRequest request) throws InvalidSearchException {
    +    LOG.debug("About to submit a search; request={}", ElasticsearchUtils.toJSON(request));
    +
    +    // submit the search request
    +    org.elasticsearch.action.search.SearchResponse esResponse;
    +    try {
    +      esResponse = client
    +              .search(request)
    +              .actionGet();
    +      LOG.debug("Got Elasticsearch response; response={}", esResponse.toString());
    +
    +    } catch (SearchPhaseExecutionException e) {
    +      String msg = String.format(
    +              "Failed to execute search; error='%s', search='%s'",
    +              ExceptionUtils.getRootCauseMessage(e),
    +              ElasticsearchUtils.toJSON(request));
    +      LOG.error(msg, e);
    +      throw new InvalidSearchException(msg, e);
    +    }
    +
    +    // check for shard failures
    +    if(esResponse.getFailedShards() > 0) {
    +      handleShardFailures(request, esResponse);
    +    }
    +
    +    // validate the response status
    +    if(RestStatus.OK == esResponse.status()) {
    +      return esResponse;
    +
    +    } else {
    +      // the search was not successful
    +      String msg = String.format(
    +              "Bad search response; status=%s, timeout=%s, terminatedEarly=%s",
    +              esResponse.status(), esResponse.isTimedOut(), esResponse.isTerminatedEarly());
    +      LOG.error(msg);
    +      throw new InvalidSearchException(msg);
    +    }
    +  }
    +
    +  /**
    +   * Handle individual shard failures that can occur even when the response is OK.  These
    +   * can indicate misconfiguration of the search indices.
    +   * @param request The search request.
    +   * @param response  The search response.
    +   */
    +  private void handleShardFailures(
    +          org.elasticsearch.action.search.SearchRequest request,
    +          org.elasticsearch.action.search.SearchResponse response) {
    +    /*
    +     * shard failures are only logged.  the search itself is not failed.  this approach
    +     * assumes that a user is interested in partial search results, even if the
    +     * entire search result set cannot be produced.
    +     *
    +     * for example, assume the user adds an additional sensor and the telemetry
    +     * is indexed into a new search index.  if that search index is misconfigured,
    +     * it can result in partial shard failures.  rather than failing the entire search,
    +     * we log the error and allow the results to be returned from shards that
    +     * are correctly configured.
    --- End diff --
    
    Hopefully this explanation makes sense.  When there are partial shard failures, we do not fail fast.  We log the problem and return partial results to the user.  I am torn as to whether this is the right approach.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149413486
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java ---
    @@ -179,4 +186,46 @@ else if(ipObj instanceof List) {
         }
         throw new IllegalStateException("Unable to read the elasticsearch ips, expected es.ip to be either a list of strings, a string hostname or a host:port string");
       }
    +
    +  /**
    +   * Converts an Elasticsearch SearchRequest to JSON.
    +   * @param esRequest The search request.
    +   * @return The JSON representation of the SearchRequest.
    +   */
    +  public static String toJSON(org.elasticsearch.action.search.SearchRequest esRequest) {
    +    String json = "null";
    --- End diff --
    
    Should we instead return back an `Optional<String>` rather than the string representation of null on a failure here?  Alternatively, maybe throw an exception so the calling function can handle the exception how it likes?


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on the issue:

    https://github.com/apache/metron/pull/832
  
    Thanks for all the reviews, guys.  Going to commit now.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149426981
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    Yes, I agree with your feedback.  I was just trying to refactor the column metadata logic and minimize other changes.  So in this case `.kibana` was already hard-coded in ElasticsearchDao.
    
    I would be totally open to making this improvement though.  I was just trying to walk the line of how much should I change when refactoring?  Considering that, what do you think?
    



---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149185308
  
    --- Diff: metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java ---
    @@ -138,6 +137,46 @@
     
       /**
        * {
    +   *  "indices": [
    +   *    "snort",
    +   *    "bro"
    +   *  ],
    +   * "query": "*",
    +   * "from": 0,
    +   * "size": 25,
    +   * "sort": [
    +   *    {
    +   *      "field": "threat:triage:score",
    +   *      "sortOrder": "asc"
    +   *    }
    +   *  ]
    +   * }
    +   */
    +  @Multiline
    +  public static String sortAscendingWithMissingFields;
    --- End diff --
    
    I added two new queries to the integration tests to exercise this defect.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149183693
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -375,79 +523,14 @@ protected UpdateRequest buildUpdateRequest(Document update, String sensorType, S
             .upsert(indexRequest);
       }
     
    -  @SuppressWarnings("unchecked")
       @Override
       public Map<String, Map<String, FieldType>> getColumnMetadata(List<String> indices) throws IOException {
    --- End diff --
    
    I did not remove the `getColumnMetadata` from the `IndexDao` interface.  I simply refer to the implementation provided by the `ColumnMetadataDao` instance.


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on the issue:

    https://github.com/apache/metron/pull/832
  
    +1 worked as advertised.  Thanks @nickwallen!


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by iraghumitra <gi...@git.apache.org>.
Github user iraghumitra commented on the issue:

    https://github.com/apache/metron/pull/832
  
    @nickwallen really liked the way you annotated the PR with your comments. You made it really easy to understand and saved a lot of time for me. As you rightly pointed out I will wait till all other dependent PR's to come in before giving this a  spin.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/metron/pull/832


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149428276
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    Ok, man I have been really bad about reading and comprehension today.  I see in your review comment feedback that you already addressed this point. Sorry.
    
    I think having this hardcoded is just not good.  The change seems simple enough.  I can probably take care of this no problem.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149471276
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java ---
    @@ -179,4 +186,46 @@ else if(ipObj instanceof List) {
         }
         throw new IllegalStateException("Unable to read the elasticsearch ips, expected es.ip to be either a list of strings, a string hostname or a host:port string");
       }
    +
    +  /**
    +   * Converts an Elasticsearch SearchRequest to JSON.
    +   * @param esRequest The search request.
    +   * @return The JSON representation of the SearchRequest.
    +   */
    +  public static String toJSON(org.elasticsearch.action.search.SearchRequest esRequest) {
    +    String json = "null";
    --- End diff --
    
    Yes, definitely this is kind of weird.  This is taken care of in the latest commit.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149409051
  
    --- Diff: metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/bro_index.template ---
    @@ -98,7 +98,7 @@
               "mapping": {
                 "type": "float"
               },
    -          "match": "threat.triage.rules:*:score",
    +          "match": "threat:triage:*score",
    --- End diff --
    
    great catch.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149410922
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java ---
    @@ -0,0 +1,192 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.FieldType;
    +import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
    +import org.elasticsearch.client.AdminClient;
    +import org.elasticsearch.cluster.metadata.MappingMetaData;
    +import org.elasticsearch.common.collect.ImmutableOpenMap;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER;
    +
    +/**
    + * Responsible for retrieving column-level metadata for Elasticsearch search indices.
    + */
    +public class ElasticsearchColumnMetadataDao implements ColumnMetadataDao {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  private static Map<String, FieldType> elasticsearchTypeMap;
    +  static {
    +    Map<String, FieldType> fieldTypeMap = new HashMap<>();
    +    fieldTypeMap.put("string", FieldType.STRING);
    +    fieldTypeMap.put("ip", FieldType.IP);
    +    fieldTypeMap.put("integer", FieldType.INTEGER);
    +    fieldTypeMap.put("long", FieldType.LONG);
    +    fieldTypeMap.put("date", FieldType.DATE);
    +    fieldTypeMap.put("float", FieldType.FLOAT);
    +    fieldTypeMap.put("double", FieldType.DOUBLE);
    +    fieldTypeMap.put("boolean", FieldType.BOOLEAN);
    +    elasticsearchTypeMap = Collections.unmodifiableMap(fieldTypeMap);
    +  }
    +
    +  private transient AdminClient adminClient;
    +  private List<String> ignoredIndices;
    +
    +  public ElasticsearchColumnMetadataDao(AdminClient adminClient, List<String> ignoredIndices) {
    --- End diff --
    
    Small nit, can we make `ignoredIndices` a `Set`


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149429061
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    Given that it's a pretty simple change and requires no new machinery to added, I think it is probably a good fit to add to this PR, but I might be on the wrong side of that line more than I'm on the right side, so take my word with a grain of salt. ;)


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on the issue:

    https://github.com/apache/metron/pull/832
  
    +1 by inspection.  Looks great!


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149428796
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    I'm almost always in favor of having hard coded things passed in via config files.  It gets us out of jams and almost every time I've convinced myself that it's not necessary, it totally is.  How convinced are you that we are only ever going to need one ignored index?
    
    If you have doubts, then I'd probably add something to the `application.yml` which gets set in the AccessConfig object when we set up the index in the IndexConfig from `metron-rest`.


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on the issue:

    https://github.com/apache/metron/pull/832
  
    Spun this up on full dev, and saw that sorting worked as expected in the UI.  This worked both when no groupings were selected and also when drilling down (e.g. grouping by ip_dst_addr and country).
    
    I also like the refactorings a lot.  It's good stuff that makes life easier and more testable.
    
    +1, assuming @cestella is happy with the changes you made on his comments.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149501925
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java ---
    @@ -234,26 +366,43 @@ public synchronized void init(AccessConfig config) {
         if(this.client == null) {
           this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings());
           this.accessConfig = config;
    +      this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin(), Collections.singletonList(".kibana"));
    --- End diff --
    
    The latest has the change that makes the ignore indices configurable.


---

[GitHub] metron pull request #832: METRON-1301 Sorting on Triage Score Unexpectedly F...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/832#discussion_r149184523
  
    --- Diff: metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchSubmitter.java ---
    @@ -0,0 +1,138 @@
    +/**
    + * 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.metron.elasticsearch.dao;
    +
    +import org.apache.commons.lang3.ArrayUtils;
    +import org.apache.commons.lang3.exception.ExceptionUtils;
    +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
    +import org.apache.metron.indexing.dao.search.InvalidSearchException;
    +import org.elasticsearch.action.search.SearchPhaseExecutionException;
    +import org.elasticsearch.action.search.SearchRequest;
    +import org.elasticsearch.action.search.SearchResponse;
    +import org.elasticsearch.action.search.ShardSearchFailure;
    +import org.elasticsearch.client.transport.TransportClient;
    +import org.elasticsearch.rest.RestStatus;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.lang.invoke.MethodHandles;
    +
    +/**
    + * Responsible for submitting searches to Elasticsearch.
    + */
    +public class ElasticsearchSearchSubmitter {
    --- End diff --
    
    This class contains all the logic for submitting requests, along with validation and error checking of the Elasticsearch responses.  Refactoring this out allows me to easily test the new logic.  This also makes sure that the same error checking logic is applied to both `/search` and `/group` requests.


---

[GitHub] metron issue #832: METRON-1301 Sorting on Triage Score Unexpectedly Filters ...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on the issue:

    https://github.com/apache/metron/pull/832
  
    I merged with the latest changed from #824 


---