You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2017/02/06 20:17:32 UTC

[04/17] incubator-metron git commit: METRON-675: Make Threat Triage rules able to be assigned names and comments closes apache/incubator-metron#426

METRON-675: Make Threat Triage rules able to be assigned names and comments closes apache/incubator-metron#426


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

Branch: refs/heads/Metron_0.3.1
Commit: 75d122d1e4c3c7d7c4bd9e3fe3bdc67a133de463
Parents: ad8724e
Author: cstella <ce...@gmail.com>
Authored: Wed Feb 1 13:46:07 2017 -0500
Committer: cstella <ce...@gmail.com>
Committed: Wed Feb 1 13:46:07 2017 -0500

----------------------------------------------------------------------
 metron-analytics/metron-maas-service/README.md  |  9 +-
 metron-analytics/metron-statistics/README.md    |  9 +-
 .../enrichment/threatintel/RiskLevelRule.java   | 90 +++++++++++++++++++
 .../threatintel/ThreatTriageConfig.java         | 27 ++++--
 .../org/apache/metron/common/writer/test.json   | 11 ++-
 .../SensorEnrichmentUpdateConfigTest.java       |  9 +-
 metron-platform/metron-enrichment/README.md     | 29 +++++-
 .../config/zookeeper/enrichments/snort.json     |  9 +-
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |  2 +-
 .../triage/ThreatTriageProcessor.java           |  7 +-
 .../bolt/ThreatIntelJoinBoltTest.java           | 27 ++++--
 .../threatintel/triage/ThreatTriageTest.java    | 54 ++++++++---
 .../main/config/zookeeper/enrichments/test.json |  9 +-
 metron-platform/metron-management/README.md     | 79 ++++++++--------
 .../management/ThreatTriageFunctions.java       | 86 ++++++++++++------
 .../management/ConfigurationFunctionsTest.java  |  2 +-
 .../management/ThreatTriageFunctionsTest.java   | 95 ++++++++++++--------
 17 files changed, 389 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-analytics/metron-maas-service/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-maas-service/README.md b/metron-analytics/metron-maas-service/README.md
index d1172b1..dd55558 100644
--- a/metron-analytics/metron-maas-service/README.md
+++ b/metron-analytics/metron-maas-service/README.md
@@ -169,9 +169,12 @@ Now that we have a deployed model, let's adjust the configurations for the Squid
   "threatIntel" : {
     "fieldMap":{},
     "triageConfig" : {
-      "riskLevelRules" : {
-        "is_malicious == 'malicious'" : 100
-      },
+      "riskLevelRules" : [
+        {
+          "rule" : "is_malicious == 'malicious'",
+          "score" : 100
+        }
+      ],
       "aggregator" : "MAX"
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-analytics/metron-statistics/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-statistics/README.md b/metron-analytics/metron-statistics/README.md
index 7b26c33..257fd0b 100644
--- a/metron-analytics/metron-statistics/README.md
+++ b/metron-analytics/metron-statistics/README.md
@@ -352,9 +352,12 @@ PROFILE_GET( 'sketchy_mad', 'global', 10, 'MINUTES') ), value)"
     "fieldMap": { },
     "fieldToTypeMap": { },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "parser_score > 3.5" : 10
-      },
+      "riskLevelRules" : [
+        {
+          "rule" : "parser_score > 3.5",
+          "score" : 10
+        }
+      ],
       "aggregator" : "MAX"
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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
new file mode 100644
index 0000000..7bf1d07
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/RiskLevelRule.java
@@ -0,0 +1,90 @@
+/**
+ * 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;
+
+public class RiskLevelRule {
+  String name;
+  String comment;
+  String rule;
+  Number score;
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  public String getRule() {
+    return rule;
+  }
+
+  public void setRule(String rule) {
+    this.rule = rule;
+  }
+
+  public Number getScore() {
+    return score;
+  }
+
+  public void setScore(Number score) {
+    this.score = score;
+  }
+
+  @Override
+  public String toString() {
+    return "RiskLevelRule{" +
+            "name='" + name + '\'' +
+            ", comment='" + comment + '\'' +
+            ", rule='" + rule + '\'' +
+            ", score=" + score +
+            '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RiskLevelRule that = (RiskLevelRule) o;
+
+    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;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name != null ? name.hashCode() : 0;
+    result = 31 * result + (comment != null ? comment.hashCode() : 0);
+    result = 31 * result + (rule != null ? rule.hashCode() : 0);
+    result = 31 * result + (score != null ? score.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 00f0087..c3f5e55 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
@@ -24,24 +24,35 @@ import org.apache.metron.common.aggregator.Aggregators;
 import org.apache.metron.common.stellar.StellarPredicateProcessor;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 
-import java.util.HashMap;
-import java.util.Map;
+import java.util.*;
 
 public class ThreatTriageConfig {
-  private Map<String, Number> riskLevelRules = new HashMap<>();
+  private List<RiskLevelRule> riskLevelRules = new ArrayList<>();
   private Aggregators aggregator = Aggregators.MAX;
   private Map<String, Object> aggregationConfig = new HashMap<>();
 
-  public Map<String, Number> getRiskLevelRules() {
+  public List<RiskLevelRule> getRiskLevelRules() {
     return riskLevelRules;
   }
 
-  public void setRiskLevelRules(Map<String, Number> riskLevelRules) {
-    this.riskLevelRules = riskLevelRules;
+  public void setRiskLevelRules(List<RiskLevelRule> riskLevelRules) {
+    List<RiskLevelRule> rules = new ArrayList<>();
+    Set<String> ruleIndex = new HashSet<>();
     StellarPredicateProcessor processor = new StellarPredicateProcessor();
-    for(String rule : riskLevelRules.keySet()) {
-      processor.validate(rule);
+    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.");
+      }
+      if(ruleIndex.contains(rule.getRule())) {
+        continue;
+      }
+      else {
+        ruleIndex.add(rule.getRule());
+      }
+      processor.validate(rule.getRule());
+      rules.add(rule);
     }
+    this.riskLevelRules = rules;
   }
 
   public Aggregators getAggregator() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-common/src/main/java/org/apache/metron/common/writer/test.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/writer/test.json b/metron-platform/metron-common/src/main/java/org/apache/metron/common/writer/test.json
index 1defcd6..023cd63 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/writer/test.json
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/writer/test.json
@@ -19,10 +19,13 @@
       "ip_dst_addr" : ["malicious_ip"]
     },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "ip_src_addr == '31.24.30.31'" : "Test"
-      },
+      "riskLevelRules" : [
+        {
+          "rule" : "ip_src_addr == '31.24.30.31'",
+          "score" : 10
+        }
+      ],
       "aggregator" : "MAX"
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
index 01a697b..65ec920 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
@@ -47,9 +47,12 @@ public class SensorEnrichmentUpdateConfigTest {
          ,"ip_src_addr" : [ "malicious_ip" ]
                           },
         "triageConfig" : {
-          "riskLevelRules" : {
-            "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
-                             },
+          "riskLevelRules" : [
+            {
+              "rule" : "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))",
+              "score" : 10
+            }
+                             ],
           "aggregator" : "MAX"
                         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-enrichment/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/README.md b/metron-platform/metron-enrichment/README.md
index ba518cb..d08929b 100644
--- a/metron-platform/metron-enrichment/README.md
+++ b/metron-platform/metron-enrichment/README.md
@@ -123,9 +123,27 @@ The `triageConfig` field is also a complex field and it bears some description:
 
 | Field            | Description                                                                                                                                             | Example                                                                  |
 |------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
-| `riskLevelRules` | The mapping of Stellar (see above) queries to a score.                                                                                                  | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
+| `riskLevelRules` | This is a list of rules (represented as Stellar expressions) associated with scores with optional names and comments                                    |  see below|
 | `aggregator`     | An aggregation function that takes all non-zero scores representing the matching queries from `riskLevelRules` and aggregates them into a single score. | `"MAX"`                                                                  |
 
+A risk level rule is of the following format:
+* `name` : The name of the threat triage rule
+* `comment` : A comment describing the rule
+* `rule` : The rule, represented as a Stellar statement
+* `score` : Associated threat triage score for the rule
+
+An example of a rule is as follows:
+```
+    "riskLevelRules" : [ 
+        { 
+          "name" : "is internal"
+        , "comment" : "determines if the destination is internal."
+        , rule" : "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')"
+        , "score" : 10 
+        }
+                       ]
+```
+
 The supported aggregation functions are:
 * `MAX` : The max of all of the associated values for matching queries
 * `MIN` : The min of all of the associated values for matching queries
@@ -177,9 +195,12 @@ An example configuration for the YAF sensor is as follows:
       ]
     },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
-      },
+      "riskLevelRules" : [ 
+        {
+          "rule" : "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'",
+          "score" : 10
+        }
+      ],
       "aggregator" : "MAX"
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
index 5bf49d7..8cd8197 100644
--- a/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
+++ b/metron-platform/metron-enrichment/src/main/config/zookeeper/enrichments/snort.json
@@ -17,9 +17,12 @@
       "ip_dst_addr" : ["malicious_ip"]
     },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
-      },
+      "riskLevelRules" : [
+        {
+          "rule" : "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))",
+          "score" : 10
+        }
+      ],
       "aggregator" : "MAX"
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 6d0ea44..6584a27 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
@@ -136,7 +136,7 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
         ThreatTriageProcessor threatTriageProcessor = new ThreatTriageProcessor(config, functionResolver, stellarContext);
         Double triageLevel = threatTriageProcessor.apply(ret);
         if(LOG.isDebugEnabled()) {
-          String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules().entrySet());
+          String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules());
           LOG.debug("Marked " + sourceType + " as triage level " + triageLevel + " with rules " + rules);
         }
         if(triageLevel != null && triageLevel > 0) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 824e94c..0c88437 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
@@ -20,6 +20,7 @@ package org.apache.metron.threatintel.triage;
 
 import com.google.common.base.Function;
 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.ThreatIntelConfig;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
 import org.apache.metron.common.dsl.*;
@@ -55,9 +56,9 @@ public class ThreatTriageProcessor implements Function<Map, Double> {
     List<Number> scores = new ArrayList<>();
     StellarPredicateProcessor predicateProcessor = new StellarPredicateProcessor();
     VariableResolver resolver = new MapVariableResolver(input, sensorConfig.getConfiguration(), threatIntelConfig.getConfig());
-    for(Map.Entry<String, Number> kv : threatTriageConfig.getRiskLevelRules().entrySet()) {
-      if(predicateProcessor.parse(kv.getKey(), resolver, functionResolver, context)) {
-        scores.add(kv.getValue());
+    for(RiskLevelRule rule : threatTriageConfig.getRiskLevelRules()) {
+      if(predicateProcessor.parse(rule.getRule(), resolver, functionResolver, context)) {
+        scores.add(rule.getScore());
       }
     }
     return threatTriageConfig.getAggregator().aggregate(scores, threatTriageConfig.getAggregationConfig());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 d942d9b..60687d8 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
@@ -89,9 +89,12 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
 
   /**
    * {
-   *  "riskLevelRules" : {
-   *    "enrichedField1 == 'enrichedValue1'" : 10
-   *  },
+   *  "riskLevelRules" : [
+   *   {
+   *    "rule" : "enrichedField1 == 'enrichedValue1'",
+   *    "score" : 10
+   *   }
+   *  ],
    *  "aggregator" : "MAX"
    * }
    */
@@ -105,9 +108,12 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
 
   /**
    * {
-   *  "riskLevelRules" : {
-   *    "enrichedField1 == 'enrichedValue1": 10
-   *  },
+   *  "riskLevelRules" : [
+   *  {
+   *    "rule" : "enrichedField1 == 'enrichedValue1",
+   *    "score" : 10
+   *  }
+   *  ],
    *  "aggregator" : "MAX"
    * }
    */
@@ -126,9 +132,12 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
 
   /**
    * {
-   *   "riskLevelRules": {
-   *      "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))": 10
-   *   },
+   *   "riskLevelRules": [
+   *   {
+   *      "rule" : "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))",
+   *      "score" : 10
+   *   }
+   *   ],
    *   "aggregator": "MAX"
    * }
    */

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 92191b6..d3389af 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
@@ -34,12 +34,26 @@ public class ThreatTriageTest {
    * {
    *  "threatIntel": {
    *    "triageConfig": {
-   *      "riskLevelRules" : {
-   *        "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10,
-   *        "asset.type == 'web'" : 5,
-   *        "user.type == 'normal'  and asset.type == 'web'" : 0,
-   *        "user.type in whitelist" : -1
-   *      },
+   *      "riskLevelRules" : [
+   *        {
+   *          "name" : "rule 1",
+   *          "rule" : "user.type in [ 'admin', 'power' ] and asset.type == 'web'",
+   *          "score" : 10
+   *        },
+   *        {
+   *         "comment" : "web type!",
+   *         "rule" : "asset.type == 'web'",
+   *         "score" : 5
+   *        },
+   *        {
+   *          "rule" : "user.type == 'normal'  and asset.type == 'web'",
+   *          "score" : 0
+   *        },
+   *        {
+   *          "rule" : "user.type in whitelist",
+   *          "score" : -1
+   *        }
+   *      ],
    *      "aggregator" : "MAX"
    *    },
    *    "config": {
@@ -115,11 +129,20 @@ public class ThreatTriageTest {
    * {
    *  "threatIntel": {
    *  "triageConfig": {
-   *    "riskLevelRules" : {
-   *      "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10,
-   *      "asset.type == 'web'" : 5,
-   *      "user.type == 'normal' and asset.type == 'web'" : 0
-   *     },
+   *    "riskLevelRules" : [
+   *      {
+   *        "rule" : "user.type in [ 'admin', 'power' ] and asset.type == 'web'",
+   *        "score" : 10
+   *      },
+   *      {
+   *        "rule" : "asset.type == 'web'",
+   *        "score" : 5
+   *      },
+   *      {
+   *        "rule" : "user.type == 'normal' and asset.type == 'web'",
+   *        "score" : 0
+   *      }
+   *     ],
    *     "aggregator" : "POSITIVE_MEAN"
    *    }
    *  }
@@ -167,9 +190,12 @@ public class ThreatTriageTest {
    * {
    *    "threatIntel" : {
    *      "triageConfig": {
-   *        "riskLevelRules": {
-   *          "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
-   *        },
+   *        "riskLevelRules": [
+   *          {
+   *            "rule" : "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))",
+   *            "score" : 10
+   *          }
+   *        ],
    *        "aggregator" : "MAX"
    *      }
    *    }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 d99f741..77e0808 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
@@ -59,9 +59,12 @@
       ]
     },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
-      },
+      "riskLevelRules" : [
+        {
+          "rule" : "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'",
+          "score": 10
+        }
+      ],
       "aggregator" : "MAX"
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-management/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/README.md b/metron-platform/metron-management/README.md
index cf922ba..f3fea80 100644
--- a/metron-platform/metron-management/README.md
+++ b/metron-platform/metron-management/README.md
@@ -166,7 +166,7 @@ The functions are split roughly into a few sections:
   * Returns: The String representation of the config in zookeeper
 
 
-### Enrichment Functions
+### Indexing Functions
 
 * `INDEXING_SET_BATCH`
   * Description: Set batch size
@@ -189,6 +189,9 @@ The functions are split roughly into a few sections:
     * writer - The writer to update (e.g. elasticsearch, solr or hdfs)
     * sensor - sensor name
   * Returns: The String representation of the config in zookeeper
+
+### Enrichment Functions
+
 * `ENRICHMENT_STELLAR_TRANSFORM_ADD`
   * Description: Add stellar field transformation.
   * Input:
@@ -219,6 +222,7 @@ The functions are split roughly into a few sections:
   * Input:
     * sensorConfig - Sensor config to add transformation to.
     * stellarTransforms - A Map associating stellar rules to scores
+    * triageRules - Map (or list of Maps) representing a triage rule.  It must contain 'rule' and 'score' keys, the stellar expression for the rule and triage score respectively.  It may contain 'name' and 'comment', the name of the rule and comment associated with the rule respectively."
   * Returns: The String representation of the threat triage rules
 * `THREAT_TRIAGE_PRINT`
   * Description: Retrieve stellar enrichment transformations.
@@ -229,7 +233,7 @@ The functions are split roughly into a few sections:
   * Description: Remove stellar threat triage rule(s).
   * Input:
     * sensorConfig - Sensor config to add transformation to.
-    * stellarTransforms - A list of stellar rules to remove
+    * rules - A list of stellar rules or rule names to remove
   * Returns: The String representation of the enrichment config
 * `THREAT_TRIAGE_SET_AGGREGATOR`
   * Description: Set the threat triage aggregator.
@@ -529,8 +533,6 @@ Functions loaded, you may refer to functions now...
 [Stellar]>>> # Just to make sure it looks right, we can view the JSON
 [Stellar]>>> squid_enrichment_config
 {
-  "index" : "squid",
-  "batchSize" : 0,
   "enrichment" : {
     "fieldMap" : { },
     "fieldToTypeMap" : { },
@@ -541,16 +543,13 @@ Functions loaded, you may refer to functions now...
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : { },
+      "riskLevelRules" : [ ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }
   },
   "configuration" : { }
 }
-[Stellar]>>> # Wait, that batch size looks terrible.  That is because it did not exist in zookeeper, so it is the default.
-[Stellar]>>> # We can correct it, thankfully. 
-[Stellar]>>> squid_enrichment_config := INDEXING_SET_BATCH( squid_enrichment_config, 100)
 [Stellar]>>> # Now that we have a config, we can add an enrichment to the Stellar adapter
 [Stellar]>>> # We should make sure that the current enrichment does not have any already
 [Stellar]>>> ?ENRICHMENT_STELLAR_TRANSFORM_PRINT
@@ -659,7 +658,7 @@ Returns: The String representation of the config in zookeeper
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : { },
+      "riskLevelRules" : [ ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }
@@ -689,7 +688,7 @@ Returns: The String representation of the config in zookeeper
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : { },
+      "riskLevelRules" : [ ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }
@@ -741,7 +740,7 @@ Returns: The String representation of the config in zookeeper
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : { },
+      "riskLevelRules" : [ ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }
@@ -765,8 +764,6 @@ Please note that functions are loading lazily in the background and will be unav
 26751 [Thread-1] INFO  o.r.Reflections - Reflections took 24407 ms to scan 22 urls, producing 17898 keys and 121520 values 
 26828 [Thread-1] INFO  o.a.m.c.d.FunctionResolverSingleton - Found 84 Stellar Functions...
 Functions loaded, you may refer to functions now...
-[Stellar]>>> # Just as in the previous example, we should adjust the batch size
-[Stellar]>>> squid_enrichment_config := INDEXING_SET_BATCH( squid_enrichment_config, 100)
 [Stellar]>>> # We should not have any threat triage rules
 [Stellar]>>> THREAT_TRIAGE_PRINT(squid_enrichment_config)
 \u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
@@ -849,20 +846,19 @@ Returns: A Map associated with the indicator and enrichment type.  Empty otherwi
 [Stellar]>>> non_us := whois_info.home_country != 'US'
 [Stellar]>>> is_local := IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')
 [Stellar]>>> is_both := whois_info.home_country != 'US' && IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')
-[Stellar]>>> rules := { SHELL_GET_EXPRESSION('non_us') : 10, SHELL_GET_EXPRESSION('is_local') : 20, SHELL_GET_EXPRESSION('is_both') : 50 }
+[Stellar]>>> rules := [ { 'name' : 'is non-us', 'rule' : SHELL_GET_EXPRESSION('non_us'), 'score' : 10 } , { 'name' : 'is local', 'rule' : SHELL_GET_EXPRESSION('is_local '), 'score' : 20 } , { 'name' : 'both non-us and local', 'comment' : 'union of both rules.',  'rule' : SHELL_GET_EXPRESSION('is_both'), 'score' : 50 } ]  
 [Stellar]>>> # Now that we have our rules staged, we can add them to our config.
 [Stellar]>>> squid_enrichment_config_new := THREAT_TRIAGE_ADD( squid_enrichment_config_new, rules )
 [Stellar]>>> THREAT_TRIAGE_PRINT(squid_enrichment_config_new)
-\u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u2557
-\u2551 Triage Rule                                                                                                       \u2502 Score \u2551
-\u2560\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u2563
-\u2551 whois_info.home_country != 'US' && IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21') \u2502 50    \u2551
-\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
-\u2551 IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')                                    \u2502 20    \u2551
-\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
-\u2551 whois_info.home_country != 'US'                                                                                   \u2502 10    \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\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u255d
-
+\u2554\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\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\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u2557
+\u2551 Name                  \u2502 Comment              \u2502 Triage Rule                                                                                                       \u2502 Score \u2551
+\u2560\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\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\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u2563
+\u2551 is non-us             \u2502                      \u2502 whois_info.home_country != 'US'                                                                                   \u2502 10    \u2551
+\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
+\u2551 is local              \u2502                      \u2502 IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')                                    \u2502 20    \u2551
+\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
+\u2551 both non-us and local \u2502 union of both rules. \u2502 whois_info.home_country != 'US' && IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21') \u2502 50    \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\u2567\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\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\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\u255d
 
 Aggregation: MAX
 [Stellar]>>> # Looks good, we can push the configs up
@@ -870,8 +866,6 @@ Aggregation: MAX
 [Stellar]>>> # And admire the resulting JSON that you did not have to edit directly.
 [Stellar]>>> CONFIG_GET('ENRICHMENT', 'squid')
 {
-  "index" : "squid",
-  "batchSize" : 100,
   "enrichment" : {
     "fieldMap" : {
       "stellar" : {
@@ -894,11 +888,20 @@ Aggregation: MAX
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : {
-        "whois_info.home_country != 'US' && IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')" : 50.0,
-        "IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')" : 20.0,
-        "whois_info.home_country != 'US'" : 10.0
-      },
+      "riskLevelRules" : [ {
+        "name" : "is non-us",
+        "rule" : "whois_info.home_country != 'US'",
+        "score" : 10.0
+      }, {
+        "name" : "is local",
+        "rule" : "IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')",
+        "score" : 20.0
+      }, {
+        "name" : "both non-us and local",
+        "comment" : "union of both rules.",
+        "rule" : "whois_info.home_country != 'US' && IN_SUBNET( if IS_IP(ip_src_addr) then ip_src_addr else NULL, '192.168.0.0/21')",
+        "score" : 50.0
+      } ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }
@@ -909,19 +912,17 @@ Aggregation: MAX
 [Stellar]>>> squid_enrichment_config_new := THREAT_TRIAGE_REMOVE( squid_enrichment_config_new, [ SHELL_GET_EXPRESSION('non_us') , SHELL_GET_EXPRESSION('is_local') , SHELL_GET_EXPRES 
 SION('is_both') ] )
 [Stellar]>>> THREAT_TRIAGE_PRINT(squid_enrichment_config_new)
-\u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
-\u2551 Triage Rule \u2502 Score \u2551
-\u2560\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\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\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
 
 [Stellar]>>> # and push configs
 [Stellar]>>> CONFIG_PUT('ENRICHMENT', squid_enrichment_config_new, 'squid')
 [Stellar]>>> # And admire the resulting JSON that is devoid of threat triage rules.
 [Stellar]>>> CONFIG_GET('ENRICHMENT', 'squid')
 {
-  "index" : "squid",
-  "batchSize" : 100,
   "enrichment" : {
     "fieldMap" : {
       "stellar" : {
@@ -944,7 +945,7 @@ SION('is_both') ] )
     "fieldToTypeMap" : { },
     "config" : { },
     "triageConfig" : {
-      "riskLevelRules" : { },
+      "riskLevelRules" : [ ],
       "aggregator" : "MAX",
       "aggregationConfig" : { }
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 966f281..4a28cce 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
@@ -19,9 +19,12 @@ package org.apache.metron.management;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
 import com.jakewharton.fliptables.FlipTable;
 import org.apache.log4j.Logger;
 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.ThreatIntelConfig;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
 import org.apache.metron.common.dsl.Context;
@@ -32,9 +35,7 @@ import org.apache.metron.common.utils.ConversionUtils;
 import org.apache.metron.common.utils.JSONUtils;
 
 import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
 import static org.apache.metron.management.EnrichmentConfigFunctions.getConfig;
@@ -70,17 +71,19 @@ public class ThreatTriageFunctions {
       if(triageConfig == null) {
         return "";
       }
-      Map<String, Number> triageRules = triageConfig.getRiskLevelRules();
+      List<RiskLevelRule> triageRules = triageConfig.getRiskLevelRules();
       if(triageRules == null) {
-        triageRules = new LinkedHashMap<>();
+        triageRules = new ArrayList<>();
       }
-      String[] headers = new String[] {"Triage Rule", "Score"};
-      String[][] data = new String[triageRules.size()][2];
+      String[] headers = new String[] {"Name", "Comment", "Triage Rule", "Score"};
+      String[][] data = new String[triageRules.size()][4];
       int i = 0;
-      for(Map.Entry<String, Number> kv : triageRules.entrySet()) {
-        double d = kv.getValue().doubleValue();
+      for(RiskLevelRule rule : triageRules) {
+        double d = rule.getScore().doubleValue();
         String val = d == (long)d ? String.format("%d", (long)d) : String.format("%s", d);
-        data[i++]  = new String[] {kv.getKey(), val};
+        String name = Optional.ofNullable(rule.getName()).orElse("");
+        String comment = Optional.ofNullable(rule.getComment()).orElse("");
+        data[i++]  = new String[] {name, comment, rule.getRule(), val};
       }
       String ret = FlipTable.of(headers, data);
       if(!triageRules.isEmpty()) {
@@ -107,7 +110,9 @@ public class ThreatTriageFunctions {
           ,name = "ADD"
           ,description = "Add a threat triage rule."
           ,params = {"sensorConfig - Sensor config to add transformation to."
-                    ,"stellarTransforms - A Map associating stellar rules to scores"
+                    ,"triageRules - A Map (or list of Maps) representing a triage rule.  It must contain 'rule' and 'score' keys, " +
+                      "the stellar expression for the rule and triage score respectively.  " +
+                      "It may contain 'name' and 'comment', the name of the rule and comment associated with the rule respectively."
                     }
           ,returns = "The String representation of the threat triage rules"
           )
@@ -133,19 +138,36 @@ public class ThreatTriageFunctions {
         triageConfig = new ThreatTriageConfig();
         tiConfig.setTriageConfig(triageConfig);
       }
-      Map<String, Number> triageRules = triageConfig.getRiskLevelRules();
+      List<RiskLevelRule> triageRules = triageConfig.getRiskLevelRules();
       if(triageRules == null) {
-        triageRules = new LinkedHashMap<>();
-        triageConfig.setRiskLevelRules(triageRules);
+        triageRules = new ArrayList<>();
+      }
+      Object newRuleObj = args.get(1);
+      List<Map<String, Object>> newRules = new ArrayList<>();
+      if(newRuleObj != null && newRuleObj instanceof List) {
+        newRules = (List<Map<String, Object>>) newRuleObj;
+      }
+      else if(newRuleObj != null && newRuleObj instanceof Map) {
+        newRules.add((Map<String, Object>) newRuleObj);
+      }
+      else if(newRuleObj != null) {
+        throw new IllegalStateException("triageRule must be either a Map representing a single rule or a List of rules.");
       }
-      Map<String, Object> newRules = (Map<String, Object>) args.get(1);
-      for(Map.Entry<String, Object> kv : newRules.entrySet()) {
-        if(kv.getKey() == null || kv.getKey().equals("null")) {
-          continue;
+      for(Map<String, Object> newRule : newRules) {
+        if(!(newRule == null || !newRule.containsKey("rule") || !newRule.containsKey("score"))) {
+          RiskLevelRule ruleToAdd = new RiskLevelRule();
+          ruleToAdd.setRule((String) newRule.get("rule"));
+          ruleToAdd.setScore(ConversionUtils.convert(newRule.get("score"), Double.class));
+          if (newRule.containsKey("name")) {
+            ruleToAdd.setName((String) newRule.get("name"));
+          }
+          if (newRule.containsKey("comment")) {
+            ruleToAdd.setComment((String) newRule.get("comment"));
+          }
+          triageRules.add(ruleToAdd);
         }
-        Double ret = ConversionUtils.convert(kv.getValue(), Double.class);
-        triageConfig.getRiskLevelRules().put(kv.getKey(), ret);
       }
+      triageConfig.setRiskLevelRules(triageRules);
       try {
         return JSONUtils.INSTANCE.toJSON(configObj, true);
       } catch (JsonProcessingException e) {
@@ -171,7 +193,7 @@ public class ThreatTriageFunctions {
           ,name = "REMOVE"
           ,description = "Remove stellar threat triage rule(s)."
           ,params = {"sensorConfig - Sensor config to add transformation to."
-                    ,"stellarTransforms - A list of stellar rules to remove"
+                    ,"rules - A list of stellar rules or rule names to remove"
                     }
           ,returns = "The String representation of the enrichment config"
           )
@@ -197,14 +219,22 @@ public class ThreatTriageFunctions {
         triageConfig = new ThreatTriageConfig();
         tiConfig.setTriageConfig(triageConfig);
       }
-      Map<String, Number> triageRules = triageConfig.getRiskLevelRules();
+      List<RiskLevelRule> triageRules = triageConfig.getRiskLevelRules();
       if(triageRules == null) {
-        triageRules = new LinkedHashMap<>();
+        triageRules = new ArrayList<>();
         triageConfig.setRiskLevelRules(triageRules);
       }
-      List<String> rulesToRemove = (List<String>) args.get(1);
-      for(String rule : rulesToRemove) {
-        triageConfig.getRiskLevelRules().remove(rule);
+
+      Set<String> toRemove = new HashSet<>(Optional.ofNullable((List<String>) args.get(1)).orElse(new ArrayList<>()));
+      for (Iterator<RiskLevelRule> it = triageRules.iterator();it.hasNext();){
+        RiskLevelRule rule = it.next();
+        boolean remove = toRemove.contains(rule.getRule());
+        if(!remove && rule.getName() != null) {
+          remove = toRemove.contains(rule.getName());
+        }
+        if(remove) {
+          it.remove();
+        }
       }
       try {
         return JSONUtils.INSTANCE.toJSON(configObj, true);
@@ -260,9 +290,9 @@ public class ThreatTriageFunctions {
         triageConfig = new ThreatTriageConfig();
         tiConfig.setTriageConfig(triageConfig);
       }
-      Map<String, Number> triageRules = triageConfig.getRiskLevelRules();
+      List<RiskLevelRule> triageRules = triageConfig.getRiskLevelRules();
       if(triageRules == null) {
-        triageRules = new LinkedHashMap<>();
+        triageRules = new ArrayList<>();
         triageConfig.setRiskLevelRules(triageRules);
       }
       String aggregator = (String) args.get(1);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java
index 794f208..ee6a362 100644
--- a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java
+++ b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java
@@ -131,7 +131,7 @@ public class ConfigurationFunctionsTest {
         "fieldToTypeMap" : { },
         "config" : { },
         "triageConfig" : {
-          "riskLevelRules" : { },
+          "riskLevelRules" : [ ],
           "aggregator" : "MAX",
           "aggregationConfig" : { }
         }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/75d122d1/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 0383a54..0c4505e 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
@@ -20,6 +20,7 @@ package org.apache.metron.management;
 import com.google.common.collect.ImmutableMap;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.ParseException;
 import org.apache.metron.common.dsl.StellarFunctions;
@@ -30,6 +31,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static org.apache.metron.management.EnrichmentConfigFunctionsTest.emptyTransformationsConfig;
@@ -54,7 +56,7 @@ public class ThreatTriageFunctionsTest {
             .build();
   }
 
-  public static Map<String, Number> getTriageRules(String config) {
+  public static List<RiskLevelRule> getTriageRules(String config) {
     SensorEnrichmentConfig sensorConfig = (SensorEnrichmentConfig) ENRICHMENT.deserialize(config);
     return sensorConfig.getThreatIntel().getTriageConfig().getRiskLevelRules();
   }
@@ -81,41 +83,48 @@ public class ThreatTriageFunctionsTest {
   public void testAddEmpty() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 } )"
             , toMap("config", configStr
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(1, triageRules.size());
-    Assert.assertEquals(10.0, triageRules.get(variables.get("less").getExpression()).doubleValue(), 1e-6 );
+    RiskLevelRule rule = triageRules.get(0);
+    Assert.assertEquals(variables.get("less").getExpression(), rule.getRule() );
+    Assert.assertEquals(10.0, rule.getScore().doubleValue(), 1e-6 );
   }
 
   @Test
   public void testAddHasExisting() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 } )"
             , toMap("config", configStr
             )
     );
 
     newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('greater') : 20 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } )"
             , toMap("config",newConfig
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(2, triageRules.size());
-    Assert.assertEquals(10.0, triageRules.get(variables.get("less").getExpression()).doubleValue(), 1e-6 );
-    Assert.assertEquals(20.0, triageRules.get(variables.get("greater").getExpression()).doubleValue(), 1e-6 );
+    RiskLevelRule less = triageRules.get(0);
+    Assert.assertEquals(variables.get("less").getExpression(), less.getRule() );
+    Assert.assertEquals(10.0, less.getScore().doubleValue(), 1e-6 );
+
+    RiskLevelRule greater = triageRules.get(1);
+    Assert.assertEquals(variables.get("greater").getExpression(), greater.getRule() );
+    Assert.assertEquals(20.0, greater.getScore().doubleValue(), 1e-6 );
   }
 
-  @Test
+  @Test(expected=ParseException.class)
   public void testAddMalformed() {
     Object o = run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('foo') : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule': SHELL_GET_EXPRESSION('foo'), 'score' : 10 } )"
             , toMap("config", configStr
             )
     );
@@ -125,26 +134,28 @@ public class ThreatTriageFunctionsTest {
   @Test
   public void testAddDuplicate() {
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 } )"
             , toMap("config", configStr
             )
     );
 
     newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10 } )"
+            "THREAT_TRIAGE_ADD(config, { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 } )"
             , toMap("config",newConfig
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(1, triageRules.size());
-    Assert.assertEquals(10.0, triageRules.get(variables.get("less").getExpression()).doubleValue(), 1e-6 );
+    RiskLevelRule rule = triageRules.get(0);
+    Assert.assertEquals(variables.get("less").getExpression(), rule.getRule() );
+    Assert.assertEquals(10.0, rule.getScore().doubleValue(), 1e-6 );
   }
 
   @Test
   public void testRemove() {
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10, SHELL_GET_EXPRESSION('greater') : 20 } )"
+            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
             , toMap("config", configStr
             )
     );
@@ -155,17 +166,18 @@ public class ThreatTriageFunctionsTest {
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(1, triageRules.size());
-    Assert.assertEquals(10.0, triageRules.get(variables.get("less").getExpression()).doubleValue(), 1e-6 );
+    RiskLevelRule rule = triageRules.get(0);
+    Assert.assertEquals(variables.get("less").getExpression(), rule.getRule() );
+    Assert.assertEquals(10.0, rule.getScore().doubleValue(), 1e-6 );
   }
 
   @Test
   public void testRemoveMultiple() {
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10, SHELL_GET_EXPRESSION('greater') : 20 } )"
-            , toMap("config", configStr
-            )
+            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
+            , toMap("config", configStr )
     );
 
     newConfig = (String) run(
@@ -174,7 +186,7 @@ public class ThreatTriageFunctionsTest {
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(0, triageRules.size());
   }
 
@@ -182,7 +194,7 @@ public class ThreatTriageFunctionsTest {
   public void testRemoveMissing() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10, SHELL_GET_EXPRESSION('greater') : 20 } )"
+            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
             , toMap("config", configStr
             )
     );
@@ -193,20 +205,25 @@ public class ThreatTriageFunctionsTest {
             )
     );
 
-    Map<String, Number> triageRules = getTriageRules(newConfig);
+    List<RiskLevelRule> triageRules = getTriageRules(newConfig);
     Assert.assertEquals(2, triageRules.size());
-    Assert.assertEquals(10.0, triageRules.get(variables.get("less").getExpression()).doubleValue(), 1e-6 );
-    Assert.assertEquals(20.0, triageRules.get(variables.get("greater").getExpression()).doubleValue(), 1e-6 );
+    RiskLevelRule less = triageRules.get(0);
+    Assert.assertEquals(variables.get("less").getExpression(), less.getRule() );
+    Assert.assertEquals(10.0, less.getScore().doubleValue(), 1e-6 );
+
+    RiskLevelRule greater = triageRules.get(1);
+    Assert.assertEquals(variables.get("greater").getExpression(), greater.getRule() );
+    Assert.assertEquals(20.0, greater.getScore().doubleValue(), 1e-6 );
   }
 
   /**
-\u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
-\u2551 Triage Rule \u2502 Score \u2551
-\u2560\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2563
-\u2551 1 > 2       \u2502 20    \u2551
-\u255f\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u253c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2562
-\u2551 1 < 2       \u2502 10    \u2551
-\u255a\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\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
 
 
 Aggregation: MAX*/
@@ -217,7 +234,7 @@ Aggregation: MAX*/
   public void testPrint() {
 
     String newConfig = (String) run(
-            "THREAT_TRIAGE_ADD(config, { SHELL_GET_EXPRESSION('less') : 10, SHELL_GET_EXPRESSION('greater') : 20 } )"
+            "THREAT_TRIAGE_ADD(config, [ { 'rule' : SHELL_GET_EXPRESSION('less'), 'score' : 10 }, { 'rule' : SHELL_GET_EXPRESSION('greater'), 'score' : 20 } ] )"
             , toMap("config", configStr
             )
     );
@@ -231,11 +248,11 @@ Aggregation: MAX*/
   }
 
   /**
-\u2554\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2564\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2557
-\u2551 Triage Rule \u2502 Score \u2551
-\u2560\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\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\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
    */
   @Multiline
   static String testPrintEmptyExpected;