You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ni...@apache.org on 2017/02/27 23:02:54 UTC

incubator-metron git commit: METRON-686 Record Rule Set that Fired During Threat Triage (nickwallen) closes apache/incubator-metron#438

Repository: incubator-metron
Updated Branches:
  refs/heads/master 41fc0ddc9 -> b34037202


METRON-686 Record Rule Set that Fired During Threat Triage (nickwallen) closes apache/incubator-metron#438


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

Branch: refs/heads/master
Commit: b34037202775f78733cd21cc0fb4159991f29cd3
Parents: 41fc0dd
Author: nickwallen <ni...@nickallen.org>
Authored: Mon Feb 27 18:02:36 2017 -0500
Committer: Nick Allen <ni...@nickallen.org>
Committed: Mon Feb 27 18:02:36 2017 -0500

----------------------------------------------------------------------
 .../enrichment/threatintel/RiskLevelRule.java   |  69 ++++++-
 .../enrichment/threatintel/RuleScore.java       |  88 +++++++++
 .../enrichment/threatintel/ThreatScore.java     |  94 ++++++++++
 .../threatintel/ThreatTriageConfig.java         |  25 ++-
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |  84 ++++++++-
 .../triage/ThreatTriageProcessor.java           |  57 +++++-
 .../bolt/ThreatIntelJoinBoltTest.java           |   8 +-
 .../integration/EnrichmentIntegrationTest.java  |  60 ++++--
 .../threatintel/triage/ThreatTriageTest.java    | 185 ++++++++++++++++---
 .../main/config/zookeeper/enrichments/test.json |   5 +-
 .../management/ThreatTriageFunctions.java       |  12 +-
 .../management/ThreatTriageFunctionsTest.java   |  28 +--
 12 files changed, 624 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java
index 7bf1d07..94ab0c8 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java
@@ -17,12 +17,51 @@
  */
 package org.apache.metron.common.configuration.enrichment.threatintel;
 
+/**
+ * This class represents a rule that is used to triage threats.
+ *
+ * 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.
+ *
+ * Tuning the threat triage process involves creating one or more rules, adjusting
+ * the score of each rule, and changing the way that each rule's score is aggregated.
+ */
 public class RiskLevelRule {
+
+  /**
+   * The name of the rule. This field is optional.
+   */
   String name;
+
+  /**
+   * A description of the rule. This field is optional.
+   */
   String comment;
+
+  /**
+   * A predicate, in the form of a Stellar expression, that determines whether
+   * the rule is applied to an alert or not.  This field is required.
+   */
   String rule;
+
+  /**
+   * A numeric value that represents the score that is applied to the alert. This
+   * field is required.
+   */
   Number score;
 
+  /**
+   * Allows a rule author to provide contextual information when a rule is applied
+   * to a message.  This can assist a SOC analyst when actioning a threat.
+   *
+   * This is expected to be a valid Stellar expression and can refer to any of the
+   * fields within the message itself.
+   */
+  String reason;
+
   public String getName() {
     return name;
   }
@@ -55,14 +94,12 @@ public class RiskLevelRule {
     this.score = score;
   }
 
-  @Override
-  public String toString() {
-    return "RiskLevelRule{" +
-            "name='" + name + '\'' +
-            ", comment='" + comment + '\'' +
-            ", rule='" + rule + '\'' +
-            ", score=" + score +
-            '}';
+  public String getReason() {
+    return reason;
+  }
+
+  public void setReason(String reason) {
+    this.reason = reason;
   }
 
   @Override
@@ -75,8 +112,8 @@ public class RiskLevelRule {
     if (name != null ? !name.equals(that.name) : that.name != null) return false;
     if (comment != null ? !comment.equals(that.comment) : that.comment != null) return false;
     if (rule != null ? !rule.equals(that.rule) : that.rule != null) return false;
-    return score != null ? score.equals(that.score) : that.score == null;
-
+    if (score != null ? !score.equals(that.score) : that.score != null) return false;
+    return reason != null ? reason.equals(that.reason) : that.reason == null;
   }
 
   @Override
@@ -85,6 +122,18 @@ public class RiskLevelRule {
     result = 31 * result + (comment != null ? comment.hashCode() : 0);
     result = 31 * result + (rule != null ? rule.hashCode() : 0);
     result = 31 * result + (score != null ? score.hashCode() : 0);
+    result = 31 * result + (reason != null ? reason.hashCode() : 0);
     return result;
   }
+
+  @Override
+  public String toString() {
+    return "RiskLevelRule{" +
+            "name='" + name + '\'' +
+            ", comment='" + comment + '\'' +
+            ", rule='" + rule + '\'' +
+            ", score=" + score +
+            ", reason='" + reason + '\'' +
+            '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RuleScore.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RuleScore.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RuleScore.java
new file mode 100644
index 0000000..71bd13b
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RuleScore.java
@@ -0,0 +1,88 @@
+/**
+ * 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.common.configuration.enrichment.threatintel;
+
+/**
+ * This class represents the score resulting from applying a RiskLevelRule
+ * to a message.
+ *
+ * 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.
+ */
+public class RuleScore {
+
+  /**
+   * The rule that when applied to a message resulted in this score.
+   */
+  RiskLevelRule rule;
+
+  /**
+   * Allows a rule author to provide contextual information when a rule is applied
+   * to a message.  This can assist a SOC analyst when actioning a threat.
+   *
+   * This is the result of executing the 'reason' Stellar expression from the
+   * associated RiskLevelRule.
+   */
+  private String reason;
+
+  /**
+   * @param rule The threat triage rule that when applied resulted in this score.
+   * @param reason The result of executing the rule's 'reason' expression.  Provides context to why a rule was applied.
+   */
+  public RuleScore(RiskLevelRule rule, String reason) {
+    this.rule = rule;
+    this.reason = reason;
+  }
+
+  public String getReason() {
+    return reason;
+  }
+
+  public RiskLevelRule getRule() {
+    return rule;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RuleScore that = (RuleScore) o;
+
+    if (rule != null ? !rule.equals(that.rule) : that.rule != null) return false;
+    return reason != null ? reason.equals(that.reason) : that.reason == null;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = rule != null ? rule.hashCode() : 0;
+    result = 31 * result + (reason != null ? reason.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "RuleScore{" +
+            "rule=" + rule +
+            ", reason='" + reason + '\'' +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatScore.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatScore.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatScore.java
new file mode 100644
index 0000000..3e5b583
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatScore.java
@@ -0,0 +1,94 @@
+/**
+ * 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.common.configuration.enrichment.threatintel;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The overall threat score which is arrived at by aggregating the individual
+ * scores from each applied rule.
+ *
+ * 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.
+ *
+ * Tuning the threat triage process involves creating one or more rules, adjusting
+ * the score of each rule, and changing the way that each rule's score is aggregated.
+ */
+public class ThreatScore {
+
+  /**
+   * The numeric threat score resulting from aggregating
+   * all of the individual scores from each applied rule.
+   */
+  private Double score;
+
+  /**
+   * The individual rule scores produced by applying each rule to the message.
+   */
+  private List<RuleScore> ruleScores;
+
+  public ThreatScore() {
+    this.ruleScores = new ArrayList<>();
+  }
+
+  public Double getScore() {
+    return score;
+  }
+
+  public void setScore(Double score) {
+    this.score = score;
+  }
+
+  public List<RuleScore> getRuleScores() {
+    return ruleScores;
+  }
+
+  public void addRuleScore(RuleScore score) {
+    this.ruleScores.add(score);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ThreatScore that = (ThreatScore) o;
+
+    if (score != null ? !score.equals(that.score) : that.score != null) return false;
+    return ruleScores != null ? ruleScores.equals(that.ruleScores) : that.ruleScores == null;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = score != null ? score.hashCode() : 0;
+    result = 31 * result + (ruleScores != null ? ruleScores.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "ThreatScore{" +
+            "score=" + score +
+            ", ruleScores=" + ruleScores +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java
index c3f5e55..d29731d 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java
@@ -19,14 +19,19 @@
 package org.apache.metron.common.configuration.enrichment.threatintel;
 
 import com.google.common.base.Joiner;
-import org.apache.metron.common.aggregator.Aggregator;
 import org.apache.metron.common.aggregator.Aggregators;
 import org.apache.metron.common.stellar.StellarPredicateProcessor;
-import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.metron.common.stellar.StellarProcessor;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 public class ThreatTriageConfig {
+
   private List<RiskLevelRule> riskLevelRules = new ArrayList<>();
   private Aggregators aggregator = Aggregators.MAX;
   private Map<String, Object> aggregationConfig = new HashMap<>();
@@ -38,7 +43,9 @@ public class ThreatTriageConfig {
   public void setRiskLevelRules(List<RiskLevelRule> riskLevelRules) {
     List<RiskLevelRule> rules = new ArrayList<>();
     Set<String> ruleIndex = new HashSet<>();
-    StellarPredicateProcessor processor = new StellarPredicateProcessor();
+    StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
+    StellarProcessor processor = new StellarProcessor();
+
     for(RiskLevelRule rule : riskLevelRules) {
       if(rule.getRule() == null || rule.getScore() == null) {
         throw new IllegalStateException("Risk level rules must contain both a rule and a score.");
@@ -49,7 +56,13 @@ public class ThreatTriageConfig {
       else {
         ruleIndex.add(rule.getRule());
       }
-      processor.validate(rule.getRule());
+
+      // validate the fields which are expected to be valid Stellar expressions
+      predicateProcessor.validate(rule.getRule());
+      if(rule.getReason() != null) {
+        processor.validate(rule.getReason());
+      }
+
       rules.add(rule);
     }
     this.riskLevelRules = rules;
@@ -59,8 +72,6 @@ public class ThreatTriageConfig {
     return aggregator;
   }
 
-
-
   public void setAggregator(String aggregator) {
     try {
       this.aggregator = Aggregators.valueOf(aggregator);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
index 6584a27..4fd5e02 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
@@ -17,19 +17,21 @@
  */
 package org.apache.metron.enrichment.bolt;
 
-import org.apache.metron.common.configuration.ConfigurationType;
-import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
-import org.apache.storm.task.TopologyContext;
 import com.google.common.base.Joiner;
+import org.apache.metron.common.configuration.ConfigurationType;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.common.configuration.enrichment.threatintel.RuleScore;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatScore;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
 import org.apache.metron.common.dsl.Context;
-import org.apache.metron.common.dsl.functions.resolver.FunctionResolver;
 import org.apache.metron.common.dsl.StellarFunctions;
+import org.apache.metron.common.dsl.functions.resolver.FunctionResolver;
 import org.apache.metron.common.utils.ConversionUtils;
 import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
 import org.apache.metron.threatintel.triage.ThreatTriageProcessor;
+import org.apache.storm.task.TopologyContext;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,8 +42,46 @@ import java.util.Map;
 public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
 
   protected static final Logger LOG = LoggerFactory.getLogger(ThreatIntelJoinBolt.class);
+
+  /**
+   * The message key under which the overall threat triage score is stored.
+   */
+  public static final String THREAT_TRIAGE_SCORE_KEY = "threat.triage.score";
+
+  /**
+   * The prefix of the message keys that record the threat triage rules that fired.
+   */
+  public static final String THREAT_TRIAGE_RULES_KEY = "threat.triage.rules";
+
+  /**
+   * The portion of the message key used to record the 'name' field of a rule.
+   */
+  public static final String THREAT_TRIAGE_RULE_NAME = "name";
+
+  /**
+   * The portion of the message key used to record the 'comment' field of a rule.
+   */
+  public static final String THREAT_TRIAGE_RULE_COMMENT = "comment";
+
+  /**
+   * The portion of the message key used to record the 'score' field of a rule.
+   */
+  public static final String THREAT_TRIAGE_RULE_SCORE = "score";
+
+  /**
+   * The portion of the message key used to record the 'reason' field of a rule.
+   */
+  public static final String THREAT_TRIAGE_RULE_REASON = "reason";
+
+  /**
+   * The Stellar function resolver.
+   */
   private FunctionResolver functionResolver;
-  private org.apache.metron.common.dsl.Context stellarContext;
+
+  /**
+   * The execution context for Stellar.
+   */
+  private Context stellarContext;
 
   public ThreatIntelJoinBolt(String zookeeperUrl) {
     super(zookeeperUrl);
@@ -133,20 +173,23 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
           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) {
+          appendThreatScore(score, ret);
         }
       }
       else {
         LOG.debug(sourceType + ": Unable to find threat triage config!");
       }
-
     }
 
     return ret;
@@ -159,4 +202,25 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
       GeoLiteDatabase.INSTANCE.updateIfNecessary(getConfigurations().getGlobalConfig());
     }
   }
+
+  /**
+   * Appends the threat score to the telemetry message.
+   * @param threatScore The threat triage score
+   * @param message The telemetry message being triaged.
+   */
+  private void appendThreatScore(ThreatScore threatScore, JSONObject message) {
+
+    // append the overall threat score
+    message.put(THREAT_TRIAGE_SCORE_KEY, threatScore.getScore());
+
+    // append each of the rules - each rule is 'flat'
+    Joiner joiner = Joiner.on(".");
+    int i = 0;
+    for(RuleScore score: threatScore.getRuleScores()) {
+      message.put(joiner.join(THREAT_TRIAGE_RULES_KEY, i, THREAT_TRIAGE_RULE_NAME), score.getRule().getName());
+      message.put(joiner.join(THREAT_TRIAGE_RULES_KEY, i, THREAT_TRIAGE_RULE_COMMENT), score.getRule().getComment());
+      message.put(joiner.join(THREAT_TRIAGE_RULES_KEY, i, THREAT_TRIAGE_RULE_SCORE), score.getRule().getScore());
+      message.put(joiner.join(THREAT_TRIAGE_RULES_KEY, i++, THREAT_TRIAGE_RULE_REASON), score.getReason());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java
index 0c88437..4d22081 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java
@@ -19,25 +19,47 @@
 package org.apache.metron.threatintel.triage;
 
 import com.google.common.base.Function;
+import org.apache.metron.common.aggregator.Aggregators;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
+import org.apache.metron.common.configuration.enrichment.threatintel.RuleScore;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatIntelConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatScore;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
-import org.apache.metron.common.dsl.*;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.MapVariableResolver;
+import org.apache.metron.common.dsl.VariableResolver;
 import org.apache.metron.common.dsl.functions.resolver.FunctionResolver;
 import org.apache.metron.common.stellar.StellarPredicateProcessor;
+import org.apache.metron.common.stellar.StellarProcessor;
+import org.apache.metron.common.utils.ConversionUtils;
 
 import javax.annotation.Nullable;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Applies the threat triage rules to an alert and produces a threat score that is
+ * attached to the alert.
+ *
+ * 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.
+ *
+ * Tuning the threat triage process involves creating one or more rules, adjusting
+ * the score of each rule, and changing the way that each rule's score is aggregated.
+ */
+public class ThreatTriageProcessor implements Function<Map, ThreatScore> {
 
-public class ThreatTriageProcessor implements Function<Map, Double> {
   private SensorEnrichmentConfig sensorConfig;
   private ThreatIntelConfig threatIntelConfig;
   private ThreatTriageConfig threatTriageConfig;
   private Context context;
   private FunctionResolver functionResolver;
+
   public ThreatTriageProcessor( SensorEnrichmentConfig config
                               , FunctionResolver functionResolver
                               , Context context
@@ -52,15 +74,36 @@ public class ThreatTriageProcessor implements Function<Map, Double> {
 
   @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);
+    return ConversionUtils.convert(result, clazz);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
index 60687d8..6fe318e 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
@@ -23,6 +23,7 @@ import junit.framework.TestCase;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.hadoop.fs.Path;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatScore;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
@@ -200,11 +201,12 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
     Assert.assertTrue(joinedMessage.containsKey("is_alert") && "true".equals(joinedMessage.get("is_alert")));
 
     if(withThreatTriage && !badConfig) {
-      Assert.assertTrue(joinedMessage.containsKey("threat.triage.level") &&
-              Math.abs(10d - (Double) joinedMessage.get("threat.triage.level")) < 1e-10);
+      Assert.assertTrue(joinedMessage.containsKey("threat.triage.score"));
+      Double score = (Double) joinedMessage.get("threat.triage.score");
+      Assert.assertTrue(Math.abs(10d - score) < 1e-10);
     }
     else {
-      Assert.assertFalse(joinedMessage.containsKey("threat.triage.level"));
+      Assert.assertFalse(joinedMessage.containsKey("threat.triage.score"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java
index 27c1d11..7306c64 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java
@@ -18,8 +18,14 @@
 package org.apache.metron.enrichment.integration;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.base.*;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.base.Splitter;
 import com.google.common.collect.Iterables;
+import org.apache.commons.collections4.ListUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.metron.TestConstants;
@@ -27,6 +33,7 @@ import org.apache.metron.common.Constants;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
 import org.apache.metron.enrichment.bolt.ErrorEnrichmentBolt;
+import org.apache.metron.enrichment.bolt.ThreatIntelJoinBolt;
 import org.apache.metron.enrichment.converter.EnrichmentHelper;
 import org.apache.metron.enrichment.converter.EnrichmentKey;
 import org.apache.metron.enrichment.converter.EnrichmentValue;
@@ -35,11 +42,14 @@ import org.apache.metron.enrichment.lookup.LookupKV;
 import org.apache.metron.enrichment.lookup.accesstracker.PersistentBloomTrackerCreator;
 import org.apache.metron.enrichment.stellar.SimpleHBaseEnrichmentFunctions;
 import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.integration.*;
+import org.apache.metron.integration.BaseIntegrationTest;
+import org.apache.metron.integration.ComponentRunner;
+import org.apache.metron.integration.Processor;
+import org.apache.metron.integration.ProcessorResult;
 import org.apache.metron.integration.components.FluxTopologyComponent;
 import org.apache.metron.integration.components.KafkaComponent;
-import org.apache.metron.integration.processors.KafkaMessageSet;
 import org.apache.metron.integration.components.ZKServerComponent;
+import org.apache.metron.integration.processors.KafkaMessageSet;
 import org.apache.metron.integration.processors.KafkaProcessor;
 import org.apache.metron.integration.utils.TestUtils;
 import org.apache.metron.test.mock.MockHTable;
@@ -53,7 +63,17 @@ import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static org.apache.metron.enrichment.bolt.ThreatIntelJoinBolt.*;
 
 public class EnrichmentIntegrationTest extends BaseIntegrationTest {
   private static final String SRC_IP = "ip_src_addr";
@@ -224,13 +244,17 @@ public class EnrichmentIntegrationTest extends BaseIntegrationTest {
   public static void baseValidation(Map<String, Object> jsonDoc) {
     assertEnrichmentsExists("threatintels.", setOf("hbaseThreatIntel"), jsonDoc.keySet());
     assertEnrichmentsExists("enrichments.", setOf("geo", "host", "hbaseEnrichment" ), jsonDoc.keySet());
+
+    //ensure no values are empty
     for(Map.Entry<String, Object> kv : jsonDoc.entrySet()) {
-      //ensure no values are empty.
-      Assert.assertTrue(kv.getValue().toString().length() > 0);
+      String actual = Objects.toString(kv.getValue(), "");
+      Assert.assertTrue(String.format("Value of '%s' is empty: '%s'", kv.getKey(), actual), StringUtils.isNotEmpty(actual));
     }
+
     //ensure we always have a source ip and destination ip
     Assert.assertNotNull(jsonDoc.get(SRC_IP));
     Assert.assertNotNull(jsonDoc.get(DST_IP));
+
     Assert.assertNotNull(jsonDoc.get("ALL_CAPS"));
     Assert.assertNotNull(jsonDoc.get("foo"));
     Assert.assertEquals("TEST", jsonDoc.get("ALL_CAPS"));
@@ -349,20 +373,34 @@ public class EnrichmentIntegrationTest extends BaseIntegrationTest {
     }
   }
   private static void threatIntelValidation(Map<String, Object> indexedDoc) {
-    if(indexedDoc.getOrDefault(SRC_IP,"").equals("10.0.2.3")
-    || indexedDoc.getOrDefault(DST_IP,"").equals("10.0.2.3")
-            ) {
+    if(indexedDoc.getOrDefault(SRC_IP,"").equals("10.0.2.3") ||
+            indexedDoc.getOrDefault(DST_IP,"").equals("10.0.2.3")) {
+
       //if we have any threat intel messages, we want to tag is_alert to true
       Assert.assertTrue(keyPatternExists("threatintels.", indexedDoc));
-      Assert.assertTrue(indexedDoc.containsKey("threat.triage.level"));
       Assert.assertEquals(indexedDoc.getOrDefault("is_alert",""), "true");
-      Assert.assertEquals((double)indexedDoc.get("threat.triage.level"), 10d, 1e-7);
+
+      // validate threat triage score
+      Assert.assertTrue(indexedDoc.containsKey(THREAT_TRIAGE_SCORE_KEY));
+      Double score = (Double) indexedDoc.get(THREAT_TRIAGE_SCORE_KEY);
+      Assert.assertEquals(score, 10d, 1e-7);
+
+      // validate threat triage rules
+      Joiner joiner = Joiner.on(".");
+      Stream.of(
+              joiner.join(THREAT_TRIAGE_RULES_KEY, 0, THREAT_TRIAGE_RULE_NAME),
+              joiner.join(THREAT_TRIAGE_RULES_KEY, 0, THREAT_TRIAGE_RULE_COMMENT),
+              joiner.join(THREAT_TRIAGE_RULES_KEY, 0, THREAT_TRIAGE_RULE_REASON),
+              joiner.join(THREAT_TRIAGE_RULES_KEY, 0, THREAT_TRIAGE_RULE_SCORE))
+              .forEach(key ->
+                      Assert.assertTrue(String.format("Missing expected key: '%s'", key), indexedDoc.containsKey(key)));
     }
     else {
       //For YAF this is the case, but if we do snort later on, this will be invalid.
       Assert.assertNull(indexedDoc.get("is_alert"));
       Assert.assertFalse(keyPatternExists("threatintels.", indexedDoc));
     }
+
     //ip threat intels
     if(keyPatternExists("threatintels.hbaseThreatIntel.", indexedDoc)) {
       if(indexedDoc.getOrDefault(SRC_IP,"").equals("10.0.2.3")) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
index d3389af..b3a0868 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
@@ -18,8 +18,11 @@
 
 package org.apache.metron.threatintel.triage;
 
+import com.google.common.collect.ImmutableList;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.RuleScore;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatScore;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.StellarFunctions;
 import org.apache.metron.common.utils.JSONUtils;
@@ -28,33 +31,44 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
 
 public class ThreatTriageTest {
+
+  private static final double delta = 1e-10;
+
   /**
    * {
    *  "threatIntel": {
    *    "triageConfig": {
    *      "riskLevelRules" : [
    *        {
-   *          "name" : "rule 1",
-   *          "rule" : "user.type in [ 'admin', 'power' ] and asset.type == 'web'",
-   *          "score" : 10
+   *          "name": "rule 1",
+   *          "rule": "user.type in [ 'admin', 'power' ] and asset.type == 'web'",
+   *          "score": 10
    *        },
    *        {
-   *         "comment" : "web type!",
-   *         "rule" : "asset.type == 'web'",
-   *         "score" : 5
+   *          "name": "rule 2",
+   *          "comment": "web type!",
+   *          "rule": "asset.type == 'web'",
+   *          "score": 5
    *        },
    *        {
-   *          "rule" : "user.type == 'normal'  and asset.type == 'web'",
-   *          "score" : 0
+   *          "name": "rule 3",
+   *          "rule": "user.type == 'normal' and asset.type == 'web'",
+   *          "score": 0
    *        },
    *        {
-   *          "rule" : "user.type in whitelist",
-   *          "score" : -1
+   *          "name": "rule 4",
+   *          "rule": "user.type in whitelist",
+   *          "score": -1,
+   *          "reason": "user.type"
    *        }
    *      ],
-   *      "aggregator" : "MAX"
+   *      "aggregator": "MAX"
    *    },
    *    "config": {
    *      "whitelist": [ "abnormal" ]
@@ -76,10 +90,10 @@ public class ThreatTriageTest {
                     new SensorEnrichmentConfig(),
                     StellarFunctions.FUNCTION_RESOLVER(),
                     Context.EMPTY_CONTEXT()).apply(
-                            new HashMap<Object, Object>() {{
-                              put("user.type", "admin");
-                              put("asset.type", "web");
-                            }}),
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "admin");
+                      put("asset.type", "web");
+                    }}).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -90,7 +104,7 @@ public class ThreatTriageTest {
                       put("user.type", "admin");
                       put("asset.type", "web");
                     }}
-            ),
+            ).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -101,7 +115,7 @@ public class ThreatTriageTest {
                       put("user.type", "normal");
                       put("asset.type", "web");
                     }}
-            ),
+            ).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -111,7 +125,7 @@ public class ThreatTriageTest {
                     new HashMap<Object, Object>() {{
                       put("user.type", "foo");
                       put("asset.type", "bar");
-                    }}),
+                    }}).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -121,11 +135,73 @@ public class ThreatTriageTest {
                     new HashMap<Object, Object>() {{
                       put("user.type", "abnormal");
                       put("asset.type", "bar");
-                    }}),
+                    }}).getScore(),
             1e-10);
   }
 
   /**
+   * Each individual rule that was applied when scoring a threat should
+   * be captured in the overall threat score.
+   */
+  @Test
+  public void testThreatScoreWithMultipleRules() throws Exception {
+
+    Map<Object, Object> message = new HashMap<Object, Object>() {{
+      put("user.type", "admin");
+      put("asset.type", "web");
+    }};
+
+    ThreatScore score = getProcessor(smokeTestProcessorConfig).apply(message);
+
+    // expect rules 1 and 2 to have been applied
+    List<String> expectedNames = ImmutableList.of("rule 1", "rule 2");
+    Assert.assertEquals(2, score.getRuleScores().size());
+    score.getRuleScores().forEach(ruleScore ->
+            Assert.assertTrue(expectedNames.contains(ruleScore.getRule().getName()))
+    );
+  }
+
+  /**
+   * Each individual rule that was applied when scoring a threat should
+   * be captured in the overall threat score.
+   */
+  @Test
+  public void testThreatScoreWithOneRule() throws Exception {
+
+    Map<Object, Object> message = new HashMap<Object, Object>() {{
+      put("user.type", "abnormal");
+      put("asset.type", "invalid");
+    }};
+
+    ThreatScore score = getProcessor(smokeTestProcessorConfig).apply(message);
+
+    // expect rule 4 to have been applied
+    List<String> expectedNames = ImmutableList.of("rule 4");
+    Assert.assertEquals(1, score.getRuleScores().size());
+    score.getRuleScores().forEach(ruleScore ->
+            Assert.assertTrue(expectedNames.contains(ruleScore.getRule().getName()))
+    );
+  }
+
+  /**
+   * Each individual rule that was applied when scoring a threat should
+   * be captured in the overall threat score.
+   */
+  @Test
+  public void testThreatScoreWithNoRules() throws Exception {
+
+    Map<Object, Object> message = new HashMap<Object, Object>() {{
+      put("user.type", "foo");
+      put("asset.type", "bar");
+    }};
+
+    ThreatScore score = getProcessor(smokeTestProcessorConfig).apply(message);
+
+    // expect no rules to have been applied
+    Assert.assertEquals(0, score.getRuleScores().size());
+  }
+
+  /**
    * {
    *  "threatIntel": {
    *  "triageConfig": {
@@ -152,7 +228,7 @@ public class ThreatTriageTest {
   public static String positiveMeanProcessorConfig;
 
   @Test
-  public void positiveMeanAggregationTest() throws Exception {
+  public void testPositiveMeanAggregationScores() throws Exception {
 
     ThreatTriageProcessor threatTriageProcessor = getProcessor(positiveMeanProcessorConfig);
     Assert.assertEquals(
@@ -162,7 +238,7 @@ public class ThreatTriageTest {
                     new HashMap<Object, Object>() {{
                       put("user.type", "normal");
                       put("asset.type", "web");
-                    }}),
+                    }}).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -172,7 +248,7 @@ public class ThreatTriageTest {
                     new HashMap<Object, Object>() {{
                       put("user.type", "admin");
                       put("asset.type", "web");
-                    }}),
+                    }}).getScore(),
             1e-10);
 
     Assert.assertEquals(
@@ -182,7 +258,7 @@ public class ThreatTriageTest {
                     new HashMap<Object, Object>() {{
                       put("user.type", "foo");
                       put("asset.type", "bar");
-                    }}),
+                    }}).getScore(),
             1e-10);
   }
 
@@ -212,10 +288,71 @@ public class ThreatTriageTest {
             threatTriageProcessor.apply(
                     new HashMap<Object, Object>() {{
                       put("ip_dst_addr", "172.2.2.2");
-                    }}),
+                    }}).getScore(),
             1e-10);
   }
 
+  /**
+   * {
+   *  "threatIntel": {
+   *    "triageConfig": {
+   *      "riskLevelRules" : [
+   *        {
+   *          "name": "Rule Name",
+   *          "comment": "Rule Comment",
+   *          "rule": "2 == 2",
+   *          "score": 10,
+   *          "reason": "variable.name"
+   *        }
+   *      ],
+   *      "aggregator": "MAX"
+   *    }
+   *  }
+   * }
+   */
+  @Multiline
+  public static String testReasonConfig;
+
+  /**
+   * The 'reason' field contained within a rule is a Stellar expression that is
+   * executed within the context of the message that the rule is applied to.
+   */
+  @Test
+  public void testReason() throws Exception {
+
+    Map<Object, Object> message = new HashMap<Object, Object>() {{
+      put("variable.name", "variable.value");
+    }};
+
+    ThreatScore score = getProcessor(testReasonConfig).apply(message);
+    assertEquals(1, score.getRuleScores().size());
+    for(RuleScore ruleScore : score.getRuleScores()) {
+
+      // the 'reason' is the result of executing the rule's 'reason' expression
+      assertEquals("variable.value", ruleScore.getReason());
+    }
+  }
+
+  /**
+   * If the 'reason' expression refers to a missing variable (the result
+   * of a data quality issue) it should not throw an exception.
+   */
+  @Test
+  public void testInvalidReason() throws Exception {
+
+    Map<Object, Object> message = new HashMap<Object, Object>() {{
+      // there is no 'variable.name' in the message
+    }};
+
+    ThreatScore score = getProcessor(testReasonConfig).apply(message);
+    assertEquals(1, score.getRuleScores().size());
+    for(RuleScore ruleScore : score.getRuleScores()) {
+
+      // the 'reason' is the result of executing the rule's 'reason' expression
+      assertEquals(null, ruleScore.getReason());
+    }
+  }
+
   private static ThreatTriageProcessor getProcessor(String config) throws IOException {
     SensorEnrichmentConfig c = JSONUtils.INSTANCE.load(config, SensorEnrichmentConfig.class);
     return new ThreatTriageProcessor(c, StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_CONTEXT());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json b/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
index 77e0808..1037b69 100644
--- a/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
+++ b/metron-platform/metron-integration-test/src/main/config/zookeeper/enrichments/test.json
@@ -61,8 +61,11 @@
     "triageConfig" : {
       "riskLevelRules" : [
         {
+          "name" : "The name of the triage rule",
+          "comment" : "A description of the triage rule",
           "rule" : "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'",
-          "score": 10
+          "score": 10,
+          "reason": "'Reason field'"
         }
       ],
       "aggregator" : "MAX"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-management/src/main/java/org/apache/metron/management/ThreatTriageFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/src/main/java/org/apache/metron/management/ThreatTriageFunctions.java b/metron-platform/metron-management/src/main/java/org/apache/metron/management/ThreatTriageFunctions.java
index 4a28cce..a16dea7 100644
--- a/metron-platform/metron-management/src/main/java/org/apache/metron/management/ThreatTriageFunctions.java
+++ b/metron-platform/metron-management/src/main/java/org/apache/metron/management/ThreatTriageFunctions.java
@@ -75,15 +75,16 @@ public class ThreatTriageFunctions {
       if(triageRules == null) {
         triageRules = new ArrayList<>();
       }
-      String[] headers = new String[] {"Name", "Comment", "Triage Rule", "Score"};
-      String[][] data = new String[triageRules.size()][4];
+      String[] headers = new String[] {"Name", "Comment", "Triage Rule", "Score", "Reason"};
+      String[][] data = new String[triageRules.size()][5];
       int i = 0;
       for(RiskLevelRule rule : triageRules) {
         double d = rule.getScore().doubleValue();
-        String val = d == (long)d ? String.format("%d", (long)d) : String.format("%s", d);
+        String score = d == (long)d ? String.format("%d", (long)d) : String.format("%s", d);
         String name = Optional.ofNullable(rule.getName()).orElse("");
         String comment = Optional.ofNullable(rule.getComment()).orElse("");
-        data[i++]  = new String[] {name, comment, rule.getRule(), val};
+        String reason = Optional.ofNullable(rule.getReason()).orElse("");
+        data[i++]  = new String[] {name, comment, rule.getRule(), score, reason};
       }
       String ret = FlipTable.of(headers, data);
       if(!triageRules.isEmpty()) {
@@ -164,6 +165,9 @@ public class ThreatTriageFunctions {
           if (newRule.containsKey("comment")) {
             ruleToAdd.setComment((String) newRule.get("comment"));
           }
+          if (newRule.containsKey("reason")) {
+            ruleToAdd.setReason((String) newRule.get("reason"));
+          }
           triageRules.add(ruleToAdd);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/b3403720/metron-platform/metron-management/src/test/java/org/apache/metron/management/ThreatTriageFunctionsTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ThreatTriageFunctionsTest.java b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ThreatTriageFunctionsTest.java
index 0c4505e..d1d13e8 100644
--- a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ThreatTriageFunctionsTest.java
+++ b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ThreatTriageFunctionsTest.java
@@ -99,7 +99,7 @@ public class ThreatTriageFunctionsTest {
   public void testAddHasExisting() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10, 'reason' : '2 + 2' } )"
             , toMap("config", configStr
             )
     );
@@ -217,13 +217,13 @@ public class ThreatTriageFunctionsTest {
   }
 
   /**
-\u2554\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\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
-\u2551 Name \u2502 Comment \u2502 Triage Rule \u2502 Score \u2551
-\u2560\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\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2563
-\u2551      \u2502         \u2502 1 < 2       \u2502 10    \u2551
-\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
-\u2551      \u2502         \u2502 1 > 2       \u2502 20    \u2551
-\u255a\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\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u255d
+\u2554\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\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\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\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\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\u2563
+\u2551      \u2502         \u2502 1 < 2       \u2502 10    \u2502 2 + 2  \u2551
+\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
+\u2551      \u2502         \u2502 1 > 2       \u2502 20    \u2502        \u2551
+\u255a\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\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u255d
 
 
 Aggregation: MAX*/
@@ -234,7 +234,7 @@ Aggregation: MAX*/
   public void testPrint() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
+            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10, 'reason' : '2 + 2' }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
             , toMap("config", configStr
             )
     );
@@ -248,11 +248,11 @@ Aggregation: MAX*/
   }
 
   /**
-\u2554\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\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
-\u2551 Name \u2502 Comment \u2502 Triage Rule \u2502 Score \u2551
-\u2560\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\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2563
-\u2551 (empty)                              \u2551
-\u255a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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
+\u2554\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\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\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\u2567\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2567\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\u2563
+\u2551 (empty)                                       \u2551
+\u255a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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
    */
   @Multiline
   static String testPrintEmptyExpected;