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/02/03 19:39:21 UTC

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

GitHub user nickwallen opened a pull request:

    https://github.com/apache/incubator-metron/pull/438

    METRON-686 Record Rule Set that Fired During Threat Triage

    The goal of threat triage is to prioritize the alerts that pose the greatest threat and thus need urgent attention.  To perform threat triage, a set of rules are applied to each message.  Each rule has a predicate to determine if the rule applies or not.  The threat score from each applied rule is aggregated into a single threat triage score that can be used to prioritize high risk threats.
    
    ### Changes
    * All of the threat triage rules that were applied to a message during threat triage are captured and recorded.
    * A new 'reason' field was added to the threat triage rules to allow a rule author to provide contextual information to assist in actioning a threat. 
    
    ### Implementation
    The `ThreatTriageProcessor` attempts to apply each `RiskLevelRule` to a message being triaged.  If the rule's predicate returns true, the rule is applied to the message resulting in a `RuleScore`.  The `RuleScore` represents the portion of the threat score contributed by a single rule.  Each of the `RuleScore`s are aggregated into an overall `ThreatScore`, which captures the outcome of the threat triage process.  This `ThreatScore` is then attached to the message.
    
    ### Example
    
    Previously the result of threat triage was a single threat score represented as a double.  
    ```
    "threat.triage.level": 10.0
    ```
    
    The result of threat triage now provides greater information about the triage process.  The rule set...
    ```
        "triageConfig" : {
          "riskLevelRules" : [ {
            "name" : "Abnormal DNS Port",
            "rule" : "source.type == \"bro\" and protocol == \"dns\" and ip_dst_port != 53",
            "score" : 10.0,
            "reason" : "FORMAT(\"Abnormal DNS Port: expected: 53, found: %s:%d\", ip_dst_addr, ip_dst_port)"
          } ],
          "aggregator" : "MAX",
          "aggregationConfig" : { }
        }
    ```
    
    produces a result like...
    ```
    "threat.triage.level":{
       "score":10.0,
       "rules":[
          { 
             "name":"Abnormal DNS Port",
             "comment":null
             "score":10.0,
             "reason":"Abnormal DNS Port: expected: 53, found: 224.0.0.251:5353",
          }
       ]
    }
    ```
    
    ### Testing
    Setup a Metron environment with the default Bro sensor running.  The following commands can be executed in the Stellar REPL.
    
    ```
    # fetch the current config
    conf := CONFIG_GET("ENRICHMENT", "bro")
    
    # triage all bro telemetry
    is_alert := source.type == 'bro'
    conf := ENRICHMENT_STELLAR_TRANSFORM_ADD(conf, 'ENRICHMENT', SHELL_VARS2MAP('is_alert'))
    
    # create a rule that looks for DNS running on non-standard ports
    rule := source.type == "bro" and protocol == "dns" and ip_dst_port != 53
    triage := { "name":"Abnormal DNS Port", "rule": SHELL_GET_EXPRESSION('rule'), "reason": SHELL_GET_EXPRESSION('reason'), "score":"10" }
    conf := THREAT_TRIAGE_ADD(conf, [triage])
    
    # save the configuration
    CONFIG_PUT("ENRICHMENT", conf, "bro")
    ```
    
    Monitor the messages being pushed to the 'indexing' topic in Kafka.  You should expect to see a message where the 'threat.triage.level' is like the example shown above.
    
    Printing the threat triage rule in the REPL is going to look something like this.
    ```
    > THREAT_TRIAGE_PRINT(conf)
    \u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
    \u2551 Name              \u2502 Comment \u2502 Triage Rule                                                      \u2502 Score \u2502 Reason                                                                            \u2551
    \u2560\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2563
    \u2551 Abnormal DNS Port \u2502         \u2502 source.type == "bro" and protocol == "dns" and ip_dst_port != 53 \u2502 10    \u2502 FORMAT("Abnormal DNS Port: expected: 53, found: %s:%d", ip_dst_addr, ip_dst_port) \u2551
    \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u255d
    
    ```

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

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

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

    https://github.com/apache/incubator-metron/pull/438.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 #438
    
----
commit fe455187ff1f928edc43a6bfa509408695783a54
Author: Nick Allen <ni...@nickallen.org>
Date:   2017-02-02T23:29:07Z

    METRON-686 Record Rule Set that Fired During Threat Triage

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I think what we are saying is 1.  I think that is the best way to move forward.
    Just make sure that this commit isn't merged into an RC for 0.3.1



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100599107
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java ---
    @@ -133,14 +136,18 @@ public JSONObject joinMessages(Map<String, JSONObject> streamMessageMap) {
               LOG.debug(sourceType + ": Empty rules!");
             }
     
    +        // triage the threat
             ThreatTriageProcessor threatTriageProcessor = new ThreatTriageProcessor(config, functionResolver, stellarContext);
    -        Double triageLevel = threatTriageProcessor.apply(ret);
    +        ThreatScore score = threatTriageProcessor.apply(ret);
    +
             if(LOG.isDebugEnabled()) {
               String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules());
    -          LOG.debug("Marked " + sourceType + " as triage level " + triageLevel + " with rules " + rules);
    +          LOG.debug("Marked " + sourceType + " as triage level " + score.getScore() + " with rules " + rules);
             }
    -        if(triageLevel != null && triageLevel > 0) {
    -          ret.put("threat.triage.level", triageLevel);
    +
    +        // attach the triage threat score to the message
    +        if(score.getRuleScores().size() > 0) {
    +          ret.put("threat.triage.level", toMap(score));
    --- End diff --
    
    I agree with Casey.  We need a flat structure.  I would turn this:
    
    "threat.triage.level":{
       "score":10.0,
       "rules":[
          { 
             "name":"Abnormal DNS Port",
             "comment":null
             "score":10.0,
             "reason":"Abnormal DNS Port: expected: 53, found: 224.0.0.251:5353",
          }
       ]
    }
    
    into
    
    "threat.triage.score": 10,
    "threat.triage.rule.RuleName1.name":"Abnormal DNS Port",
    "threat.triage.rule.RuleName1.comment":"Some comment...."
    "threat.triage.rule.RuleName1.score": 10.0
    "threat.triage.rule.RuleName1.reason": "Abnormal DNS Port: expected: 53, found: 224.0.0.251:5353"



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Re: [GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

Posted by Otto Fowler <ot...@gmail.com>.
Could you re-use the jsonmap parser stuff?


On February 10, 2017 at 13:58:54, cestella (git@git.apache.org) wrote:

Github user cestella commented on the issue:

https://github.com/apache/incubator-metron/pull/438

It occurs to me that one thing we might consider going forward is to
flatten the map upon writing to the index. We could configure the message
flattening as a capability on a per-writer basis. This would let us adjust
the message representation to the index.

We already have a hook to transform the messages (the `FieldNameConverter`
class). That being said, we'd have to have different hooks than just
`FieldNameConverter`, which just operates on field names. Instead, you'd
need a `FieldValueConverter` as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    It occurs to me that one thing we might consider going forward is to flatten the map upon writing to the index.  We could configure the message flattening as a capability on a per-writer basis.  This would let us adjust the message representation to the index.
    
    We already have a hook to transform the messages (the `FieldNameConverter` class). That being said, we'd have to have different hooks than just `FieldNameConverter`, which just operates on field names.  Instead, you'd need a `FieldValueConverter` as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I would be open to flattening if we have some sort of convention, but it's just not ideal. Much easier to grok like it is in my opinion.
    
    Q: If we do have this convention, is it documented somewhere? I want to make sure I am not breaking other conventions.
    
    Q. Is it worthwhile to move in the direction of supporting complex types?
    
    Q. Is Solr the only problem with complex types that we know of? I would like to open a JIRA to move toward complex types (assuming we agree that complex types are a good thing.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Created [METRON-735](https://issues.apache.org/jira/browse/METRON-735), in case this idea gains wider support from the community.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100442785
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    It is a cool idea @justinleet.  
    
    If we could build the abstract syntax tree and then cache that, we could certainly improve performance.  Of course, we'd have to confirm how much time we spend building the AST versus execution, but I'd be willing to bet you are right.
    
    I don't know that we have the ability to do this today though.  At least, I have not seen us do something like this elsewhere.  But I could be wrong.  
    
    Might be a good general strategy to explore for improving performance in a lot of places considering how much we rely on Stellar executions nowadays.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Ha... oops.  Yep, we agree.  Add your commentary to METRON-735.  You're suggesting a specific implementation, which is good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Ok, I'm +1 on this pending successful travis.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100542070
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    Ok, so, yes, I have an opinion about that.  Sadly, with the current implementation of Stellar, we cannot cache the parse tree and then apply it after the fact.  It's just an artifact of how we do the parsing: we actually execute the statement as we parse rather than constructing an AST that can then be evaluated later given a message.
    
    I think this would be a GREAT modification to Stellar and one that I had on my mental backlog for some time.  In fact, I took the opportunity as part of the profiler window DSL to do it the right way (parse to an object that can then be cached and evaluated on multiple sets of inputs without reparse).
    
    tl;dr: We can't do it now easily, BUT we should for multiple reasons:
    * code clarity - decoupling the stellar language from the generated parser code
    * performance - saving lexing and parsing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Do we know how much work it is the create the enforcement layer?  Is it something we want to hold this up for, or is it something that gets flattened here and a ticket gets made to improve it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I saw some odd behavior I think is unrelated to this PR itself while testing.
    
    I tried to remove the threat triage rule, messed up, fixed it, and then borked my conf variable.  After recovering I was able to successfully test (and run THREAT_TRIAGE_REMOVE without breaking conf).
    
    ```
    [Stellar]>>> conf
    {
      "enrichment" : {
        "fieldMap" : {
          "geo" : [ "ip_dst_addr", "ip_src_addr" ],
          "host" : [ "host" ],
          "stellar" : {
            "config" : {
              "is_alert" : "source.type == 'bro'"
            }
          }
        },
        "fieldToTypeMap" : { },
        "config" : { }
      },
      "threatIntel" : {
        "fieldMap" : {
          "hbaseThreatIntel" : [ "ip_src_addr", "ip_dst_addr" ]
        },
        "fieldToTypeMap" : {
          "ip_src_addr" : [ "malicious_ip" ],
          "ip_dst_addr" : [ "malicious_ip" ]
        },
        "config" : { },
        "triageConfig" : {
          "riskLevelRules" : [ {
            "name" : "Abnormal DNS Port",
            "rule" : "source.type == \"bro\" and protocol == \"dns\" and ip_dst_port != 53",
            "score" : 10.0
          } ],
          "aggregator" : "MAX",
          "aggregationConfig" : { }
        }
      },
      "configuration" : { }
    }
    [Stellar]>>> conf := THREAT_TRIAGE_REMOVE(conf, 'Abnormal DNS Port')
    [!] Unable to execute: java.lang.String cannot be cast to java.util.List
    org.apache.metron.common.dsl.ParseException: Unable to execute: java.lang.String cannot be cast to java.util.List
    	at org.apache.metron.common.stellar.StellarCompiler.getResult(StellarCompiler.java:409)
    	at org.apache.metron.common.stellar.BaseStellarProcessor.parse(BaseStellarProcessor.java:127)
    	at org.apache.metron.common.stellar.shell.StellarExecutor.execute(StellarExecutor.java:275)
    	at org.apache.metron.common.stellar.shell.StellarShell.executeStellar(StellarShell.java:373)
    	at org.apache.metron.common.stellar.shell.StellarShell.handleStellar(StellarShell.java:276)
    	at org.apache.metron.common.stellar.shell.StellarShell.execute(StellarShell.java:412)
    	at org.jboss.aesh.console.AeshProcess.run(AeshProcess.java:53)
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
    	at java.lang.Thread.run(Thread.java:745)
    Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.List
    	at org.apache.metron.management.ThreatTriageFunctions$RemoveStellarTransformation.apply(ThreatTriageFunctions.java:232)
    	at org.apache.metron.common.stellar.StellarCompiler.exitTransformationFunc(StellarCompiler.java:267)
    	at org.apache.metron.common.stellar.generated.StellarParser$TransformationFuncContext.exitRule(StellarParser.java:1689)
    	at org.antlr.v4.runtime.Parser.triggerExitRuleEvent(Parser.java:422)
    	at org.antlr.v4.runtime.Parser.exitRule(Parser.java:632)
    	at org.apache.metron.common.stellar.generated.StellarParser.functions(StellarParser.java:1712)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_operands(StellarParser.java:1846)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_expr_mul(StellarParser.java:1609)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_expr(StellarParser.java:1469)
    	at org.apache.metron.common.stellar.generated.StellarParser.transformation_expr(StellarParser.java:308)
    	at org.apache.metron.common.stellar.generated.StellarParser.transformation(StellarParser.java:149)
    	at org.apache.metron.common.stellar.BaseStellarProcessor.parse(BaseStellarProcessor.java:126)
    	... 8 more
    [Stellar]>>> conf := THREAT_TRIAGE_REMOVE(conf, ['Abnormal DNS Port'])
    [Stellar]>>> conf
    {
      "enrichment" : {
        "fieldMap" : { },
        "fieldToTypeMap" : { },
        "config" : { }
      },
      "threatIntel" : {
        "fieldMap" : { },
        "fieldToTypeMap" : { },
        "config" : { },
        "triageConfig" : {
          "riskLevelRules" : [ ],
          "aggregator" : "MAX",
          "aggregationConfig" : { }
        }
      },
      "configuration" : { }
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    rgr, deleting 736


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    @nickwallen I have slight preference towards flattening, fixing, and unflattening. I'd rather conform to convention and keep things consistent for now.  I could pretty easily be persuaded to go with 1 if there's enough support for it and we think we'll address it relatively quickly.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    One thing to consider is that we have other enrichments, like the `GEO_GET` function, that add a complex type (map) to the message.  So right now, that functionality is in the same boat as this PR.
    
    I like the trend that was introduced (or reinforced?) by `GEO_GET`.  That is moving enrichments from custom code to Stellar.  I don't think it makes sense to require all sources of enrichment (like `GEO_GET`) to flatten the data that it produces.  I think doing see places an unfair burden on our users.
    
    
    
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Sounds good, looks like there are more votes for option 2.  I'll go that route.  Gracias!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I commented on 735 so it wins. #settled


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Ok, I flattened the threat score with the latest commit.  The PR description has also been updated to match current state.  Please take a gander.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100395910
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    Honestly, I am not aware enough of the underpinnings to give a good suggestion.  My immediate thought is caching the parsed version of the rule, and only parsing if we don't have it.
    
    @cestella Any thoughts on if this causes potential performance issues and if caching is a) possible and b) useful here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I think a better approach is to bake in an enforcement layer within Metron to only allow flat maps (key-value pairs where the value cannot be a complex object).  You would enforce that in the parsers as well as enrichers. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Seems like THREAT_TRIAGE_REMOVE just behaves really badly
    
    ```
    [Stellar]>>> conf := THREAT_TRIAGE_ADD(conf, [triage])
    [Stellar]>>> conf := THREAT_TRIAGE_REMOVE(conf, 'Abnormal DNS Port')
    [!] Unable to execute: java.lang.String cannot be cast to java.util.List
    org.apache.metron.common.dsl.ParseException: Unable to execute: java.lang.String cannot be cast to java.util.List
    	at org.apache.metron.common.stellar.StellarCompiler.getResult(StellarCompiler.java:409)
    	at org.apache.metron.common.stellar.BaseStellarProcessor.parse(BaseStellarProcessor.java:127)
    	at org.apache.metron.common.stellar.shell.StellarExecutor.execute(StellarExecutor.java:275)
    	at org.apache.metron.common.stellar.shell.StellarShell.executeStellar(StellarShell.java:373)
    	at org.apache.metron.common.stellar.shell.StellarShell.handleStellar(StellarShell.java:276)
    	at org.apache.metron.common.stellar.shell.StellarShell.execute(StellarShell.java:412)
    	at org.jboss.aesh.console.AeshProcess.run(AeshProcess.java:53)
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
    	at java.lang.Thread.run(Thread.java:745)
    Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to java.util.List
    	at org.apache.metron.management.ThreatTriageFunctions$RemoveStellarTransformation.apply(ThreatTriageFunctions.java:232)
    	at org.apache.metron.common.stellar.StellarCompiler.exitTransformationFunc(StellarCompiler.java:267)
    	at org.apache.metron.common.stellar.generated.StellarParser$TransformationFuncContext.exitRule(StellarParser.java:1689)
    	at org.antlr.v4.runtime.Parser.triggerExitRuleEvent(Parser.java:422)
    	at org.antlr.v4.runtime.Parser.exitRule(Parser.java:632)
    	at org.apache.metron.common.stellar.generated.StellarParser.functions(StellarParser.java:1712)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_operands(StellarParser.java:1846)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_expr_mul(StellarParser.java:1609)
    	at org.apache.metron.common.stellar.generated.StellarParser.arithmetic_expr(StellarParser.java:1469)
    	at org.apache.metron.common.stellar.generated.StellarParser.transformation_expr(StellarParser.java:308)
    	at org.apache.metron.common.stellar.generated.StellarParser.transformation(StellarParser.java:149)
    	at org.apache.metron.common.stellar.BaseStellarProcessor.parse(BaseStellarProcessor.java:126)
    	... 8 more
    [Stellar]>>> conf
    [Stellar]>>> conf
    [Stellar]>>> conf := THREAT_TRIAGE_REMOVE(conf, ['Abnormal DNS Port'])
    [Stellar]>>> conf
    {
      "enrichment" : {
        "fieldMap" : { },
        "fieldToTypeMap" : { },
        "config" : { }
      },
      "threatIntel" : {
        "fieldMap" : { },
        "fieldToTypeMap" : { },
        "config" : { },
        "triageConfig" : {
          "riskLevelRules" : [ ],
          "aggregator" : "MAX",
          "aggregationConfig" : { }
        }
      },
      "configuration" : { }
    }
    ```
    
    I'd have to check if the same thing happens on master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100379863
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    This gets called to execute the 'reason' Stellar expression that is part of a threat triage rule.  It is only called on the rules that are 'fired' (rule predicate is true) for a particular message (line 89).  
    
    A rule's 'reason' is something that could be changed by the user at any time.  I am unsure of a better way to do it.  Thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    @ottobackwards Yep, good call out.  We should address the JSON mapper as you described also. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100542945
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    I went ahead and created a [JIRA](https://issues.apache.org/jira/browse/METRON-712) for this to capture the notion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I think the only thing that's concerning me here is that we have avoided complex types in values in the past.  I believe it had to do with ES performance and the fact that not all indexes (i.e. Solr) support it.  
    
    I'd suggest flattening that `threat.triage.level` map into
    * `threat.triage.level.score`
    * `threat.triage.level.name`
    * `threat.triage.level.comment`
    * `threat.triage.level.reason`
    
    Or, alternatively, just drop the level and make it `threat.triage.score`, etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100596601
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java ---
    @@ -133,14 +136,18 @@ public JSONObject joinMessages(Map<String, JSONObject> streamMessageMap) {
               LOG.debug(sourceType + ": Empty rules!");
             }
     
    +        // triage the threat
             ThreatTriageProcessor threatTriageProcessor = new ThreatTriageProcessor(config, functionResolver, stellarContext);
    -        Double triageLevel = threatTriageProcessor.apply(ret);
    +        ThreatScore score = threatTriageProcessor.apply(ret);
    +
             if(LOG.isDebugEnabled()) {
               String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules());
    -          LOG.debug("Marked " + sourceType + " as triage level " + triageLevel + " with rules " + rules);
    +          LOG.debug("Marked " + sourceType + " as triage level " + score.getScore() + " with rules " + rules);
             }
    -        if(triageLevel != null && triageLevel > 0) {
    -          ret.put("threat.triage.level", triageLevel);
    +
    +        // attach the triage threat score to the message
    +        if(score.getRuleScores().size() > 0) {
    +          ret.put("threat.triage.level", toMap(score));
    --- End diff --
    
    I thought there was a reason that we didn't put complex types as values in the JSON.  I didn't think ES or kibana was able to handle them.  Maybe instead of a map, we can unfold to `threat.triage.level.score`, `threat.triage.level.rules`, etc.
    
    Also, since we're changing this, do we have any dashboards that need to be changed in kibana?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    So assuming we put aside the issue of flattening the data per METRON-735, what should be the go-forward for this PR?  I outlined two options above.  Please share your opinions on those options.
    
    > (1) Assume that any indexer that cannot handle complex types is currently broken.
    > 
    > Since the assumption is that the Solr indexer is 'broke', then we can move forward and commit this PR (pending further review) BEFORE addressing the Solr indexer.
    > I will then create a JIRA to test and fix any indexer that cannot handle complex types. Based on the discussion here, I am assuming this includes the Solr indexer.
    >
    > (2) Assume that we need to live in our current box.
    > 
    > I will update this PR to flatten the data to make the Solr Indexer happy.
    > I will then create a JIRA to fix the Solr Indexer.
    > After the Solr Indexer is fixed, I will then create another PR to return threat triage to its current state (using a complex type instead of flattening.)
    
    Personally I am +1 on option 1, but either works for me.  Maybe there is a better approach that I am not thinking of.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    After talking with Casey, it's an issue with StellarShell, not this PR.  I'll make a ticket and get it done.  So feel free to ignore this issue, @nickwallen 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    @nickwallen I've been able to repeat this, by leaving reason undefined.  Any idea why that would cause the configs to go haywire?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100901277
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java ---
    @@ -133,14 +136,18 @@ public JSONObject joinMessages(Map<String, JSONObject> streamMessageMap) {
               LOG.debug(sourceType + ": Empty rules!");
             }
     
    +        // triage the threat
             ThreatTriageProcessor threatTriageProcessor = new ThreatTriageProcessor(config, functionResolver, stellarContext);
    -        Double triageLevel = threatTriageProcessor.apply(ret);
    +        ThreatScore score = threatTriageProcessor.apply(ret);
    +
             if(LOG.isDebugEnabled()) {
               String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules());
    -          LOG.debug("Marked " + sourceType + " as triage level " + triageLevel + " with rules " + rules);
    +          LOG.debug("Marked " + sourceType + " as triage level " + score.getScore() + " with rules " + rules);
             }
    -        if(triageLevel != null && triageLevel > 0) {
    -          ret.put("threat.triage.level", triageLevel);
    +
    +        // attach the triage threat score to the message
    +        if(score.getRuleScores().size() > 0) {
    +          ret.put("threat.triage.level", toMap(score));
    --- End diff --
    
    I think it makes the user's life harder if we put the rule name as part of the key.  We would have to enforce our internal Metron conventions (no spaces, special characters, whatever) on the user's rule set. Conventions that we could change over time and then break a user's rule set during an upgrade.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Bump?  Haven't received feedback on my last comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Nick - if you look at the JSONMapParser, Casey and I implemented a flattener that you would use for this.  So, all of our json stuff from that _was_ already flattened.  I am more confused about there being *non* flattened things going through actually.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #438: METRON-686 Record Rule Set that Fired Du...

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

    https://github.com/apache/incubator-metron/pull/438#discussion_r100372924
  
    --- Diff: metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java ---
    @@ -52,15 +74,36 @@ public ThreatTriageProcessor( SensorEnrichmentConfig config
     
       @Nullable
       @Override
    -  public Double apply(@Nullable Map input) {
    -    List<Number> scores = new ArrayList<>();
    +  public ThreatScore apply(@Nullable Map input) {
    +
    +    ThreatScore threatScore = new ThreatScore();
         StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
    +    StellarProcessor processor = new StellarProcessor();
         VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
    +
    +    // attempt to apply each rule to the threat
         for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
           if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
    -        scores.add(rule.getScore());
    +
    +        // add the rule's score to the overall threat score
    +        String reason = execute(rule.getReason(), processor, resolver, String.class);
    +        RuleScore score = new RuleScore(rule, reason);
    +        threatScore.addRuleScore(score);
           }
         }
    -    return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());
    +
    +    // calculate the aggregate threat score
    +    Aggregators aggregators = threatTriageConfig.getAggregator();
    +    List<Number> allScores = threatScore.getRuleScores().stream().map(score -> score.getRule().getScore()).collect(Collectors.toList());
    +    Double aggregateScore = aggregators.aggregate(allScores, threatTriageConfig.getAggregationConfig());
    +
    +    // return the overall threat score
    +    threatScore.setScore(aggregateScore);
    +    return threatScore;
    +  }
    +
    +  private <T> T execute(String expression, StellarProcessor processor, VariableResolver resolver, Class<T> clazz) {
    +    Object result = processor.parse(expression, resolver, functionResolver, context);
    --- End diff --
    
    This parse() gets called on every message that has "is_alert", correct?  Does that have any negative perf implications?  My assumption is that parsing every time isn't cheap, but I'm unsure how expensive it actually is in this context


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    let me know which one to comment on!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I think it is much cleaner to transform at the indexer, per the requirements of the index destination. Otherwise, we will be forced to adhere to the lowest common denominator of index destinations that we happen to support at any point in time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I'm of the opinion that the flattening should be writer-specific and that should be a function of the writer config with the default to be specified by the writer implementation.  This way we can have our cake and eat it too.  Also, we could ALREADY be in a situation where messages aren't flat (imagine a situation where a stellar function returns a map or a list and it get assigned to a field).  The *only* safe way to do this is to enforce it at the writer, IMO.  This is one of the stated benefits to extracting writer configs into their own structures.
    
    Regarding existing conventions, this one was around when I joined the project.  I might be wrong, but it was an early convention.  The reasoning, as I recall, was multi-fold:
    * Solr didn't handle it
    * Interacting with complex structures was deemed to be difficult
    * Indexing nested structures had some performance implications
    
    As to how to move forward, my suggestion is conform to convention for this JIRA and flatten.  I created a JIRA to track the flattening effort at [METRON-736](https://issues.apache.org/jira/browse/METRON-736)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    Ok, it appears that I tramped you, @nickwallen Our JIRAs seem to be suggesting the same thing, am I right?  If you say so, I'll delete mine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    I really could use some opinions on the go-forward here.  I see two options.
    
    (1) Assume that any indexer that cannot handle complex types is currently broken.  
    
    * Since the assumption is that the Solr indexer is 'broke', then we can move forward and commit this PR (pending further review) BEFORE addressing the Solr indexer.
    * I will then create a JIRA to test and fix any indexer that cannot handle complex types.  Based on the discussion here, I am assuming this includes the Solr indexer.  
    
    (2) Assume that we need to live in our current box.
    
    * I will update this PR to flatten the data to make the Solr Indexer happy.
    * I will then create a JIRA to fix the Solr Indexer.
    * After the Solr Indexer is fixed, I will then create another PR to return threat triage to its current state (using a complex type instead of flattening.)
    
    I think those are the best options forward.  Let me know if there is another alternative.  Thanks all!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    @ottobackwards @cestella @justinleet  - Let me know what you guys think on my previous comment about how to move forward on this PR.  
    
    I think we have a rough outline of the "flattening" stuff, but how should we move forward on this PR specifically.  See options outlined in my previous comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    +1, Everything looks good.  As noted above, both the issue I had and the review comment were addressed by adding new Jiras and aren't needed here.
    
    Thanks a lot, Nick. This is really helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    > if you look at the JSONMapParser, Casey and I implemented a flattener...
    
    @ottobackwards Thanks for pointing that out.  If we end up going with option 2, I will try and reuse that flattener. 
    
     > I am more confused about there being non flattened things going through actually.
    
    Even if I use a parser that generates only flattened data, I could create an enrichment (like using `GEO_GET`) that appends non-flat data to a message.  Thus we have non-flat data coursing through the veins of Metron. ;)
    
    I think the original idea of flattening data was because one of our Indexers could not handle non-flat, complex data types.  At the time, we just decided, well don't create any non-flat data.  
    
    But now, since we have a completely 'programmable' system, I don't think it is safe to assume that the data will always be non-flat.  A user could create their own Stellar function to use during enrichment.  Should we force on them the burden of flattening the data?
    
    It makes way more sense in my mind, to make the indexer transform the data however it needs to , to correctly index the data.  If the current issue is with the Solr Indexer, then we should fix that to flatten any data that it needs to.  There would be one touch point to address this issue rather than many.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #438: METRON-686 Record Rule Set that Fired During Th...

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

    https://github.com/apache/incubator-metron/pull/438
  
    That makes sense, thanks for the explanation.
    I am +1 for having SOLR specific flattening, although documenting well needs to be a priority.
    Also, if we do this, please log a jira that the flattening in the json mapper needs to be optional by configuration


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---