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 2016/05/10 16:53:53 UTC

[1/5] incubator-metron git commit: METRON-141: The ability to do threat triage closes apache/incubator-metron#108

Repository: incubator-metron
Updated Branches:
  refs/heads/master 743f37b28 -> deed21e6e


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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
new file mode 100644
index 0000000..90a4c9b
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
@@ -0,0 +1,136 @@
+/**
+ * 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.threatintel.triage;
+
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
+import org.apache.metron.common.utils.JSONUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+public class ThreatTriageTest {
+  /**
+   {
+    "riskLevelRules" : {
+        "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10
+       ,"asset.type == 'web'" : 5
+       ,"user.type == 'normal'  and asset.type == 'web'" : 0
+                          }
+   ,"aggregator" : "MAX"
+   }
+   */
+  @Multiline
+  public static String smokeTestProcessorConfig;
+
+  private static ThreatTriageProcessor getProcessor(String config) throws IOException {
+    ThreatTriageConfig c = JSONUtils.INSTANCE.load(config, ThreatTriageConfig.class);
+    return new ThreatTriageProcessor(c);
+  }
+
+  @Test
+  public void smokeTest() throws Exception {
+    ThreatTriageProcessor threatTriageProcessor = getProcessor(smokeTestProcessorConfig);
+    Assert.assertEquals("Expected a score of 0"
+                       , 0d
+                       ,new ThreatTriageProcessor(new ThreatTriageConfig()).apply(new HashMap<Object, Object>() {{
+                          put("user.type", "admin");
+                          put("asset.type", "web");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+    Assert.assertEquals("Expected a score of 10"
+                       , 10d
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "admin");
+                          put("asset.type", "web");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+    Assert.assertEquals("Expected a score of 5"
+                       , 5d
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "normal");
+                          put("asset.type", "web");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+    Assert.assertEquals("Expected a score of 0"
+                       , 0d
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "foo");
+                          put("asset.type", "bar");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+  }
+
+  /**
+   {
+    "riskLevelRules" : {
+        "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10
+       ,"asset.type == 'web'" : 5
+       ,"user.type == 'normal'  and asset.type == 'web'" : 0
+                          }
+   ,"aggregator" : "POSITIVE_MEAN"
+   }
+   */
+  @Multiline
+  public static String positiveMeanProcessorConfig;
+
+  @Test
+  public void positiveMeanAggregationTest() throws Exception {
+
+    ThreatTriageProcessor threatTriageProcessor = getProcessor(positiveMeanProcessorConfig);
+    Assert.assertEquals("Expected a score of 0"
+                       , 5d
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "normal");
+                          put("asset.type", "web");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+    Assert.assertEquals("Expected a score of 7.5"
+                       , (10 + 5)/2.0
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "admin");
+                          put("asset.type", "web");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+
+    Assert.assertEquals("Expected a score of 0"
+                       , 0d
+                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
+                          put("user.type", "foo");
+                          put("asset.type", "bar");
+                                        }}
+                                        )
+                       , 1e-10
+                       );
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
index 3f21c0d..ec04fe0 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/BaseIntegrationTest.java
@@ -20,7 +20,7 @@ package org.apache.metron.integration;
 import com.google.common.base.Function;
 import org.apache.metron.TestConstants;
 import org.apache.metron.integration.components.KafkaWithZKComponent;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 
 import javax.annotation.Nullable;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
index 21019c3..b392448 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/EnrichmentIntegrationTest.java
@@ -354,7 +354,9 @@ public abstract class EnrichmentIntegrationTest extends BaseIntegrationTest {
             ) {
       //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.get("is_alert"), "true");
+      Assert.assertEquals((double)indexedDoc.get("threat.triage.level"), 10d, 1e-7);
     }
     else {
       //For YAF this is the case, but if we do snort later on, this will be invalid.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
index a172760..ace428d 100644
--- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
+++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/utils/SampleUtil.java
@@ -19,7 +19,7 @@ package org.apache.metron.integration.utils;
 
 import org.apache.metron.TestConstants;
 import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 
 import java.io.IOException;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
index 8886495..422200c 100644
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/bro.json
@@ -1,19 +1,29 @@
 {
   "index": "bro",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment" : {
+    "fieldMap": {
+      "geo": [
+        "ip_dst_addr",
+        "ip_src_addr"
+      ],
+      "host": [
+        "host"
+      ]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_dst_addr" : [ "malicious_ip" ]
-    ,"ip_src_addr" : [ "malicious_ip" ]
+  "threatIntel" : {
+    "fieldMap":
+    {
+      "hbaseThreatIntel": [
+        "ip_dst_addr",
+        "ip_src_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_dst_addr": ["malicious_ip"]
+    , "ip_src_addr": ["malicious_ip"]
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
index 82c7c5e..b4308ca 100644
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/pcap.json
@@ -1,13 +1,24 @@
 {
   "index": "pcap",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"]
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "ip": ["ip_src_addr", "ip_dst_addr"]
+  "threatIntel": {
+    "fieldMap": {
+      "ip": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
index b7cc22b..4d0d366 100644
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/snort.json
@@ -1,19 +1,32 @@
 {
   "index": "snort",
   "batchSize": 1,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_dst_addr",
+        "ip_src_addr"
+      ],
+      "host": [
+        "host"
+      ]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_dst_addr" : [ "malicious_ip" ]
-    ,"ip_src_addr" : [ "malicious_ip" ]
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": [
+        "ip_dst_addr",
+        "ip_src_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_dst_addr": [
+        "malicious_ip"
+      ],
+      "ip_src_addr": [
+        "malicious_ip"
+      ]
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
index 98da265..d72851e 100644
--- a/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
+++ b/metron-platform/metron-integration-test/src/main/resources/sample/config/sensors/yaf.json
@@ -1,25 +1,51 @@
 {
   "index": "yaf",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"],
-    "hbaseEnrichment" : ["ip_src_addr", "ip_dst_addr"]
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "hbaseEnrichment": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
+  ,"fieldToTypeMap": {
+      "ip_src_addr": [
+        "playful_classification"
+      ],
+      "ip_dst_addr": [
+        "playful_classification"
+      ]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_src_addr" : ["malicious_ip"],
-    "ip_dst_addr" : ["malicious_ip"]
-  },
-  "fieldToEnrichmentTypeMap":
-  {
-    "ip_src_addr" : ["playful_classification"],
-    "ip_dst_addr" : ["playful_classification"]
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr": [
+        "malicious_ip"
+      ],
+      "ip_dst_addr": [
+        "malicious_ip"
+      ]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
+      },
+      "aggregator" : "MAX"
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
index 4dfdad5..e878912 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrEnrichmentIntegrationTest.java
@@ -26,7 +26,7 @@ import org.apache.metron.integration.Processor;
 import org.apache.metron.integration.ReadinessState;
 import org.apache.metron.solr.integration.components.SolrComponent;
 import org.apache.metron.integration.utils.SampleUtil;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.utils.JSONUtils;
 
 import javax.annotation.Nullable;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/style/checkstyle.xml
----------------------------------------------------------------------
diff --git a/metron-platform/style/checkstyle.xml b/metron-platform/style/checkstyle.xml
index 5f8b5c5..6a6b427 100644
--- a/metron-platform/style/checkstyle.xml
+++ b/metron-platform/style/checkstyle.xml
@@ -19,11 +19,17 @@
 
 <module name="Checker">
   <property name="localeLanguage" value="en"/>
-
+  <module name="TreeWalker">
+    <module name="FileContentsHolder"/>
+  </module>
+  <module name="SuppressionCommentFilter"/>
+  <module name="SuppressionFilter">
+    <property name="file" value="style/suppressions.xml"/>
+  </module>
   <!-- Verify that EVERY source file has the appropriate license -->
   <module name="Header">
     <property name="headerFile" value="style/LICENSE.java"/>
-    <property name="fileExtensions" value="java"/>
+    <property name="fileExtensions" value="java, g4"/>
   </module>
   <module name="Header">
     <property name="headerFile" value="style/LICENSE.config"/>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/style/suppressions.xml
----------------------------------------------------------------------
diff --git a/metron-platform/style/suppressions.xml b/metron-platform/style/suppressions.xml
new file mode 100644
index 0000000..be02dda
--- /dev/null
+++ b/metron-platform/style/suppressions.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0"?>
+<!--
+   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.
+  -->
+
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+    <suppress checks="HeaderCheck"
+              files="org[\\/]apache[\\/]metron[\\/]common[\\/]query[\\/]generated[\\/].*.java"
+              />
+</suppressions>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fe1ecd8..762abe6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,6 +39,7 @@
                         <exclude>**/README.md</exclude>
                         <exclude>**/VERSION</exclude>
                         <exclude>**/*.json</exclude>
+                        <exclude>**/*.tokens</exclude>
                         <exclude>**/*.log</exclude>
                         <exclude>**/*.template</exclude>
                         <exclude>**/.*</exclude>


[4/5] incubator-metron git commit: METRON-141: The ability to do threat triage closes apache/incubator-metron#108

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentConfig.java
new file mode 100644
index 0000000..562a928
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentConfig.java
@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatIntelConfig;
+import org.apache.metron.common.utils.JSONUtils;
+
+import java.io.IOException;
+
+public class SensorEnrichmentConfig {
+
+  private String index;
+  private int batchSize;
+  private EnrichmentConfig enrichment = new EnrichmentConfig();
+  private ThreatIntelConfig threatIntel = new ThreatIntelConfig();
+
+  public EnrichmentConfig getEnrichment() {
+    return enrichment;
+  }
+
+  public void setEnrichment(EnrichmentConfig enrichment) {
+    this.enrichment = enrichment;
+  }
+
+  public ThreatIntelConfig getThreatIntel() {
+    return threatIntel;
+  }
+
+  public void setThreatIntel(ThreatIntelConfig threatIntel) {
+    this.threatIntel = threatIntel;
+  }
+
+  public String getIndex() {
+    return index;
+  }
+
+  public void setIndex(String index) {
+    this.index = index;
+  }
+
+
+  public int getBatchSize() {
+    return batchSize;
+  }
+
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override
+  public String toString() {
+    return "SensorEnrichmentConfig{" +
+            "index='" + index + '\'' +
+            ", batchSize=" + batchSize +
+            ", enrichment=" + enrichment +
+            ", threatIntel=" + threatIntel +
+            '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    SensorEnrichmentConfig that = (SensorEnrichmentConfig) o;
+
+    if (getBatchSize() != that.getBatchSize()) return false;
+    if (getIndex() != null ? !getIndex().equals(that.getIndex()) : that.getIndex() != null) return false;
+    if (getEnrichment() != null ? !getEnrichment().equals(that.getEnrichment()) : that.getEnrichment() != null)
+      return false;
+    return getThreatIntel() != null ? getThreatIntel().equals(that.getThreatIntel()) : that.getThreatIntel() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getIndex() != null ? getIndex().hashCode() : 0;
+    result = 31 * result + getBatchSize();
+    result = 31 * result + (getEnrichment() != null ? getEnrichment().hashCode() : 0);
+    result = 31 * result + (getThreatIntel() != null ? getThreatIntel().hashCode() : 0);
+    return result;
+  }
+
+  public static SensorEnrichmentConfig fromBytes(byte[] config) throws IOException {
+    return JSONUtils.INSTANCE.load(new String(config), SensorEnrichmentConfig.class);
+  }
+
+  public String toJSON() throws JsonProcessingException {
+    return JSONUtils.INSTANCE.toJSON(this, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentUpdateConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentUpdateConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentUpdateConfig.java
new file mode 100644
index 0000000..99e681a
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/SensorEnrichmentUpdateConfig.java
@@ -0,0 +1,209 @@
+/**
+ * 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;
+
+import com.google.common.base.Joiner;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+public class SensorEnrichmentUpdateConfig {
+
+  protected static final Logger _LOG = LoggerFactory.getLogger(SensorEnrichmentUpdateConfig.class);
+  public static class FieldList {
+    Type type;
+    Map<String, List<String>> fieldToEnrichmentTypes;
+
+    public Type getType() {
+      return type;
+    }
+
+    public void setType(Type type) {
+      this.type = type;
+    }
+
+    public Map<String, List<String>> getFieldToEnrichmentTypes() {
+      return fieldToEnrichmentTypes;
+    }
+
+    public void setFieldToEnrichmentTypes(Map<String, List<String>> fieldToEnrichmentTypes) {
+      this.fieldToEnrichmentTypes = fieldToEnrichmentTypes;
+    }
+  }
+  public String zkQuorum;
+  public Map<String, FieldList> sensorToFieldList;
+
+  public String getZkQuorum() {
+    return zkQuorum;
+  }
+
+  public void setZkQuorum(String zkQuorum) {
+    this.zkQuorum = zkQuorum;
+  }
+
+  public Map<String, FieldList> getSensorToFieldList() {
+    return sensorToFieldList;
+  }
+
+  public void setSensorToFieldList(Map<String, FieldList> sensorToFieldList) {
+    this.sensorToFieldList = sensorToFieldList;
+  }
+
+  public void updateSensorConfigs( ) throws Exception {
+    CuratorFramework client = ConfigurationsUtils.getClient(getZkQuorum());
+    try {
+      client.start();
+      updateSensorConfigs(new ZKSourceConfigHandler(client), sensorToFieldList);
+    }
+    finally {
+      client.close();
+    }
+  }
+
+  public static interface SourceConfigHandler {
+    SensorEnrichmentConfig readConfig(String sensor) throws Exception;
+    void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception;
+  }
+
+  public static class ZKSourceConfigHandler implements SourceConfigHandler {
+    CuratorFramework client;
+    public ZKSourceConfigHandler(CuratorFramework client) {
+      this.client = client;
+    }
+    @Override
+    public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
+      SensorEnrichmentConfig sensorEnrichmentConfig = new SensorEnrichmentConfig();
+      try {
+        sensorEnrichmentConfig = SensorEnrichmentConfig.fromBytes(ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensor, client));
+      }catch (KeeperException.NoNodeException e) {
+        sensorEnrichmentConfig.setIndex(sensor);
+        sensorEnrichmentConfig.setBatchSize(1);
+      }
+      return sensorEnrichmentConfig;
+    }
+
+    @Override
+    public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
+      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensor, config.toJSON().getBytes(), client);
+    }
+  }
+
+  public static void updateSensorConfigs( SourceConfigHandler scHandler
+                                        , Map<String, FieldList> sensorToFieldList
+                                        ) throws Exception
+  {
+    Map<String, SensorEnrichmentConfig> sourceConfigsChanged = new HashMap<>();
+    for (Map.Entry<String, FieldList> kv : sensorToFieldList.entrySet()) {
+      SensorEnrichmentConfig config = sourceConfigsChanged.get(kv.getKey());
+      if(config == null) {
+        config = scHandler.readConfig(kv.getKey());
+        if(_LOG.isDebugEnabled()) {
+          _LOG.debug(config.toJSON());
+        }
+      }
+      Map<String, List<String> > fieldMap = null;
+      Map<String, List<String> > fieldToTypeMap = null;
+      List<String> fieldList = null;
+      if(kv.getValue().type == Type.THREAT_INTEL) {
+        fieldMap = config.getThreatIntel().getFieldMap();
+        if(fieldMap!= null) {
+          fieldList = fieldMap.get(Constants.SIMPLE_HBASE_THREAT_INTEL);
+        } else {
+          fieldMap = new HashMap<>();
+        }
+        if(fieldList == null) {
+          fieldList = new ArrayList<>();
+          fieldMap.put(Constants.SIMPLE_HBASE_THREAT_INTEL, fieldList);
+        }
+        fieldToTypeMap = config.getThreatIntel().getFieldToTypeMap();
+        if(fieldToTypeMap == null) {
+          fieldToTypeMap = new HashMap<>();
+          config.getThreatIntel().setFieldToTypeMap(fieldToTypeMap);
+        }
+      }
+      else if(kv.getValue().type == Type.ENRICHMENT) {
+        fieldMap = config.getEnrichment().getFieldMap();
+        if(fieldMap!= null) {
+          fieldList = fieldMap.get(Constants.SIMPLE_HBASE_ENRICHMENT);
+        } else {
+          fieldMap = new HashMap<>();
+        }
+        if(fieldList == null) {
+          fieldList = new ArrayList<>();
+          fieldMap.put(Constants.SIMPLE_HBASE_ENRICHMENT, fieldList);
+        }
+        fieldToTypeMap = config.getEnrichment().getFieldToTypeMap();
+        if(fieldToTypeMap == null) {
+          fieldToTypeMap = new HashMap<>();
+          config.getEnrichment().setFieldToTypeMap(fieldToTypeMap);
+        }
+      }
+      if(fieldToTypeMap == null  || fieldMap == null) {
+        _LOG.debug("fieldToTypeMap is null or fieldMap is null, so skipping");
+        continue;
+      }
+      //Add the additional fields to the field list associated with the hbase adapter
+      {
+        HashSet<String> fieldSet = new HashSet<>(fieldList);
+        List<String> additionalFields = new ArrayList<>();
+        for (String field : kv.getValue().getFieldToEnrichmentTypes().keySet()) {
+          if (!fieldSet.contains(field)) {
+            additionalFields.add(field);
+          }
+        }
+        //adding only the ones that we don't already have to the field list
+        if (additionalFields.size() > 0) {
+          _LOG.debug("Adding additional fields: " + Joiner.on(',').join(additionalFields));
+          fieldList.addAll(additionalFields);
+          sourceConfigsChanged.put(kv.getKey(), config);
+        }
+      }
+      //Add the additional enrichment types to the mapping between the fields
+      {
+        for(Map.Entry<String, List<String>> fieldToType : kv.getValue().getFieldToEnrichmentTypes().entrySet()) {
+          String field = fieldToType.getKey();
+          final HashSet<String> types = new HashSet<>(fieldToType.getValue());
+          int sizeBefore = 0;
+          if(fieldToTypeMap.containsKey(field)) {
+            List<String> typeList = fieldToTypeMap.get(field);
+            sizeBefore = new HashSet<>(typeList).size();
+            types.addAll(typeList);
+          }
+          int sizeAfter = types.size();
+          boolean changed = sizeBefore != sizeAfter;
+          if(changed) {
+            fieldToTypeMap.put(field, new ArrayList<String>() {{
+                addAll(types);
+              }});
+            sourceConfigsChanged.put(kv.getKey(), config);
+          }
+        }
+      }
+    }
+    for(Map.Entry<String, SensorEnrichmentConfig> kv : sourceConfigsChanged.entrySet()) {
+      scHandler.persistConfig(kv.getKey(), kv.getValue());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/Type.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/Type.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/Type.java
new file mode 100644
index 0000000..dc56495
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/Type.java
@@ -0,0 +1,24 @@
+/**
+ * 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;
+
+public enum Type {
+   THREAT_INTEL
+  ,ENRICHMENT
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatIntelConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatIntelConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatIntelConfig.java
new file mode 100644
index 0000000..3b34a0a
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatIntelConfig.java
@@ -0,0 +1,59 @@
+/**
+ * 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 org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
+
+public class ThreatIntelConfig extends EnrichmentConfig {
+  private ThreatTriageConfig triageConfig = new ThreatTriageConfig();
+
+  public ThreatTriageConfig getTriageConfig() {
+    return triageConfig;
+  }
+
+  public void setTriageConfig(ThreatTriageConfig triageConfig) {
+    this.triageConfig = triageConfig;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    ThreatIntelConfig that = (ThreatIntelConfig) o;
+
+    return getTriageConfig() != null ? getTriageConfig().equals(that.getTriageConfig()) : that.getTriageConfig() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (getTriageConfig() != null ? getTriageConfig().hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "ThreatIntelConfig{" +
+            "triageConfig=" + triageConfig +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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
new file mode 100644
index 0000000..9dea047
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/threatintel/ThreatTriageConfig.java
@@ -0,0 +1,101 @@
+/**
+ * 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 com.google.common.base.Joiner;
+import org.apache.metron.common.aggregator.Aggregator;
+import org.apache.metron.common.aggregator.Aggregators;
+import org.apache.metron.common.query.PredicateProcessor;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ThreatTriageConfig {
+  private Map<String, Number> riskLevelRules = new HashMap<>();
+  private Aggregator aggregator = Aggregators.MAX;
+  private Map<String, Object> aggregationConfig = new HashMap<>();
+
+  public Map<String, Number> getRiskLevelRules() {
+    return riskLevelRules;
+  }
+
+  public void setRiskLevelRules(Map<String, Number> riskLevelRules) {
+    this.riskLevelRules = riskLevelRules;
+    PredicateProcessor processor = new PredicateProcessor();
+    for(String rule : riskLevelRules.keySet()) {
+      processor.validate(rule);
+    }
+  }
+
+  public Aggregator getAggregator() {
+    return aggregator;
+  }
+
+  public void setAggregator(String aggregator) {
+    try {
+      this.aggregator = Aggregators.valueOf(aggregator);
+    }
+    catch(IllegalArgumentException iae) {
+      throw new IllegalArgumentException("Unable to load aggregator of " + aggregator
+                                        + ".  Valid aggregators are " + Joiner.on(',').join(Aggregators.values())
+                                        );
+    }
+  }
+
+  public Map<String, Object> getAggregationConfig() {
+    return aggregationConfig;
+  }
+
+  public void setAggregationConfig(Map<String, Object> aggregationConfig) {
+    this.aggregationConfig = aggregationConfig;
+  }
+
+  @Override
+  public String toString() {
+    return "ThreatTriageConfig{" +
+            "riskLevelRules=" + riskLevelRules +
+            ", aggregator=" + aggregator +
+            ", aggregationConfig=" + aggregationConfig +
+            '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ThreatTriageConfig that = (ThreatTriageConfig) o;
+
+    if (getRiskLevelRules() != null ? !getRiskLevelRules().equals(that.getRiskLevelRules()) : that.getRiskLevelRules() != null)
+      return false;
+    if (getAggregator() != null ? !getAggregator().equals(that.getAggregator()) : that.getAggregator() != null)
+      return false;
+    return getAggregationConfig() != null ? getAggregationConfig().equals(that.getAggregationConfig()) : that.getAggregationConfig() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getRiskLevelRules() != null ? getRiskLevelRules().hashCode() : 0;
+    result = 31 * result + (getAggregator() != null ? getAggregator().hashCode() : 0);
+    result = 31 * result + (getAggregationConfig() != null ? getAggregationConfig().hashCode() : 0);
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/BooleanOp.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/BooleanOp.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/BooleanOp.java
new file mode 100644
index 0000000..cedc9a3
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/BooleanOp.java
@@ -0,0 +1,23 @@
+/**
+ * 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.query;
+
+public interface BooleanOp {
+  boolean op(boolean left, boolean right);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ErrorListener.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ErrorListener.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ErrorListener.java
new file mode 100644
index 0000000..883dee2
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ErrorListener.java
@@ -0,0 +1,50 @@
+/**
+ * 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.query;
+
+import org.antlr.v4.runtime.ANTLRErrorListener;
+import org.antlr.v4.runtime.Parser;
+import org.antlr.v4.runtime.RecognitionException;
+import org.antlr.v4.runtime.Recognizer;
+import org.antlr.v4.runtime.atn.ATNConfigSet;
+import org.antlr.v4.runtime.dfa.DFA;
+
+import java.util.BitSet;
+
+public class ErrorListener implements ANTLRErrorListener {
+  @Override
+  public void syntaxError(Recognizer<?, ?> recognizer, Object offendingSymbol, int line, int charPositionInLine, String msg, RecognitionException e) {
+    throw new ParseException("Syntax error @ " + line + ":" + charPositionInLine+ " " + msg, e);
+  }
+
+
+  @Override
+  public void reportAmbiguity(Parser recognizer, DFA dfa, int startIndex, int stopIndex, boolean exact, BitSet ambigAlts, ATNConfigSet configs) {
+  }
+
+
+  @Override
+  public void reportAttemptingFullContext(Parser recognizer, DFA dfa, int startIndex, int stopIndex, BitSet conflictingAlts, ATNConfigSet configs) {
+  }
+
+
+  @Override
+  public void reportContextSensitivity(Parser recognizer, DFA dfa, int startIndex, int stopIndex, int prediction, ATNConfigSet configs) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/FunctionMarker.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/FunctionMarker.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/FunctionMarker.java
new file mode 100644
index 0000000..aefbbbf
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/FunctionMarker.java
@@ -0,0 +1,21 @@
+/**
+ * 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.query;
+
+public class FunctionMarker { }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
new file mode 100644
index 0000000..77aa173
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
@@ -0,0 +1,99 @@
+/**
+ * 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.query;
+
+import com.google.common.base.Function;
+import org.apache.commons.net.util.SubnetUtils;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+public enum LogicalFunctions implements Function<List<String>, Boolean> {
+  IS_EMPTY ( list -> {
+    if(list.size() == 0) {
+      throw new IllegalStateException("IS_EMPTY expects one string arg");
+    }
+    String val = list.get(0);
+    return val == null || val.isEmpty() ? true:false;
+  })
+  ,IN_SUBNET( list -> {
+    if(list.size() < 2) {
+      throw new IllegalStateException("IN_SUBNET expects at least two args: [ip, cidr1, cidr2, ...]"
+                                     + " where cidr is the subnet mask in cidr form"
+                                     );
+    }
+    String ip = list.get(0);
+    if(ip == null) {
+      return false;
+    }
+    boolean inSubnet = false;
+    for(int i = 1;i < list.size() && !inSubnet;++i) {
+      String cidr = list.get(1);
+      if(cidr == null) {
+        continue;
+      }
+      inSubnet |= new SubnetUtils(cidr).getInfo().isInRange(ip);
+    }
+
+    return inSubnet;
+  })
+  ,STARTS_WITH( list -> {
+    if(list.size() < 2) {
+      throw new IllegalStateException("STARTS_WITH expects two args: [string, prefix] where prefix is the string fragment that the string should start with");
+    }
+    String prefix = list.get(1);
+    String str = list.get(0);
+    if(str == null || prefix == null) {
+      return false;
+    }
+    return str.startsWith(prefix);
+  })
+  ,ENDS_WITH( list -> {
+    if(list.size() < 2) {
+      throw new IllegalStateException("ENDS_WITH expects two args: [string, suffix] where suffix is the string fragment that the string should end with");
+    }
+    String prefix = list.get(1);
+    String str = list.get(0);
+    if(str == null || prefix == null) {
+      return false;
+    }
+    return str.endsWith(prefix);
+  })
+  ,REGEXP_MATCH( list -> {
+     if(list.size() < 2) {
+      throw new IllegalStateException("REGEXP_MATCH expects two args: [string, pattern] where pattern is a regexp pattern");
+    }
+    String pattern = list.get(1);
+    String str = list.get(0);
+    if(str == null || pattern == null) {
+      return false;
+    }
+    return str.matches(pattern);
+  })
+  ;
+  Function<List<String>, Boolean> func;
+  LogicalFunctions(Function<List<String>, Boolean> func) {
+    this.func = func;
+  }
+  @Nullable
+  @Override
+  public Boolean apply(@Nullable List<String> input) {
+    return func.apply(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/MapVariableResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/MapVariableResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/MapVariableResolver.java
new file mode 100644
index 0000000..ad1f810
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/MapVariableResolver.java
@@ -0,0 +1,33 @@
+/**
+ * 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.query;
+
+import java.util.Map;
+
+public class MapVariableResolver implements VariableResolver {
+  Map variableMapping;
+  public MapVariableResolver(Map variableMapping) {
+    this.variableMapping = variableMapping;
+  }
+  @Override
+  public String resolve(String variable) {
+    Object o = variableMapping.get(variable);
+    return o == null?null:o.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ParseException.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ParseException.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ParseException.java
new file mode 100644
index 0000000..7d658f0
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/ParseException.java
@@ -0,0 +1,28 @@
+/**
+ * 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.query;
+
+public class ParseException extends RuntimeException {
+  public ParseException(String reason) {
+    super(reason);
+  }
+  public ParseException(String reason, Throwable t) {
+    super(reason, t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
new file mode 100644
index 0000000..6499319
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
@@ -0,0 +1,59 @@
+/**
+ * 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.query;
+
+import org.antlr.v4.runtime.*;
+import org.apache.metron.common.query.generated.*;
+
+
+public class PredicateProcessor {
+  public boolean parse(String rule, VariableResolver resolver) {
+    ANTLRInputStream input = new ANTLRInputStream(rule);
+    PredicateLexer lexer = new PredicateLexer(input);
+    lexer.removeErrorListeners();
+    lexer.addErrorListener(new ErrorListener());
+    TokenStream tokens = new CommonTokenStream(lexer);
+    PredicateParser parser = new PredicateParser(tokens);
+
+    QueryCompiler treeBuilder = new QueryCompiler(resolver);
+    parser.addParseListener(treeBuilder);
+    parser.removeErrorListeners();
+    parser.addErrorListener(new ErrorListener());
+    parser.single_rule();
+    return treeBuilder.getResult();
+  }
+
+  public boolean validate(String rule) throws ParseException {
+    return validate(rule, true);
+  }
+  public boolean validate(String rule, boolean throwException) throws ParseException {
+    try {
+      parse(rule, x -> null);
+      return true;
+    }
+    catch(Throwable t) {
+      if(throwException) {
+        throw new ParseException("Unable to parse " + rule + ": " + t.getMessage(), t);
+      }
+      else {
+        return false;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateToken.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateToken.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateToken.java
new file mode 100644
index 0000000..54ac17d
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateToken.java
@@ -0,0 +1,58 @@
+/**
+ * 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.query;
+
+public class PredicateToken<T> {
+  T value;
+  Class<T> underlyingType;
+  public PredicateToken(T value, Class<T> clazz) {
+    this.value = value;
+    this.underlyingType = clazz;
+  }
+  public T getValue() {
+    return value;
+  }
+  public Class<T> getUnderlyingType() {
+    return underlyingType;
+  }
+
+  @Override
+  public String toString() {
+    return value.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    PredicateToken<?> predicateToken = (PredicateToken<?>) o;
+
+    if (getValue() != null ? !getValue().equals(predicateToken.getValue()) : predicateToken.getValue() != null) return false;
+    return getUnderlyingType() != null ? getUnderlyingType().equals(predicateToken.getUnderlyingType()) : predicateToken.getUnderlyingType() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getValue() != null ? getValue().hashCode() : 0;
+    result = 31 * result + (getUnderlyingType() != null ? getUnderlyingType().hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
new file mode 100644
index 0000000..6351eed
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
@@ -0,0 +1,288 @@
+/**
+ * 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.query;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import org.apache.metron.common.query.generated.PredicateBaseListener;
+import org.apache.metron.common.query.generated.PredicateParser;
+
+import java.util.*;
+
+class QueryCompiler extends PredicateBaseListener {
+  private VariableResolver resolver = null;
+  private Stack<PredicateToken> tokenStack = new Stack<>();
+
+  public QueryCompiler(VariableResolver resolver) {
+    this.resolver = resolver;
+  }
+
+  @Override
+  public void enterSingle_rule(org.apache.metron.common.query.generated.PredicateParser.Single_ruleContext ctx) {
+    tokenStack.clear();
+  }
+
+  @Override
+  public void exitSingle_rule(org.apache.metron.common.query.generated.PredicateParser.Single_ruleContext ctx) {
+  }
+
+  @Override
+  public void exitLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx) {
+    PredicateToken<?> left = popStack();
+    PredicateToken<?> right = popStack();
+    tokenStack.push(new PredicateToken<>(booleanOp(left, right, (l, r) -> l && r, "&&"), Boolean.class));
+  }
+
+  @Override
+  public void exitLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx) {
+    PredicateToken<?> left = popStack();
+    PredicateToken<?> right = popStack();
+
+    tokenStack.push(new PredicateToken<>(booleanOp(left, right, (l, r) -> l || r, "||"), Boolean.class));
+  }
+
+  private boolean booleanOp(PredicateToken<?> left, PredicateToken<?> right, BooleanOp op, String opName)
+  {
+    if(left.getUnderlyingType().equals(right.getUnderlyingType()) && left.getUnderlyingType().equals(Boolean.class)) {
+      Boolean l = (Boolean) left.getValue();
+      Boolean r = (Boolean) right.getValue();
+      if(l == null || r == null) {
+        throw new ParseException("Unable to operate on " + left.getValue()  + " " + opName + " " + right.getValue() + ", null value");
+      }
+      return op.op(l, r);
+    }
+    else {
+      throw new ParseException("Unable to operate on " + left.getValue()  + " " + opName + " " + right.getValue() + ", bad types");
+    }
+  }
+
+
+  @Override
+  public void exitLogicalConst(PredicateParser.LogicalConstContext ctx) {
+    Boolean b = null;
+    switch(ctx.getText().toUpperCase()) {
+      case "TRUE":
+        b = true;
+        break;
+      case "FALSE":
+        b = false;
+        break;
+      default:
+        throw new ParseException("Unable to process " + ctx.getText() + " as a boolean constant");
+    }
+    tokenStack.push(new PredicateToken<>(b, Boolean.class));
+  }
+
+  @Override
+  public void exitComparisonExpressionWithOperator(PredicateParser.ComparisonExpressionWithOperatorContext ctx) {
+    boolean isEqualsOp = ctx.getChild(1).getText().equals("==");
+    PredicateToken<?> left = popStack();
+    PredicateToken<?> right = popStack();
+    if(left.getUnderlyingType().equals(right.getUnderlyingType())) {
+      boolean isEquals = left.equals(right);
+      tokenStack.push(new PredicateToken<>(isEqualsOp?isEquals:!isEquals, Boolean.class));
+    }
+    else {
+      throw new ParseException("Unable to compare " + left.getValue() + " " + ctx.getText() + " " + right.getValue());
+    }
+  }
+
+  public PredicateToken<?> popStack() {
+    if(tokenStack.empty()) {
+      throw new ParseException("Unable to pop an empty stack");
+    }
+    return tokenStack.pop();
+  }
+
+  @Override
+  public void exitLogicalVariable(PredicateParser.LogicalVariableContext ctx) {
+    tokenStack.push(new PredicateToken<>(resolver.resolve(ctx.getText()), String.class));
+  }
+
+
+  @Override
+  public void exitStringLiteral(PredicateParser.StringLiteralContext ctx) {
+    String val = ctx.getText();
+    tokenStack.push(new PredicateToken<>(val.substring(1, val.length() - 1), String.class));
+  }
+
+
+  @Override
+  public void enterList_entity(PredicateParser.List_entityContext ctx) {
+    tokenStack.push(new PredicateToken<>(new FunctionMarker(), FunctionMarker.class));
+  }
+
+
+  @Override
+  public void exitList_entity(PredicateParser.List_entityContext ctx) {
+    Set<String> inSet = new HashSet<>();
+    while(true) {
+      PredicateToken<?> token = popStack();
+      if(token.getUnderlyingType().equals(FunctionMarker.class)) {
+        break;
+      }
+      else {
+        inSet.add((String)token.getValue());
+      }
+    }
+    tokenStack.push(new PredicateToken<>(inSet, Set.class));
+  }
+
+
+  @Override
+  public void enterFunc_args(PredicateParser.Func_argsContext ctx) {
+    tokenStack.push(new PredicateToken<>(new FunctionMarker(), FunctionMarker.class));
+  }
+
+
+  @Override
+  public void exitFunc_args(PredicateParser.Func_argsContext ctx) {
+    LinkedList<String> args = new LinkedList<>();
+    while(true) {
+      PredicateToken<?> token = popStack();
+      if(token.getUnderlyingType().equals(FunctionMarker.class)) {
+        break;
+      }
+      else {
+        args.addFirst((String)token.getValue());
+      }
+    }
+    tokenStack.push(new PredicateToken<>(args, List.class));
+  }
+
+  @Override
+  public void exitInExpression(PredicateParser.InExpressionContext ctx) {
+    PredicateToken<?> left = popStack();
+    PredicateToken<?> right = popStack();
+    String key = null;
+    Set<String> set = null;
+    if(left.getUnderlyingType().equals(Set.class)) {
+      set = (Set<String>) left.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + left.getValue() + " is not a set");
+    }
+    if(right.getUnderlyingType().equals(String.class)) {
+      key = (String) right.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + right.getValue() + " is not a string");
+    }
+    tokenStack.push(new PredicateToken<>(set.contains(key), Boolean.class));
+  }
+
+  @Override
+  public void exitNInExpression(PredicateParser.NInExpressionContext ctx) {
+    PredicateToken<?> left = popStack();
+    PredicateToken<?> right = popStack();
+    String key = null;
+    Set<String> set = null;
+    if(left.getUnderlyingType().equals(Set.class)) {
+      set = (Set<String>) left.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + left.getValue() + " is not a set");
+    }
+    if(right.getUnderlyingType().equals(String.class)) {
+      key = (String) right.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + right.getValue() + " is not a string");
+    }
+    tokenStack.push(new PredicateToken<>(!set.contains(key), Boolean.class));
+  }
+
+
+  @Override
+  public void exitLogicalFunc(PredicateParser.LogicalFuncContext ctx) {
+    String funcName = ctx.getChild(0).getText();
+    Function<List<String>, Boolean> func;
+    try {
+      func = LogicalFunctions.valueOf(funcName);
+    }
+    catch(IllegalArgumentException iae) {
+      throw new ParseException("Unable to find logical function " + funcName + ".  Valid functions are "
+              + Joiner.on(',').join(LogicalFunctions.values())
+      );
+    }
+    PredicateToken<?> left = popStack();
+    List<String> argList = null;
+    if(left.getUnderlyingType().equals(List.class)) {
+      argList = (List<String>) left.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + left.getValue() + " is not a set");
+    }
+    Boolean result = func.apply(argList);
+    tokenStack.push(new PredicateToken<>(result, Boolean.class));
+  }
+
+  @Override
+  public void exitStringFunc(PredicateParser.StringFuncContext ctx) {
+    String funcName = ctx.getChild(0).getText();
+    Function<List<String>, String> func;
+    try {
+      func = StringFunctions.valueOf(funcName);
+    }
+    catch(IllegalArgumentException iae) {
+      throw new ParseException("Unable to find string function " + funcName + ".  Valid functions are "
+              + Joiner.on(',').join(StringFunctions.values())
+      );
+    }
+    PredicateToken<?> left = popStack();
+    List<String> argList = null;
+    if(left.getUnderlyingType().equals(List.class)) {
+      argList = (List<String>) left.getValue();
+    }
+    else {
+      throw new ParseException("Unable to process in clause because " + left.getValue() + " is not a set");
+    }
+    String result = func.apply(argList);
+    tokenStack.push(new PredicateToken<>(result, String.class));
+  }
+
+  @Override
+  public void exitExistsFunc(PredicateParser.ExistsFuncContext ctx) {
+    String variable = ctx.getChild(2).getText();
+    boolean exists = resolver.resolve(variable) != null;
+    tokenStack.push(new PredicateToken<>(exists, Boolean.class));
+  }
+
+  @Override
+  public void exitNotFunc(PredicateParser.NotFuncContext ctx) {
+    PredicateToken<Boolean> arg = (PredicateToken<Boolean>) popStack();
+    tokenStack.push(new PredicateToken<>(!arg.getValue(), Boolean.class));
+  }
+
+  public boolean getResult() throws ParseException {
+    if(tokenStack.empty()) {
+      throw new ParseException("Invalid predicate: Empty stack.");
+    }
+    PredicateToken<?> token = popStack();
+    if(token.getUnderlyingType().equals(Boolean.class) && tokenStack.empty()) {
+      return (Boolean)token.getValue();
+    }
+    if(tokenStack.empty()) {
+      throw new ParseException("Invalid parse, stack not empty: " + Joiner.on(',').join(tokenStack));
+    }
+    else {
+      throw new ParseException("Invalid parse, found " + token + " but expected boolean");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/StringFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/StringFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/StringFunctions.java
new file mode 100644
index 0000000..911063e
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/StringFunctions.java
@@ -0,0 +1,42 @@
+/**
+ * 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.query;
+
+
+import com.google.common.base.Function;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+public enum StringFunctions implements Function<List<String>, String> {
+  TO_LOWER(strings -> strings.get(0)==null?null:strings.get(0).toLowerCase())
+  ,TO_UPPER(strings -> strings.get(0) == null?null:strings.get(0).toUpperCase())
+  ,TRIM(strings -> strings.get(0) == null?null:strings.get(0).trim())
+  ;
+  Function<List<String>, String> func;
+  StringFunctions(Function<List<String>, String> func) {
+    this.func = func;
+  }
+
+  @Nullable
+  @Override
+  public String apply(@Nullable List<String> input) {
+    return func.apply(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/VariableResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/VariableResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/VariableResolver.java
new file mode 100644
index 0000000..d0a6625
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/VariableResolver.java
@@ -0,0 +1,23 @@
+/**
+ * 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.query;
+
+public interface VariableResolver {
+  String resolve(String variable);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateBaseListener.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateBaseListener.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateBaseListener.java
new file mode 100644
index 0000000..ca7ac84
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateBaseListener.java
@@ -0,0 +1,336 @@
+// Generated from org/apache/metron/common/query/generated/Predicate.g4 by ANTLR 4.5
+package org.apache.metron.common.query.generated;
+
+//CHECKSTYLE:OFF
+/**
+ * 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.
+ */
+
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.ErrorNode;
+import org.antlr.v4.runtime.tree.TerminalNode;
+
+/**
+ * This class provides an empty implementation of {@link PredicateListener},
+ * which can be extended to create a listener which only needs to handle a subset
+ * of the available methods.
+ */
+public class PredicateBaseListener implements PredicateListener {
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterSingle_rule(PredicateParser.Single_ruleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitSingle_rule(PredicateParser.Single_ruleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalEntity(PredicateParser.LogicalEntityContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalEntity(PredicateParser.LogicalEntityContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterComparisonExpression(PredicateParser.ComparisonExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitComparisonExpression(PredicateParser.ComparisonExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalExpressionInParen(PredicateParser.LogicalExpressionInParenContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalExpressionInParen(PredicateParser.LogicalExpressionInParenContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterNotFunc(PredicateParser.NotFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitNotFunc(PredicateParser.NotFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterComparisonExpressionWithOperator(PredicateParser.ComparisonExpressionWithOperatorContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitComparisonExpressionWithOperator(PredicateParser.ComparisonExpressionWithOperatorContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterInExpression(PredicateParser.InExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitInExpression(PredicateParser.InExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterNInExpression(PredicateParser.NInExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitNInExpression(PredicateParser.NInExpressionContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterComparisonExpressionParens(PredicateParser.ComparisonExpressionParensContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitComparisonExpressionParens(PredicateParser.ComparisonExpressionParensContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalConst(PredicateParser.LogicalConstContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalConst(PredicateParser.LogicalConstContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterExistsFunc(PredicateParser.ExistsFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitExistsFunc(PredicateParser.ExistsFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalFunc(PredicateParser.LogicalFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalFunc(PredicateParser.LogicalFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterList_entity(PredicateParser.List_entityContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitList_entity(PredicateParser.List_entityContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterFunc_args(PredicateParser.Func_argsContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitFunc_args(PredicateParser.Func_argsContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterOp_list(PredicateParser.Op_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitOp_list(PredicateParser.Op_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterStringLiteral(PredicateParser.StringLiteralContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitStringLiteral(PredicateParser.StringLiteralContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalVariable(PredicateParser.LogicalVariableContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalVariable(PredicateParser.LogicalVariableContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterStringFunc(PredicateParser.StringFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitStringFunc(PredicateParser.StringFuncContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterIdentifierOperand(PredicateParser.IdentifierOperandContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitIdentifierOperand(PredicateParser.IdentifierOperandContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterLogicalConstComparison(PredicateParser.LogicalConstComparisonContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitLogicalConstComparison(PredicateParser.LogicalConstComparisonContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterComparisonOp(PredicateParser.ComparisonOpContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitComparisonOp(PredicateParser.ComparisonOpContext ctx) { }
+
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterEveryRule(ParserRuleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitEveryRule(ParserRuleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void visitTerminal(TerminalNode node) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void visitErrorNode(ErrorNode node) { }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateLexer.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateLexer.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateLexer.java
new file mode 100644
index 0000000..ebc0b00
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateLexer.java
@@ -0,0 +1,178 @@
+// Generated from org/apache/metron/common/query/generated/Predicate.g4 by ANTLR 4.5
+package org.apache.metron.common.query.generated;
+
+//CHECKSTYLE:OFF
+/**
+ * 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.
+ */
+
+import org.antlr.v4.runtime.Lexer;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.Token;
+import org.antlr.v4.runtime.TokenStream;
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.atn.*;
+import org.antlr.v4.runtime.dfa.DFA;
+import org.antlr.v4.runtime.misc.*;
+
+@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
+public class PredicateLexer extends Lexer {
+	static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+
+	protected static final DFA[] _decisionToDFA;
+	protected static final PredictionContextCache _sharedContextCache =
+		new PredictionContextCache();
+	public static final int
+		AND=1, OR=2, NOT=3, TRUE=4, FALSE=5, EQ=6, NEQ=7, COMMA=8, LBRACKET=9, 
+		RBRACKET=10, LPAREN=11, RPAREN=12, IN=13, NIN=14, EXISTS=15, IDENTIFIER=16, 
+		STRING_LITERAL=17, SEMI=18, COMMENT=19, WS=20;
+	public static String[] modeNames = {
+		"DEFAULT_MODE"
+	};
+
+	public static final String[] ruleNames = {
+		"AND", "OR", "NOT", "TRUE", "FALSE", "EQ", "NEQ", "COMMA", "LBRACKET", 
+		"RBRACKET", "LPAREN", "RPAREN", "IN", "NIN", "EXISTS", "IDENTIFIER", "SCHAR", 
+		"STRING_LITERAL", "SEMI", "COMMENT", "WS"
+	};
+
+	private static final String[] _LITERAL_NAMES = {
+		null, null, null, null, null, null, "'=='", "'!='", "','", "'['", "']'", 
+		"'('", "')'", "'in'", "'not in'", "'exists'", null, null, "';'"
+	};
+	private static final String[] _SYMBOLIC_NAMES = {
+		null, "AND", "OR", "NOT", "TRUE", "FALSE", "EQ", "NEQ", "COMMA", "LBRACKET", 
+		"RBRACKET", "LPAREN", "RPAREN", "IN", "NIN", "EXISTS", "IDENTIFIER", "STRING_LITERAL", 
+		"SEMI", "COMMENT", "WS"
+	};
+	public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES);
+
+	/**
+	 * @deprecated Use {@link #VOCABULARY} instead.
+	 */
+	@Deprecated
+	public static final String[] tokenNames;
+	static {
+		tokenNames = new String[_SYMBOLIC_NAMES.length];
+		for (int i = 0; i < tokenNames.length; i++) {
+			tokenNames[i] = VOCABULARY.getLiteralName(i);
+			if (tokenNames[i] == null) {
+				tokenNames[i] = VOCABULARY.getSymbolicName(i);
+			}
+
+			if (tokenNames[i] == null) {
+				tokenNames[i] = "<INVALID>";
+			}
+		}
+	}
+
+	@Override
+	@Deprecated
+	public String[] getTokenNames() {
+		return tokenNames;
+	}
+
+	@Override
+
+	public Vocabulary getVocabulary() {
+		return VOCABULARY;
+	}
+
+
+	public PredicateLexer(CharStream input) {
+		super(input);
+		_interp = new LexerATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache);
+	}
+
+	@Override
+	public String getGrammarFileName() { return "Predicate.g4"; }
+
+	@Override
+	public String[] getRuleNames() { return ruleNames; }
+
+	@Override
+	public String getSerializedATN() { return _serializedATN; }
+
+	@Override
+	public String[] getModeNames() { return modeNames; }
+
+	@Override
+	public ATN getATN() { return _ATN; }
+
+	public static final String _serializedATN =
+		"\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\26\u00af\b\1\4\2"+
+		"\t\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4"+
+		"\13\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22"+
+		"\t\22\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\3\2\3\2\3\2\3\2\3\2\3\2"+
+		"\3\2\3\2\5\2\66\n\2\3\3\3\3\3\3\3\3\3\3\3\3\5\3>\n\3\3\4\3\4\3\4\3\4\3"+
+		"\4\3\4\5\4F\n\4\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\5\5P\n\5\3\6\3\6\3\6\3"+
+		"\6\3\6\3\6\3\6\3\6\3\6\3\6\5\6\\\n\6\3\7\3\7\3\7\3\b\3\b\3\b\3\t\3\t\3"+
+		"\n\3\n\3\13\3\13\3\f\3\f\3\r\3\r\3\16\3\16\3\16\3\17\3\17\3\17\3\17\3"+
+		"\17\3\17\3\17\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\21\3\21\7\21\u0081"+
+		"\n\21\f\21\16\21\u0084\13\21\3\22\3\22\3\23\3\23\7\23\u008a\n\23\f\23"+
+		"\16\23\u008d\13\23\3\23\3\23\3\23\7\23\u0092\n\23\f\23\16\23\u0095\13"+
+		"\23\3\23\5\23\u0098\n\23\3\24\3\24\3\25\3\25\3\25\3\25\6\25\u00a0\n\25"+
+		"\r\25\16\25\u00a1\3\25\5\25\u00a5\n\25\3\25\3\25\3\26\6\26\u00aa\n\26"+
+		"\r\26\16\26\u00ab\3\26\3\26\3\u00a1\2\27\3\3\5\4\7\5\t\6\13\7\r\b\17\t"+
+		"\21\n\23\13\25\f\27\r\31\16\33\17\35\20\37\21!\22#\2%\23\'\24)\25+\26"+
+		"\3\2\7\5\2C\\aac|\b\2\60\60\62;C\\^^aac|\7\2\f\f\17\17$$))^^\3\3\f\f\5"+
+		"\2\13\f\16\17\"\"\u00ba\2\3\3\2\2\2\2\5\3\2\2\2\2\7\3\2\2\2\2\t\3\2\2"+
+		"\2\2\13\3\2\2\2\2\r\3\2\2\2\2\17\3\2\2\2\2\21\3\2\2\2\2\23\3\2\2\2\2\25"+
+		"\3\2\2\2\2\27\3\2\2\2\2\31\3\2\2\2\2\33\3\2\2\2\2\35\3\2\2\2\2\37\3\2"+
+		"\2\2\2!\3\2\2\2\2%\3\2\2\2\2\'\3\2\2\2\2)\3\2\2\2\2+\3\2\2\2\3\65\3\2"+
+		"\2\2\5=\3\2\2\2\7E\3\2\2\2\tO\3\2\2\2\13[\3\2\2\2\r]\3\2\2\2\17`\3\2\2"+
+		"\2\21c\3\2\2\2\23e\3\2\2\2\25g\3\2\2\2\27i\3\2\2\2\31k\3\2\2\2\33m\3\2"+
+		"\2\2\35p\3\2\2\2\37w\3\2\2\2!~\3\2\2\2#\u0085\3\2\2\2%\u0097\3\2\2\2\'"+
+		"\u0099\3\2\2\2)\u009b\3\2\2\2+\u00a9\3\2\2\2-.\7c\2\2./\7p\2\2/\66\7f"+
+		"\2\2\60\61\7(\2\2\61\66\7(\2\2\62\63\7C\2\2\63\64\7P\2\2\64\66\7F\2\2"+
+		"\65-\3\2\2\2\65\60\3\2\2\2\65\62\3\2\2\2\66\4\3\2\2\2\678\7q\2\28>\7t"+
+		"\2\29:\7~\2\2:>\7~\2\2;<\7Q\2\2<>\7T\2\2=\67\3\2\2\2=9\3\2\2\2=;\3\2\2"+
+		"\2>\6\3\2\2\2?@\7p\2\2@A\7q\2\2AF\7v\2\2BC\7P\2\2CD\7Q\2\2DF\7V\2\2E?"+
+		"\3\2\2\2EB\3\2\2\2F\b\3\2\2\2GH\7v\2\2HI\7t\2\2IJ\7w\2\2JP\7g\2\2KL\7"+
+		"V\2\2LM\7T\2\2MN\7W\2\2NP\7G\2\2OG\3\2\2\2OK\3\2\2\2P\n\3\2\2\2QR\7h\2"+
+		"\2RS\7c\2\2ST\7n\2\2TU\7u\2\2U\\\7g\2\2VW\7H\2\2WX\7C\2\2XY\7N\2\2YZ\7"+
+		"U\2\2Z\\\7G\2\2[Q\3\2\2\2[V\3\2\2\2\\\f\3\2\2\2]^\7?\2\2^_\7?\2\2_\16"+
+		"\3\2\2\2`a\7#\2\2ab\7?\2\2b\20\3\2\2\2cd\7.\2\2d\22\3\2\2\2ef\7]\2\2f"+
+		"\24\3\2\2\2gh\7_\2\2h\26\3\2\2\2ij\7*\2\2j\30\3\2\2\2kl\7+\2\2l\32\3\2"+
+		"\2\2mn\7k\2\2no\7p\2\2o\34\3\2\2\2pq\7p\2\2qr\7q\2\2rs\7v\2\2st\7\"\2"+
+		"\2tu\7k\2\2uv\7p\2\2v\36\3\2\2\2wx\7g\2\2xy\7z\2\2yz\7k\2\2z{\7u\2\2{"+
+		"|\7v\2\2|}\7u\2\2} \3\2\2\2~\u0082\t\2\2\2\177\u0081\t\3\2\2\u0080\177"+
+		"\3\2\2\2\u0081\u0084\3\2\2\2\u0082\u0080\3\2\2\2\u0082\u0083\3\2\2\2\u0083"+
+		"\"\3\2\2\2\u0084\u0082\3\2\2\2\u0085\u0086\n\4\2\2\u0086$\3\2\2\2\u0087"+
+		"\u008b\7$\2\2\u0088\u008a\5#\22\2\u0089\u0088\3\2\2\2\u008a\u008d\3\2"+
+		"\2\2\u008b\u0089\3\2\2\2\u008b\u008c\3\2\2\2\u008c\u008e\3\2\2\2\u008d"+
+		"\u008b\3\2\2\2\u008e\u0098\7$\2\2\u008f\u0093\7)\2\2\u0090\u0092\5#\22"+
+		"\2\u0091\u0090\3\2\2\2\u0092\u0095\3\2\2\2\u0093\u0091\3\2\2\2\u0093\u0094"+
+		"\3\2\2\2\u0094\u0096\3\2\2\2\u0095\u0093\3\2\2\2\u0096\u0098\7)\2\2\u0097"+
+		"\u0087\3\2\2\2\u0097\u008f\3\2\2\2\u0098&\3\2\2\2\u0099\u009a\7=\2\2\u009a"+
+		"(\3\2\2\2\u009b\u009c\7\61\2\2\u009c\u009d\7\61\2\2\u009d\u009f\3\2\2"+
+		"\2\u009e\u00a0\13\2\2\2\u009f\u009e\3\2\2\2\u00a0\u00a1\3\2\2\2\u00a1"+
+		"\u00a2\3\2\2\2\u00a1\u009f\3\2\2\2\u00a2\u00a4\3\2\2\2\u00a3\u00a5\t\5"+
+		"\2\2\u00a4\u00a3\3\2\2\2\u00a5\u00a6\3\2\2\2\u00a6\u00a7\b\25\2\2\u00a7"+
+		"*\3\2\2\2\u00a8\u00aa\t\6\2\2\u00a9\u00a8\3\2\2\2\u00aa\u00ab\3\2\2\2"+
+		"\u00ab\u00a9\3\2\2\2\u00ab\u00ac\3\2\2\2\u00ac\u00ad\3\2\2\2\u00ad\u00ae"+
+		"\b\26\2\2\u00ae,\3\2\2\2\17\2\65=EO[\u0082\u008b\u0093\u0097\u00a1\u00a4"+
+		"\u00ab\3\b\2\2";
+	public static final ATN _ATN =
+		new ATNDeserializer().deserialize(_serializedATN.toCharArray());
+	static {
+		_decisionToDFA = new DFA[_ATN.getNumberOfDecisions()];
+		for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) {
+			_decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateListener.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateListener.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateListener.java
new file mode 100644
index 0000000..6a8b669
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateListener.java
@@ -0,0 +1,299 @@
+// Generated from org/apache/metron/common/query/generated/Predicate.g4 by ANTLR 4.5
+package org.apache.metron.common.query.generated;
+
+//CHECKSTYLE:OFF
+/**
+ * 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.
+ */
+
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.ParseTreeListener;
+
+/**
+ * This interface defines a complete listener for a parse tree produced by
+ * {@link PredicateParser}.
+ */
+public interface PredicateListener extends ParseTreeListener {
+	/**
+	 * Enter a parse tree produced by {@link PredicateParser#single_rule}.
+	 * @param ctx the parse tree
+	 */
+	void enterSingle_rule(PredicateParser.Single_ruleContext ctx);
+	/**
+	 * Exit a parse tree produced by {@link PredicateParser#single_rule}.
+	 * @param ctx the parse tree
+	 */
+	void exitSingle_rule(PredicateParser.Single_ruleContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalEntity}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalEntity(PredicateParser.LogicalEntityContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalEntity}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalEntity(PredicateParser.LogicalEntityContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code ComparisonExpression}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterComparisonExpression(PredicateParser.ComparisonExpressionContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code ComparisonExpression}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitComparisonExpression(PredicateParser.ComparisonExpressionContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalExpressionInParen}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalExpressionInParen(PredicateParser.LogicalExpressionInParenContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalExpressionInParen}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalExpressionInParen(PredicateParser.LogicalExpressionInParenContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code NotFunc}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterNotFunc(PredicateParser.NotFuncContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code NotFunc}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitNotFunc(PredicateParser.NotFuncContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalExpressionAnd}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalExpressionAnd}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalExpressionOr}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalExpressionOr}
+	 * labeled alternative in {@link PredicateParser#logical_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code ComparisonExpressionWithOperator}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterComparisonExpressionWithOperator(PredicateParser.ComparisonExpressionWithOperatorContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code ComparisonExpressionWithOperator}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitComparisonExpressionWithOperator(PredicateParser.ComparisonExpressionWithOperatorContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code InExpression}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterInExpression(PredicateParser.InExpressionContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code InExpression}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitInExpression(PredicateParser.InExpressionContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code NInExpression}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterNInExpression(PredicateParser.NInExpressionContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code NInExpression}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitNInExpression(PredicateParser.NInExpressionContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code ComparisonExpressionParens}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void enterComparisonExpressionParens(PredicateParser.ComparisonExpressionParensContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code ComparisonExpressionParens}
+	 * labeled alternative in {@link PredicateParser#comparison_expr}.
+	 * @param ctx the parse tree
+	 */
+	void exitComparisonExpressionParens(PredicateParser.ComparisonExpressionParensContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalConst}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalConst(PredicateParser.LogicalConstContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalConst}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalConst(PredicateParser.LogicalConstContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code ExistsFunc}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void enterExistsFunc(PredicateParser.ExistsFuncContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code ExistsFunc}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void exitExistsFunc(PredicateParser.ExistsFuncContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalFunc}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalFunc(PredicateParser.LogicalFuncContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalFunc}
+	 * labeled alternative in {@link PredicateParser#logical_entity}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalFunc(PredicateParser.LogicalFuncContext ctx);
+	/**
+	 * Enter a parse tree produced by {@link PredicateParser#list_entity}.
+	 * @param ctx the parse tree
+	 */
+	void enterList_entity(PredicateParser.List_entityContext ctx);
+	/**
+	 * Exit a parse tree produced by {@link PredicateParser#list_entity}.
+	 * @param ctx the parse tree
+	 */
+	void exitList_entity(PredicateParser.List_entityContext ctx);
+	/**
+	 * Enter a parse tree produced by {@link PredicateParser#func_args}.
+	 * @param ctx the parse tree
+	 */
+	void enterFunc_args(PredicateParser.Func_argsContext ctx);
+	/**
+	 * Exit a parse tree produced by {@link PredicateParser#func_args}.
+	 * @param ctx the parse tree
+	 */
+	void exitFunc_args(PredicateParser.Func_argsContext ctx);
+	/**
+	 * Enter a parse tree produced by {@link PredicateParser#op_list}.
+	 * @param ctx the parse tree
+	 */
+	void enterOp_list(PredicateParser.Op_listContext ctx);
+	/**
+	 * Exit a parse tree produced by {@link PredicateParser#op_list}.
+	 * @param ctx the parse tree
+	 */
+	void exitOp_list(PredicateParser.Op_listContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code StringLiteral}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void enterStringLiteral(PredicateParser.StringLiteralContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code StringLiteral}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void exitStringLiteral(PredicateParser.StringLiteralContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalVariable}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalVariable(PredicateParser.LogicalVariableContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalVariable}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalVariable(PredicateParser.LogicalVariableContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code StringFunc}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void enterStringFunc(PredicateParser.StringFuncContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code StringFunc}
+	 * labeled alternative in {@link PredicateParser#identifier_operand}.
+	 * @param ctx the parse tree
+	 */
+	void exitStringFunc(PredicateParser.StringFuncContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code IdentifierOperand}
+	 * labeled alternative in {@link PredicateParser#comparison_operand}.
+	 * @param ctx the parse tree
+	 */
+	void enterIdentifierOperand(PredicateParser.IdentifierOperandContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code IdentifierOperand}
+	 * labeled alternative in {@link PredicateParser#comparison_operand}.
+	 * @param ctx the parse tree
+	 */
+	void exitIdentifierOperand(PredicateParser.IdentifierOperandContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code LogicalConstComparison}
+	 * labeled alternative in {@link PredicateParser#comparison_operand}.
+	 * @param ctx the parse tree
+	 */
+	void enterLogicalConstComparison(PredicateParser.LogicalConstComparisonContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code LogicalConstComparison}
+	 * labeled alternative in {@link PredicateParser#comparison_operand}.
+	 * @param ctx the parse tree
+	 */
+	void exitLogicalConstComparison(PredicateParser.LogicalConstComparisonContext ctx);
+	/**
+	 * Enter a parse tree produced by the {@code ComparisonOp}
+	 * labeled alternative in {@link PredicateParser#comp_operator}.
+	 * @param ctx the parse tree
+	 */
+	void enterComparisonOp(PredicateParser.ComparisonOpContext ctx);
+	/**
+	 * Exit a parse tree produced by the {@code ComparisonOp}
+	 * labeled alternative in {@link PredicateParser#comp_operator}.
+	 * @param ctx the parse tree
+	 */
+	void exitComparisonOp(PredicateParser.ComparisonOpContext ctx);
+}
\ No newline at end of file



[5/5] incubator-metron git commit: METRON-141: The ability to do threat triage closes apache/incubator-metron#108

Posted by ce...@apache.org.
METRON-141: The ability to do threat triage closes apache/incubator-metron#108


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

Branch: refs/heads/master
Commit: deed21e6e8d8a8924c199c38e68c69481263d0d1
Parents: 743f37b
Author: cstella <ce...@gmail.com>
Authored: Tue May 10 12:53:35 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Tue May 10 12:53:35 2016 -0400

----------------------------------------------------------------------
 .../files/config/sensors/bro.json               |   25 +-
 .../files/config/sensors/pcap.json              |   26 +-
 .../files/config/sensors/snort.json             |   34 +-
 .../files/config/sensors/yaf.json               |   29 +-
 .../metron_streaming/tasks/source_config.yml    |    2 +-
 metron-platform/metron-common/README.md         |  186 +++
 metron-platform/metron-common/pom.xml           |   22 +
 .../metron/common/query/generated/Predicate.g4  |  131 +++
 .../src/main/java/Predicate.tokens              |   31 +
 .../src/main/java/PredicateLexer.tokens         |   31 +
 .../metron/common/aggregator/Aggregator.java    |   26 +
 .../metron/common/aggregator/Aggregators.java   |   62 +
 .../metron/common/bolt/ConfiguredBolt.java      |    2 +-
 .../metron/common/cli/ConfigurationManager.java |  200 ++++
 .../metron/common/cli/ConfigurationsUtils.java  |  232 ----
 .../common/configuration/Configuration.java     |    1 -
 .../common/configuration/ConfigurationType.java |   75 ++
 .../common/configuration/Configurations.java    |    1 +
 .../configuration/ConfigurationsUtils.java      |  195 +++
 .../common/configuration/EnrichmentConfig.java  |  213 ----
 .../configuration/SensorEnrichmentConfig.java   |  129 --
 .../enrichment/EnrichmentConfig.java            |   71 ++
 .../enrichment/SensorEnrichmentConfig.java      |  108 ++
 .../SensorEnrichmentUpdateConfig.java           |  209 ++++
 .../common/configuration/enrichment/Type.java   |   24 +
 .../threatintel/ThreatIntelConfig.java          |   59 +
 .../threatintel/ThreatTriageConfig.java         |  101 ++
 .../apache/metron/common/query/BooleanOp.java   |   23 +
 .../metron/common/query/ErrorListener.java      |   50 +
 .../metron/common/query/FunctionMarker.java     |   21 +
 .../metron/common/query/LogicalFunctions.java   |   99 ++
 .../common/query/MapVariableResolver.java       |   33 +
 .../metron/common/query/ParseException.java     |   28 +
 .../metron/common/query/PredicateProcessor.java |   59 +
 .../metron/common/query/PredicateToken.java     |   58 +
 .../metron/common/query/QueryCompiler.java      |  288 +++++
 .../metron/common/query/StringFunctions.java    |   42 +
 .../metron/common/query/VariableResolver.java   |   23 +
 .../query/generated/PredicateBaseListener.java  |  336 ++++++
 .../common/query/generated/PredicateLexer.java  |  178 +++
 .../query/generated/PredicateListener.java      |  299 +++++
 .../common/query/generated/PredicateParser.java | 1108 ++++++++++++++++++
 .../src/main/scripts/zk_load_configs.sh         |    2 +-
 .../metron/common/bolt/ConfiguredBoltTest.java  |    8 +-
 .../ConfigurationManagerIntegrationTest.java    |  176 +++
 .../common/cli/ConfigurationsUtilsTest.java     |   13 +-
 .../configuration/EnrichmentConfigTest.java     |  211 ----
 .../SensorEnrichmentConfigTest.java             |    2 +-
 .../SensorEnrichmentUpdateConfigTest.java       |  224 ++++
 .../metron/common/query/QueryParserTest.java    |  152 +++
 .../src/test/resources/config/sensors/bro.json  |   30 +-
 .../src/main/assembly/assembly.xml              |    2 +-
 .../src/main/bash/Whois_CSV_to_JSON.py          |  208 ----
 .../src/main/bash/flatfile_loader.sh            |   42 -
 .../main/bash/prune_elasticsearch_indices.sh    |   21 -
 .../src/main/bash/prune_hdfs_files.sh           |   21 -
 .../src/main/bash/threatintel_bulk_load.sh      |   41 -
 .../src/main/bash/threatintel_bulk_prune.sh     |   40 -
 .../src/main/bash/threatintel_taxii_load.sh     |   42 -
 .../dataloads/bulk/ThreatIntelBulkLoader.java   |   12 +-
 .../SimpleEnrichmentFlatFileLoader.java         |   12 +-
 .../dataloads/nonbulk/taxii/TaxiiLoader.java    |   10 +-
 .../src/main/scripts/Whois_CSV_to_JSON.py       |  208 ++++
 .../src/main/scripts/flatfile_loader.sh         |   42 +
 .../main/scripts/prune_elasticsearch_indices.sh |   21 +
 .../src/main/scripts/prune_hdfs_files.sh        |   21 +
 .../src/main/scripts/threatintel_bulk_load.sh   |   41 +
 .../src/main/scripts/threatintel_bulk_prune.sh  |   40 +
 .../src/main/scripts/threatintel_taxii_load.sh  |   42 +
 .../writer/ElasticsearchWriter.java             |    2 +-
 metron-platform/metron-enrichment/pom.xml       |    1 +
 .../simplehbase/SimpleHBaseAdapter.java         |    2 +-
 .../threatintel/ThreatIntelAdapter.java         |    4 +-
 .../enrichment/bolt/BulkMessageWriterBolt.java  |    2 +-
 .../apache/metron/enrichment/bolt/CacheKey.java |    2 +-
 .../enrichment/bolt/EnrichmentJoinBolt.java     |   10 +-
 .../enrichment/bolt/EnrichmentSplitterBolt.java |    4 +-
 .../enrichment/bolt/GenericEnrichmentBolt.java  |    2 +-
 .../apache/metron/enrichment/bolt/JoinBolt.java |   33 +-
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |   59 +-
 .../bolt/ThreatIntelSplitterBolt.java           |    4 +-
 .../triage/ThreatTriageProcessor.java           |   51 +
 .../simplehbase/SimpleHBaseAdapterTest.java     |   34 +-
 .../threatintel/ThreatIntelAdapterTest.java     |   42 +-
 .../bolt/GenericEnrichmentBoltTest.java         |    4 +-
 .../bolt/ThreatIntelJoinBoltTest.java           |   65 +-
 .../threatintel/triage/ThreatTriageTest.java    |  136 +++
 .../metron/integration/BaseIntegrationTest.java |    2 +-
 .../integration/EnrichmentIntegrationTest.java  |    2 +
 .../metron/integration/utils/SampleUtil.java    |    2 +-
 .../resources/sample/config/sensors/bro.json    |   34 +-
 .../resources/sample/config/sensors/pcap.json   |   25 +-
 .../resources/sample/config/sensors/snort.json  |   37 +-
 .../resources/sample/config/sensors/yaf.json    |   62 +-
 .../SolrEnrichmentIntegrationTest.java          |    2 +-
 metron-platform/style/checkstyle.xml            |   10 +-
 metron-platform/style/suppressions.xml          |   27 +
 pom.xml                                         |    1 +
 98 files changed, 5793 insertions(+), 1422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json b/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
index 2b534b4..0eb34b3 100644
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
+++ b/metron-deployment/roles/metron_streaming/files/config/sensors/bro.json
@@ -1,19 +1,20 @@
 {
   "index": "bro",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment" : {
+    "fieldMap": {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_src_addr" : ["malicious_ip"],
-    "ip_dst_addr" : ["malicious_ip"]
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json b/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
index 7792165..8a3eab1 100644
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
+++ b/metron-deployment/roles/metron_streaming/files/config/sensors/pcap.json
@@ -1,19 +1,23 @@
 {
   "index": "pcap",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_src_addr", "ip_dst_addr"],
-    "host": ["ip_src_addr", "ip_dst_addr"]
+  "enrichment" : {
+    "fieldMap":
+      {
+      "geo": ["ip_src_addr", "ip_dst_addr"],
+      "host": ["ip_src_addr", "ip_dst_addr"]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_dst_addr" : [ "malicious_ip" ]
+  "threatIntel" : {
+    "fieldMap":
+      {
+      "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+    },
+    "fieldToTypeMap":
+      {
+      "ip_dst_addr" : [ "malicious_ip" ]
     ,"ip_src_addr" : [ "malicious_ip" ]
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json b/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
index c5b6dcc..9dfc80e 100644
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
+++ b/metron-deployment/roles/metron_streaming/files/config/sensors/snort.json
@@ -1,18 +1,28 @@
 {
   "index": "snort",
   "batchSize": 1,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment" : {
+    "fieldMap":
+      {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
   },
- "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_src_addr" : ["malicious_ip"],
-    "ip_dst_addr" : ["malicious_ip"]
+  "threatIntel" : {
+    "fieldMap":
+      {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap":
+      {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
+      },
+      "aggregator" : "MAX"
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json b/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
index 2b46c9a..4e67748 100644
--- a/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
+++ b/metron-deployment/roles/metron_streaming/files/config/sensors/yaf.json
@@ -1,19 +1,22 @@
 {
   "index": "yaf",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment" : {
+    "fieldMap":
+      {
+      "geo": ["ip_dst_addr", "ip_src_addr"],
+      "host": ["host"]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_src_addr" : ["malicious_ip"],
-    "ip_dst_addr" : ["malicious_ip"]
+  "threatIntel": {
+    "fieldMap":
+      {
+      "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"]
+    },
+    "fieldToTypeMap":
+      {
+      "ip_src_addr" : ["malicious_ip"],
+      "ip_dst_addr" : ["malicious_ip"]
+    }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-deployment/roles/metron_streaming/tasks/source_config.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/source_config.yml b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
index f3039af..21292b1 100644
--- a/metron-deployment/roles/metron_streaming/tasks/source_config.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/source_config.yml
@@ -43,7 +43,7 @@
     - ../roles/metron_streaming/files/config/
 
 - name: Load Config
-  shell: "{{ metron_directory }}/bin/zk_load_configs.sh -p {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"
+  shell: "{{ metron_directory }}/bin/zk_load_configs.sh --mode PUSH -i {{ zookeeper_config_path }} -z {{ zookeeper_url }} && touch {{ zookeeper_config_path }}/configured"
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md
new file mode 100644
index 0000000..60bd287
--- /dev/null
+++ b/metron-platform/metron-common/README.md
@@ -0,0 +1,186 @@
+#Query Language
+
+For a variety of components (currently only threat intelligence triage) we have the need to determine if a condition is true of the JSON documents being enriched.  For those purposes, there exists a simple DSL created to define those conditions.
+
+The query language supports the following:
+* Referencing fields in the enriched JSON
+* Simple boolean operations: and, not, or
+* Determining whether a field exists (via `exists`)
+* The ability to have parenthesis to make order of operations explicit
+* A fixed set of functions which take strings and return boolean.  Currently:
+    * `IN_SUBNET(ip, cidr1, cidr2, ...)`
+    * `IS_EMPTY(str)`
+    * `STARTS_WITH(str, prefix)`
+    * `ENDS_WITH(str, suffix)`
+    * `REGEXP_MATCH(str, pattern)`
+* A fixed set of string to string transformation functions:
+    * `TO_LOWER`
+    * `TO_UPPER`
+    * `TRIM`
+
+Example query:
+
+`IN_SUBNET( ip, '192.168.0.0/24') or ip in [ '10.0.0.1', '10.0.0.2' ] or exists(is_local)`
+
+This evaluates to true precisely when one of the following is true:
+* The value of the `ip` field is in the `192.168.0.0/24` subnet
+* The value of the `ip` field is `10.0.0.1` or `10.0.0.2`
+* The field `is_local` exists
+
+#Enrichment Configuration
+
+The configuration for the `enrichment` topology, the topology primarily
+responsible for enrichment and threat intelligence enrichment, is
+defined by JSON documents stored in zookeeper.
+
+There are two types of configurations at the moment, `global` and
+`sensor` specific.  
+
+##Global Configuration
+The format of the global enrichment is a JSON String to Object map.  This is intended for
+configuration which is non sensor specific configuration.
+
+This configuration is stored in zookeeper, but looks something like
+
+```json
+{
+  "es.clustername": "metron",
+  "es.ip": "node1",
+  "es.port": "9300",
+  "es.date.format": "yyyy.MM.dd.HH"
+}
+```
+
+##Sensor Enrichment Configuration
+
+The sensor specific configuration is intended to configure the
+individual enrichments and threat intelligence enrichments for a given
+sensor type (e.g. `snort`).
+
+Just like the global config, the format is a JSON stored in zookeeper.
+The configuration is a complex JSON object with the following top level fields:
+
+* `index` : The name of the sensor
+* `batchSize` : The size of the batch that is written to the indices at once.
+* `enrichment` : A complex JSON object representing the configuration of the enrichments
+* `threatIntel` : A complex JSON object representing the configuration of the threat intelligence enrichments
+
+###The `enrichment` Configuration 
+
+
+| Field            | Description                                                                                                                                                                                                                      | Example                                                                   |
+|------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------|
+| `fieldToTypeMap` | In the case of a simple HBase enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "asset_enrichment" ] }`  |
+| `fieldMap`       | The map of enrichment bolts names to fields in the JSON messages.,Each field is sent to the enrichment referenced in the key.                                                                                                    | `"fieldMap": {"hbaseEnrichment": ["ip_src_addr","ip_dst_addr"]}` |
+
+###The `threatIntel` Configuration 
+
+| Field            | Description                                                                                                                                                                                                                                      | Example                                                                  |
+|------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
+| `fieldToTypeMap` | In the case of a simple HBase threat intel enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "malicious_ips" ] }`             |
+| `fieldMap`       | The map of threat intel enrichment bolts names to fields in the JSON messages. Each field is sent to the threat intel enrichment bolt referenced in the key.                                                                              | `"fieldMap": {"hbaseThreatIntel": ["ip_src_addr","ip_dst_addr"]}`        |
+| `triageConfig`   | The configuration of the threat triage scorer.  In the situation where a threat is detected, a score is assigned to the message and embedded in the indexed message.                                                                             | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
+
+The `triageConfig` field is also a complex field and it bears some description:
+
+| Field            | Description                                                                                                                                                | Example                                                                  |
+|------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
+| `riskLevelRules` | The mapping of Metron Query Language (see above) queries to a score.                                                                                       | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
+| `aggregator`     | An aggregation function that takes all non-zero scores representing the matching queries from `riskLevelRules` and aggregates them into a single score. | `"MAX"`                                                                  |
+
+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
+* `MEAN` : The mean of all of the associated values for matching queries
+* `POSITIVE_MEAN` : The mean of the positive associated values for the matching queries.
+
+###Example
+
+An example configuration for the YAF sensor is as follows:
+```json
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "hbaseEnrichment": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
+  ,"fieldToTypeMap": {
+      "ip_src_addr": [
+        "playful_classification"
+      ],
+      "ip_dst_addr": [
+        "playful_classification"
+      ]
+    }
+  },
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr": [
+        "malicious_ip"
+      ],
+      "ip_dst_addr": [
+        "malicious_ip"
+      ]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
+      },
+      "aggregator" : "MAX"
+    }
+  }
+}
+```
+
+
+##Management Utility
+
+Configurations should be stored on disk in the following structure starting at `$BASE_DIR`:
+* global.json : The global config
+* `sensors` : The subdirectory containing sensor enrichment configuration JSON (e.g. `snort.json`, `bro.json`)
+
+By default, this directory as deployed by the ansible infrastructure is at `$METRON_HOME/config/zookeeper`
+
+While the configs are stored on disk, they must be loaded into Zookeeper to be used.  To this end, there is a
+utility program to assist in this called `$METRON_HOME/bin/zk_load_config.sh`
+
+This has the following options:
+
+```
+ -f,--force                                Force operation
+ -h,--help                                 Generate Help screen
+ -i,--input_dir <DIR>                      The input directory containing
+                                           the configuration files named
+                                           like "$source.json"
+ -m,--mode <MODE>                          The mode of operation: DUMP,
+                                           PULL, PUSH
+ -o,--output_dir <DIR>                     The output directory which will
+                                           store the JSON configuration
+                                           from Zookeeper
+ -z,--zk_quorum <host:port,[host:port]*>   Zookeeper Quorum URL
+                                           (zk1:port,zk2:port,...)
+```
+
+Usage examples:
+
+* To dump the existing configs from zookeeper on the singlenode vagrant machine: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m DUMP`
+* To push the configs into zookeeper on the singlenode vagrant machine: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m PUSH -i $METRON_HOME/config/zookeeper`
+* To pull the configs from zookeeper to the singlenode vagrant machine disk: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m PULL -o $METRON_HOME/config/zookeeper -f`

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml
index 91d50fe..9d439e0 100644
--- a/metron-platform/metron-common/pom.xml
+++ b/metron-platform/metron-common/pom.xml
@@ -27,6 +27,7 @@
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <commons.config.version>1.10</commons.config.version>
+        <antlr.version>4.5</antlr.version>
     </properties>
     <repositories>
         <repository>
@@ -47,6 +48,11 @@
             <version>${global_json_simple_version}</version>
         </dependency>
         <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+            <version>${antlr.version}</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-core</artifactId>
             <version>${global_storm_version}</version>
@@ -236,6 +242,22 @@
                 </configuration>
             </plugin>
             <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${antlr.version}</version>
+                <configuration>
+                  <outputDirectory>${basedir}/src/main/java</outputDirectory>
+                </configuration>
+                <executions>
+                  <execution>
+                    <goals>
+                      <goal>antlr4</goal>
+                    </goals>
+                  </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-shade-plugin</artifactId>
                 <version>${global_shade_version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/antlr4/org/apache/metron/common/query/generated/Predicate.g4
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/antlr4/org/apache/metron/common/query/generated/Predicate.g4 b/metron-platform/metron-common/src/main/antlr4/org/apache/metron/common/query/generated/Predicate.g4
new file mode 100644
index 0000000..73c60cd
--- /dev/null
+++ b/metron-platform/metron-common/src/main/antlr4/org/apache/metron/common/query/generated/Predicate.g4
@@ -0,0 +1,131 @@
+/**
+ * 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.
+ */
+
+grammar Predicate;
+
+@header {
+//CHECKSTYLE:OFF
+/**
+ * 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.
+ */
+}
+
+/* Lexical rules */
+
+AND : 'and'
+    | '&&'
+    | 'AND'
+    ;
+OR  : 'or'
+    | '||'
+    | 'OR';
+
+NOT : 'not'
+    | 'NOT';
+
+TRUE  : 'true'
+      | 'TRUE' ;
+
+FALSE : 'false'
+      | 'FALSE';
+
+EQ : '==' ;
+NEQ : '!=' ;
+COMMA : ',';
+
+LBRACKET : '[';
+RBRACKET : ']';
+LPAREN : '(' ;
+RPAREN : ')' ;
+
+IN : 'in'
+   ;
+NIN : 'not in'
+   ;
+EXISTS : 'exists';
+IDENTIFIER : [a-zA-Z_][a-zA-Z_\.0-9]* ;
+fragment SCHAR:  ~['"\\\r\n];
+STRING_LITERAL : '"' SCHAR* '"'
+               | '\'' SCHAR* '\'' ;
+SEMI : ';' ;
+
+
+// COMMENT and WS are stripped from the output token stream by sending
+// to a different channel 'skip'
+
+COMMENT : '//' .+? ('\n'|EOF) -> skip ;
+
+WS : [ \r\t\u000C\n]+ -> skip ;
+
+
+/* Parser rules */
+
+single_rule : logical_expr EOF;
+
+logical_expr
+ : logical_expr AND logical_expr # LogicalExpressionAnd
+ | logical_expr OR logical_expr  # LogicalExpressionOr
+ | comparison_expr               # ComparisonExpression
+ | LPAREN logical_expr RPAREN    # LogicalExpressionInParen
+ | NOT LPAREN logical_expr RPAREN #NotFunc
+ | logical_entity                # LogicalEntity
+ ;
+
+comparison_expr : comparison_operand comp_operator comparison_operand # ComparisonExpressionWithOperator
+                | identifier_operand IN list_entity #InExpression
+                | identifier_operand NIN list_entity #NInExpression
+                | LPAREN comparison_expr RPAREN # ComparisonExpressionParens
+                ;
+
+logical_entity : (TRUE | FALSE) # LogicalConst
+               | EXISTS LPAREN IDENTIFIER RPAREN #ExistsFunc
+               | IDENTIFIER LPAREN func_args RPAREN #LogicalFunc
+               ;
+
+list_entity : LBRACKET op_list RBRACKET
+            ;
+func_args : op_list
+          ;
+op_list : identifier_operand
+        | op_list COMMA identifier_operand
+        ;
+identifier_operand : STRING_LITERAL # StringLiteral
+                   | IDENTIFIER     # LogicalVariable
+                   | IDENTIFIER LPAREN func_args RPAREN #StringFunc
+                   ;
+
+comparison_operand : identifier_operand #IdentifierOperand
+                   | logical_entity # LogicalConstComparison
+                   ;
+
+comp_operator : (EQ | NEQ) # ComparisonOp
+              ;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/Predicate.tokens
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/Predicate.tokens b/metron-platform/metron-common/src/main/java/Predicate.tokens
new file mode 100644
index 0000000..e0726f9
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/Predicate.tokens
@@ -0,0 +1,31 @@
+AND=1
+OR=2
+NOT=3
+TRUE=4
+FALSE=5
+EQ=6
+NEQ=7
+COMMA=8
+LBRACKET=9
+RBRACKET=10
+LPAREN=11
+RPAREN=12
+IN=13
+NIN=14
+EXISTS=15
+IDENTIFIER=16
+STRING_LITERAL=17
+SEMI=18
+COMMENT=19
+WS=20
+'=='=6
+'!='=7
+','=8
+'['=9
+']'=10
+'('=11
+')'=12
+'in'=13
+'not in'=14
+'exists'=15
+';'=18

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/PredicateLexer.tokens
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/PredicateLexer.tokens b/metron-platform/metron-common/src/main/java/PredicateLexer.tokens
new file mode 100644
index 0000000..e0726f9
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/PredicateLexer.tokens
@@ -0,0 +1,31 @@
+AND=1
+OR=2
+NOT=3
+TRUE=4
+FALSE=5
+EQ=6
+NEQ=7
+COMMA=8
+LBRACKET=9
+RBRACKET=10
+LPAREN=11
+RPAREN=12
+IN=13
+NIN=14
+EXISTS=15
+IDENTIFIER=16
+STRING_LITERAL=17
+SEMI=18
+COMMENT=19
+WS=20
+'=='=6
+'!='=7
+','=8
+'['=9
+']'=10
+'('=11
+')'=12
+'in'=13
+'not in'=14
+'exists'=15
+';'=18

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregator.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregator.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregator.java
new file mode 100644
index 0000000..d7ca31e
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregator.java
@@ -0,0 +1,26 @@
+/**
+ * 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.aggregator;
+
+import java.util.List;
+import java.util.Map;
+
+public interface Aggregator {
+  Double aggregate(List<Number> scores, Map<String, Object> config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregators.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregators.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregators.java
new file mode 100644
index 0000000..35b7ada
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/aggregator/Aggregators.java
@@ -0,0 +1,62 @@
+/**
+ * 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.aggregator;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.BinaryOperator;
+import java.util.function.Predicate;
+
+public enum Aggregators implements Aggregator {
+   MAX( (numbers, config) -> accumulate(0d, (x,y) -> Math.max(x.doubleValue(),y.doubleValue()), numbers))
+  ,MIN( (numbers, config) -> accumulate(0d, (x,y) -> Math.min(x.doubleValue(),y.doubleValue()), numbers))
+  ,SUM( (numbers, config) -> accumulate(0d, (x,y) -> x.doubleValue() + y.doubleValue(), numbers))
+  ,MEAN( (numbers, config) -> scale(SUM.aggregate(numbers, config), numbers, n -> true))
+  ,POSITIVE_MEAN( (numbers, config) -> scale(SUM.aggregate(numbers, config), numbers, n -> n.doubleValue() > 0))
+  ;
+  Aggregator aggregator;
+  Aggregators(Aggregator agg) {
+    aggregator = agg;
+  }
+  public Aggregator getAggregator() {
+    return aggregator;
+  }
+
+  private static double accumulate(double initial, BinaryOperator<Number> op, List<Number> list) {
+    if(list.isEmpty()) {
+      return 0d;
+    }
+    return list.stream()
+               .reduce(initial, op)
+               .doubleValue();
+  }
+
+  private static double scale(double numberToScale, List<Number> list, Predicate<Number> filterFunc) {
+    double scale = list.stream().filter(filterFunc).count();
+    if(scale < 1e-5) {
+      scale = 1;
+    }
+    return numberToScale / scale;
+  }
+
+  @Override
+  public Double aggregate(List<Number> scores, Map<String, Object> config) {
+    return aggregator.aggregate(scores, config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
index aa654fb..1364305 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
@@ -30,7 +30,7 @@ import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.log4j.Logger;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 
 import java.io.IOException;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationManager.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationManager.java
new file mode 100644
index 0000000..9685841
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationManager.java
@@ -0,0 +1,200 @@
+/**
+ * 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.cli;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.io.Files;
+import org.apache.commons.cli.*;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationType;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.charset.Charset;
+
+public class ConfigurationManager {
+  public enum ConfigurationOptions {
+    HELP("h", s -> new Option(s, "help", false, "Generate Help screen"))
+   ,INPUT("i", s -> OptionBuilder.isRequired(false)
+                                 .withLongOpt("input_dir")
+                                 .hasArg()
+                                 .withArgName("DIR")
+                                 .withDescription("The input directory containing the configuration files named like \"$source.json\"")
+                                 .create(s)
+         )
+    ,OUTPUT("o", s -> OptionBuilder.isRequired(false)
+                                 .hasArg()
+                                 .withLongOpt("output_dir")
+                                 .withArgName("DIR")
+                                 .withDescription("The output directory which will store the JSON configuration from Zookeeper")
+                                 .create(s)
+         )
+    ,ZK_QUORUM("z", s -> OptionBuilder.isRequired(true)
+                                 .hasArg()
+                                 .withLongOpt("zk_quorum")
+                                 .withArgName("host:port,[host:port]*")
+                                 .withDescription("Zookeeper Quorum URL (zk1:port,zk2:port,...)")
+                                 .create(s)
+         )
+    ,MODE("m", s -> OptionBuilder.isRequired(true)
+                                 .hasArg()
+                                 .withLongOpt("mode")
+                                 .withArgName("MODE")
+                                 .withDescription("The mode of operation: DUMP, PULL, PUSH")
+                                 .create(s)
+         )
+    ,FORCE("f", s -> new Option(s, "force", false, "Force operation"))
+    ;
+    Option option;
+    String shortCode;
+    ConfigurationOptions(String shortCode, Function<String, Option> optionHandler) {
+      this.shortCode = shortCode;
+      this.option = optionHandler.apply(shortCode);
+    }
+
+    public boolean has(CommandLine cli) {
+      return cli.hasOption(shortCode);
+    }
+
+    public String get(CommandLine cli) {
+      return cli.getOptionValue(shortCode);
+    }
+
+    public static CommandLine parse(CommandLineParser parser, String[] args) {
+      try {
+        CommandLine cli = parser.parse(getOptions(), args);
+        if(ConfigurationOptions.HELP.has(cli)) {
+          printHelp();
+          System.exit(0);
+        }
+        return cli;
+      } catch (ParseException e) {
+        System.err.println("Unable to parse args: " + Joiner.on(' ').join(args));
+        e.printStackTrace(System.err);
+        printHelp();
+        System.exit(-1);
+        return null;
+      }
+    }
+
+    public static void printHelp() {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp( "configuration_manager", getOptions());
+    }
+
+    public static Options getOptions() {
+      Options ret = new Options();
+      for(ConfigurationOptions o : ConfigurationOptions.values()) {
+        ret.addOption(o.option);
+      }
+      return ret;
+    }
+  }
+
+  public void dump(CuratorFramework client) throws Exception {
+    ConfigurationsUtils.dumpConfigs(System.out, client);
+  }
+
+
+
+  public void pull(CuratorFramework client, String outFileStr, final boolean force) throws Exception {
+    final File outputDir = new File(outFileStr);
+    if (!outputDir.exists()) {
+      if (!outputDir.mkdirs()) {
+        throw new IllegalStateException("Unable to make directories: " + outputDir.getAbsolutePath());
+      }
+    }
+
+    ConfigurationsUtils.visitConfigs(client, new ConfigurationsUtils.ConfigurationVisitor() {
+      @Override
+      public void visit(ConfigurationType configurationType, String name, String data) {
+        File out = getFile(outputDir, configurationType, name);
+        if (!out.exists() || force) {
+          if(!out.exists()) {
+            out.getParentFile().mkdirs();
+          }
+          try {
+            Files.write(data, out, Charset.defaultCharset());
+          } catch (IOException e) {
+            throw new RuntimeException("Sorry, something bad happened writing the config to " + out.getAbsolutePath() + ": " + e.getMessage(), e);
+          }
+        }
+        else if(out.exists() && !force) {
+          throw new IllegalStateException("Unable to overwrite existing file (" + out.getAbsolutePath() + ") without the force flag (-f or --force) being set.");
+        }
+      }
+    });
+  }
+
+  public void push(String inputDirStr, CuratorFramework client) throws Exception {
+      final File inputDir = new File(inputDirStr);
+
+      if(!inputDir.exists() || !inputDir.isDirectory()) {
+        throw new IllegalStateException("Input directory: " + inputDir + " does not exist or is not a directory.");
+      }
+      ConfigurationsUtils.uploadConfigsToZookeeper(inputDirStr, client);
+  }
+
+  public void run(CommandLine cli) throws Exception {
+    try(CuratorFramework client = ConfigurationsUtils.getClient(ConfigurationOptions.ZK_QUORUM.get(cli))) {
+      client.start();
+      run(client, cli);
+    }
+  }
+  public void run(CuratorFramework client, CommandLine cli) throws Exception {
+    final boolean force = ConfigurationOptions.FORCE.has(cli);
+    String mode = ConfigurationOptions.MODE.get(cli);
+
+    if (mode.toLowerCase().equals("push")) {
+      String inputDirStr = ConfigurationOptions.INPUT.get(cli);
+      push(inputDirStr, client);
+    }
+    else {
+
+      switch (mode.toLowerCase()) {
+
+        case "dump":
+          dump(client);
+          break;
+
+        case "pull":
+          pull(client, ConfigurationOptions.OUTPUT.get(cli), force);
+          break;
+
+        default:
+          throw new IllegalStateException("Invalid mode: " + mode + " expected DUMP, PULL or PUSH");
+      }
+    }
+  }
+
+  private static File getFile(File baseDir, ConfigurationType configurationType, String name) {
+    return new File(new File(baseDir, configurationType.getDirectory()), name + ".json");
+  }
+
+  public static void main(String... argv) throws Exception {
+    CommandLineParser parser = new PosixParser();
+    CommandLine cli = ConfigurationOptions.parse(parser, argv);
+    ConfigurationManager manager = new ConfigurationManager();
+    manager.run(cli);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationsUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationsUtils.java
deleted file mode 100644
index 27f4c2a..0000000
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/cli/ConfigurationsUtils.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/**
- * 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.cli;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.cli.PosixParser;
-import org.apache.commons.io.FilenameUtils;
-import org.apache.curator.RetryPolicy;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.metron.common.Constants;
-import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
-import org.apache.metron.common.utils.JSONUtils;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ConfigurationsUtils {
-
-  public static CuratorFramework getClient(String zookeeperUrl) {
-    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
-    return CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy);
-  }
-
-  public static void writeGlobalConfigToZookeeper(Map<String, Object> globalConfig, String zookeeperUrl) throws Exception {
-    writeGlobalConfigToZookeeper(JSONUtils.INSTANCE.toJSON(globalConfig), zookeeperUrl);
-  }
-
-  public static void writeGlobalConfigToZookeeper(byte[] globalConfig, String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    try {
-      writeGlobalConfigToZookeeper(globalConfig, client);
-    }
-    finally {
-      client.close();
-    }
-  }
-
-  public static void writeGlobalConfigToZookeeper(byte[] globalConfig, CuratorFramework client) throws Exception {
-    writeToZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, globalConfig, client);
-  }
-
-  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig, String zookeeperUrl) throws Exception {
-    writeSensorEnrichmentConfigToZookeeper(sensorType, JSONUtils.INSTANCE.toJSON(sensorEnrichmentConfig), zookeeperUrl);
-  }
-
-  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    try {
-      writeSensorEnrichmentConfigToZookeeper(sensorType, configData, client);
-    }
-    finally {
-      client.close();
-    }
-  }
-
-  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception {
-    writeToZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, configData, client);
-  }
-
-  public static void writeConfigToZookeeper(String name, Map<String, Object> config, String zookeeperUrl) throws Exception {
-    writeConfigToZookeeper(name, JSONUtils.INSTANCE.toJSON(config), zookeeperUrl);
-  }
-
-  public static void writeConfigToZookeeper(String name, byte[] config, String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    try {
-      writeToZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, config, client);
-    }
-    finally {
-      client.close();
-    }
-  }
-
-  public static void writeToZookeeper(String path, byte[] configData, CuratorFramework client) throws Exception {
-    try {
-      client.setData().forPath(path, configData);
-    } catch (KeeperException.NoNodeException e) {
-      client.create().creatingParentsIfNeeded().forPath(path, configData);
-    }
-  }
-
-  public static void updateConfigsFromZookeeper(Configurations configurations, CuratorFramework client) throws Exception {
-    configurations.updateGlobalConfig(readGlobalConfigBytesFromZookeeper(client));
-    List<String> sensorTypes = client.getChildren().forPath(Constants.ZOOKEEPER_SENSOR_ROOT);
-    for(String sensorType: sensorTypes) {
-      configurations.updateSensorEnrichmentConfig(sensorType, readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client));
-    }
-  }
-
-  public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception {
-    return readFromZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, client);
-  }
-
-  public static byte[] readSensorEnrichmentConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
-    return readFromZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, client);
-  }
-
-  public static byte[] readConfigBytesFromZookeeper(String name, CuratorFramework client) throws Exception {
-    return readFromZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, client);
-  }
-
-  public static byte[] readFromZookeeper(String path, CuratorFramework client) throws Exception {
-    return client.getData().forPath(path);
-  }
-
-  public static void uploadConfigsToZookeeper(String rootFilePath, String zookeeperUrl) throws Exception {
-    ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(rootFilePath), zookeeperUrl);
-    Map<String, byte[]> sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(rootFilePath);
-    for(String sensorType: sensorEnrichmentConfigs.keySet()) {
-      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), zookeeperUrl);
-    }
-  }
-
-  public static byte[] readGlobalConfigFromFile(String rootFilePath) throws IOException {
-    return Files.readAllBytes(Paths.get(rootFilePath, Constants.GLOBAL_CONFIG_NAME + ".json"));
-  }
-
-  public static Map<String, byte[]> readSensorEnrichmentConfigsFromFile(String rootPath) throws IOException {
-    Map<String, byte[]> sensorEnrichmentConfigs = new HashMap<>();
-    for(File file: new File(rootPath, Constants.SENSORS_CONFIG_NAME).listFiles()) {
-      sensorEnrichmentConfigs.put(FilenameUtils.removeExtension(file.getName()), Files.readAllBytes(file.toPath()));
-    }
-    return sensorEnrichmentConfigs;
-  }
-
-  public static void dumpConfigs(String zookeeperUrl) throws Exception {
-    CuratorFramework client = getClient(zookeeperUrl);
-    client.start();
-    //Output global configs
-    {
-      System.out.println("Global config");
-      byte[] globalConfigData = client.getData().forPath(Constants.ZOOKEEPER_GLOBAL_ROOT);
-      System.out.println(new String(globalConfigData));
-    }
-    //Output sensor specific configs
-    {
-      List<String> children = client.getChildren().forPath(Constants.ZOOKEEPER_SENSOR_ROOT);
-      for (String child : children) {
-        byte[] data = client.getData().forPath(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + child);
-        System.out.println("Config for source " + child);
-        System.out.println(new String(data));
-        System.out.println();
-      }
-    }
-    client.close();
-  }
-
-  public static void main(String[] args) {
-
-    Options options = new Options();
-    {
-      Option o = new Option("h", "help", false, "This screen");
-      o.setRequired(false);
-      options.addOption(o);
-    }
-    {
-      Option o = new Option("p", "config_files", true, "Path to the source config files.  Must be named like \"$source\".json");
-      o.setArgName("DIR_NAME");
-      o.setRequired(false);
-      options.addOption(o);
-    }
-    {
-      Option o = new Option("z", "zk", true, "Zookeeper Quroum URL (zk1:2181,zk2:2181,...");
-      o.setArgName("ZK_QUORUM");
-      o.setRequired(true);
-      options.addOption(o);
-    }
-
-    try {
-      CommandLineParser parser = new PosixParser();
-      CommandLine cmd = null;
-      try {
-        cmd = parser.parse(options, args);
-      } catch (ParseException pe) {
-        pe.printStackTrace();
-        final HelpFormatter usageFormatter = new HelpFormatter();
-        usageFormatter.printHelp("ConfigurationsUtils", null, options, null, true);
-        System.exit(-1);
-      }
-      if (cmd.hasOption("h")) {
-        final HelpFormatter usageFormatter = new HelpFormatter();
-        usageFormatter.printHelp("ConfigurationsUtils", null, options, null, true);
-        System.exit(0);
-      }
-
-      String zkQuorum = cmd.getOptionValue("z");
-      if (cmd.hasOption("p")) {
-        String sourcePath = cmd.getOptionValue("p");
-        uploadConfigsToZookeeper(sourcePath, zkQuorum);
-      }
-
-      ConfigurationsUtils.dumpConfigs(zkQuorum);
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      System.exit(-1);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
index 1ccf47b..e526ee4 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configuration.java
@@ -18,7 +18,6 @@
 package org.apache.metron.common.configuration;
 
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.metron.common.cli.ConfigurationsUtils;
 
 import java.nio.file.Path;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
new file mode 100644
index 0000000..2b9f6cf
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Function;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.utils.JSONUtils;
+
+import java.io.IOException;
+import java.util.Map;
+
+public enum ConfigurationType implements Function<String, Object> {
+  GLOBAL("."
+        ,Constants.ZOOKEEPER_GLOBAL_ROOT
+        , s -> {
+    try {
+      return JSONUtils.INSTANCE.load(s, new TypeReference<Map<String, Object>>() {
+      });
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to load " + s, e);
+    }
+  })
+  , SENSOR(Constants.SENSORS_CONFIG_NAME
+          ,Constants.ZOOKEEPER_SENSOR_ROOT
+          , s -> {
+    try {
+      return JSONUtils.INSTANCE.load(s, SensorEnrichmentConfig.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to load " + s, e);
+    }
+  });
+  String directory;
+  String zookeeperRoot;
+  Function<String,?> deserializer;
+  ConfigurationType(String directory, String zookeeperRoot, Function<String, ?> deserializer) {
+    this.directory = directory;
+    this.zookeeperRoot = zookeeperRoot;
+    this.deserializer = deserializer;
+  }
+
+  public String getDirectory() {
+    return directory;
+  }
+
+  public Object deserialize(String s)
+  {
+    return deserializer.apply(s);
+  }
+  @Override
+  public Object apply(String s) {
+    return deserialize(s);
+  }
+
+  public String getZookeeperRoot() {
+    return zookeeperRoot;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
index 6aaa2b4..a152d40 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
@@ -19,6 +19,7 @@ package org.apache.metron.common.configuration;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import org.apache.log4j.Logger;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.utils.JSONUtils;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
new file mode 100644
index 0000000..1aa2ca8
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
@@ -0,0 +1,195 @@
+/**
+ * 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;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.utils.JSONUtils;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ConfigurationsUtils {
+
+  public static CuratorFramework getClient(String zookeeperUrl) {
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    return CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy);
+  }
+
+  public static void writeGlobalConfigToZookeeper(Map<String, Object> globalConfig, String zookeeperUrl) throws Exception {
+    try(CuratorFramework client = getClient(zookeeperUrl)) {
+     client.start();
+      writeGlobalConfigToZookeeper(globalConfig, client);
+    }
+  }
+  public static void writeGlobalConfigToZookeeper(Map<String, Object> globalConfig, CuratorFramework client) throws Exception {
+    writeGlobalConfigToZookeeper(JSONUtils.INSTANCE.toJSON(globalConfig), client);
+  }
+
+  public static void writeGlobalConfigToZookeeper(byte[] globalConfig, String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    try {
+      writeGlobalConfigToZookeeper(globalConfig, client);
+    }
+    finally {
+      client.close();
+    }
+  }
+
+  public static void writeGlobalConfigToZookeeper(byte[] globalConfig, CuratorFramework client) throws Exception {
+    ConfigurationType.GLOBAL.deserialize(new String(globalConfig));
+    writeToZookeeper(ConfigurationType.GLOBAL.getZookeeperRoot(), globalConfig, client);
+  }
+
+  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig, String zookeeperUrl) throws Exception {
+    writeSensorEnrichmentConfigToZookeeper(sensorType, JSONUtils.INSTANCE.toJSON(sensorEnrichmentConfig), zookeeperUrl);
+  }
+
+  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    try {
+      writeSensorEnrichmentConfigToZookeeper(sensorType, configData, client);
+    }
+    finally {
+      client.close();
+    }
+  }
+
+  public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception {
+    ConfigurationType.SENSOR.deserialize(new String(configData));
+    writeToZookeeper(ConfigurationType.SENSOR.getZookeeperRoot()+ "/" + sensorType, configData, client);
+  }
+
+  public static void writeConfigToZookeeper(String name, Map<String, Object> config, String zookeeperUrl) throws Exception {
+    writeConfigToZookeeper(name, JSONUtils.INSTANCE.toJSON(config), zookeeperUrl);
+  }
+
+  public static void writeConfigToZookeeper(String name, byte[] config, String zookeeperUrl) throws Exception {
+    CuratorFramework client = getClient(zookeeperUrl);
+    client.start();
+    try {
+      writeToZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, config, client);
+    }
+    finally {
+      client.close();
+    }
+  }
+
+  public static void writeToZookeeper(String path, byte[] configData, CuratorFramework client) throws Exception {
+    try {
+      client.setData().forPath(path, configData);
+    } catch (KeeperException.NoNodeException e) {
+      client.create().creatingParentsIfNeeded().forPath(path, configData);
+    }
+  }
+
+  public static void updateConfigsFromZookeeper(Configurations configurations, CuratorFramework client) throws Exception {
+    configurations.updateGlobalConfig(readGlobalConfigBytesFromZookeeper(client));
+    List<String> sensorTypes = client.getChildren().forPath(Constants.ZOOKEEPER_SENSOR_ROOT);
+    for(String sensorType: sensorTypes) {
+      configurations.updateSensorEnrichmentConfig(sensorType, readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client));
+    }
+  }
+
+  public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception {
+    return readFromZookeeper(Constants.ZOOKEEPER_GLOBAL_ROOT, client);
+  }
+
+  public static byte[] readSensorEnrichmentConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception {
+    return readFromZookeeper(Constants.ZOOKEEPER_SENSOR_ROOT + "/" + sensorType, client);
+  }
+
+  public static byte[] readConfigBytesFromZookeeper(String name, CuratorFramework client) throws Exception {
+    return readFromZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, client);
+  }
+
+  public static byte[] readFromZookeeper(String path, CuratorFramework client) throws Exception {
+    return client.getData().forPath(path);
+  }
+
+  public static void uploadConfigsToZookeeper(String rootFilePath, CuratorFramework client) throws Exception {
+    ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(rootFilePath), client);
+    Map<String, byte[]> sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(rootFilePath);
+    for(String sensorType: sensorEnrichmentConfigs.keySet()) {
+      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), client);
+    }
+  }
+
+  public static void uploadConfigsToZookeeper(String rootFilePath, String zookeeperUrl) throws Exception {
+    try(CuratorFramework client = getClient(zookeeperUrl)) {
+      client.start();
+      uploadConfigsToZookeeper(rootFilePath, client);
+    }
+  }
+
+  public static byte[] readGlobalConfigFromFile(String rootFilePath) throws IOException {
+    return Files.readAllBytes(Paths.get(rootFilePath, Constants.GLOBAL_CONFIG_NAME + ".json"));
+  }
+
+  public static Map<String, byte[]> readSensorEnrichmentConfigsFromFile(String rootPath) throws IOException {
+    Map<String, byte[]> sensorEnrichmentConfigs = new HashMap<>();
+    for(File file: new File(rootPath, Constants.SENSORS_CONFIG_NAME).listFiles()) {
+      if(file.getName().endsWith(".json")) {
+        sensorEnrichmentConfigs.put(FilenameUtils.removeExtension(file.getName()), Files.readAllBytes(file.toPath()));
+      }
+    }
+    return sensorEnrichmentConfigs;
+  }
+
+  public interface ConfigurationVisitor{
+    void visit(ConfigurationType configurationType, String name, String data);
+  }
+  public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback) throws Exception {
+    //Output global configs
+    {
+      ConfigurationType configType = ConfigurationType.GLOBAL;
+      byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot());
+      callback.visit(configType, "global", new String(globalConfigData));
+    }
+    //Output sensor specific configs
+    {
+      ConfigurationType configType = ConfigurationType.SENSOR;
+      List<String> children = client.getChildren().forPath(configType.getZookeeperRoot());
+      for (String child : children) {
+        byte[] data = client.getData().forPath(configType.getZookeeperRoot() + "/" + child);
+        callback.visit(configType, child, new String(data));
+      }
+    }
+  }
+  public static void dumpConfigs(PrintStream out, CuratorFramework client) throws Exception {
+    ConfigurationsUtils.visitConfigs(client, (type, name, data) -> {
+      type.deserialize(data);
+      out.println(type + " Config: " + name + "\n" + data);
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
deleted file mode 100644
index bcc91fa..0000000
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/EnrichmentConfig.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/**
- * 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;
-
-import com.google.common.base.Joiner;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.metron.common.Constants;
-import org.apache.metron.common.cli.ConfigurationsUtils;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-
-public class EnrichmentConfig {
-  public static enum Type {
-     THREAT_INTEL
-    ,ENRICHMENT
-  }
-
-  protected static final Logger _LOG = LoggerFactory.getLogger(EnrichmentConfig.class);
-  public static class FieldList {
-    Type type;
-    Map<String, List<String>> fieldToEnrichmentTypes;
-
-    public Type getType() {
-      return type;
-    }
-
-    public void setType(Type type) {
-      this.type = type;
-    }
-
-    public Map<String, List<String>> getFieldToEnrichmentTypes() {
-      return fieldToEnrichmentTypes;
-    }
-
-    public void setFieldToEnrichmentTypes(Map<String, List<String>> fieldToEnrichmentTypes) {
-      this.fieldToEnrichmentTypes = fieldToEnrichmentTypes;
-    }
-  }
-  public String zkQuorum;
-  public Map<String, FieldList> sensorToFieldList;
-
-  public String getZkQuorum() {
-    return zkQuorum;
-  }
-
-  public void setZkQuorum(String zkQuorum) {
-    this.zkQuorum = zkQuorum;
-  }
-
-  public Map<String, FieldList> getSensorToFieldList() {
-    return sensorToFieldList;
-  }
-
-  public void setSensorToFieldList(Map<String, FieldList> sensorToFieldList) {
-    this.sensorToFieldList = sensorToFieldList;
-  }
-
-  public void updateSensorConfigs( ) throws Exception {
-    CuratorFramework client = ConfigurationsUtils.getClient(getZkQuorum());
-    try {
-      client.start();
-      updateSensorConfigs(new ZKSourceConfigHandler(client), sensorToFieldList);
-    }
-    finally {
-      client.close();
-    }
-  }
-
-  public static interface SourceConfigHandler {
-    SensorEnrichmentConfig readConfig(String sensor) throws Exception;
-    void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception;
-  }
-
-  public static class ZKSourceConfigHandler implements SourceConfigHandler {
-    CuratorFramework client;
-    public ZKSourceConfigHandler(CuratorFramework client) {
-      this.client = client;
-    }
-    @Override
-    public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
-      SensorEnrichmentConfig sensorEnrichmentConfig = new SensorEnrichmentConfig();
-      try {
-        sensorEnrichmentConfig = SensorEnrichmentConfig.fromBytes(ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensor, client));
-      }catch (KeeperException.NoNodeException e) {
-        sensorEnrichmentConfig.setIndex(sensor);
-        sensorEnrichmentConfig.setBatchSize(1);
-      }
-      return sensorEnrichmentConfig;
-    }
-
-    @Override
-    public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
-      ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensor, config.toJSON().getBytes(), client);
-    }
-  }
-
-  public static void updateSensorConfigs( SourceConfigHandler scHandler
-                                        , Map<String, FieldList> sensorToFieldList
-                                        ) throws Exception
-  {
-    Map<String, SensorEnrichmentConfig> sourceConfigsChanged = new HashMap<>();
-    for (Map.Entry<String, FieldList> kv : sensorToFieldList.entrySet()) {
-      SensorEnrichmentConfig config = sourceConfigsChanged.get(kv.getKey());
-      if(config == null) {
-        config = scHandler.readConfig(kv.getKey());
-        if(_LOG.isDebugEnabled()) {
-          _LOG.debug(config.toJSON());
-        }
-      }
-      Map<String, List<String> > fieldMap = null;
-      Map<String, List<String> > fieldToTypeMap = null;
-      List<String> fieldList = null;
-      if(kv.getValue().type == Type.THREAT_INTEL) {
-        fieldMap = config.getThreatIntelFieldMap();
-        if(fieldMap!= null) {
-          fieldList = fieldMap.get(Constants.SIMPLE_HBASE_THREAT_INTEL);
-        } else {
-          fieldMap = new HashMap<>();
-        }
-        if(fieldList == null) {
-          fieldList = new ArrayList<>();
-          fieldMap.put(Constants.SIMPLE_HBASE_THREAT_INTEL, fieldList);
-        }
-        fieldToTypeMap = config.getFieldToThreatIntelTypeMap();
-        if(fieldToTypeMap == null) {
-          fieldToTypeMap = new HashMap<>();
-          config.setFieldToThreatIntelTypeMap(fieldToTypeMap);
-        }
-      }
-      else if(kv.getValue().type == Type.ENRICHMENT) {
-        fieldMap = config.getEnrichmentFieldMap();
-        if(fieldMap!= null) {
-          fieldList = fieldMap.get(Constants.SIMPLE_HBASE_ENRICHMENT);
-        } else {
-          fieldMap = new HashMap<>();
-        }
-        if(fieldList == null) {
-          fieldList = new ArrayList<>();
-          fieldMap.put(Constants.SIMPLE_HBASE_ENRICHMENT, fieldList);
-        }
-        fieldToTypeMap = config.getFieldToEnrichmentTypeMap();
-        if(fieldToTypeMap == null) {
-          fieldToTypeMap = new HashMap<>();
-          config.setFieldToEnrichmentTypeMap(fieldToTypeMap);
-        }
-      }
-      if(fieldToTypeMap == null  || fieldMap == null) {
-        _LOG.debug("fieldToTypeMap is null or fieldMap is null, so skipping");
-        continue;
-      }
-      //Add the additional fields to the field list associated with the hbase adapter
-      {
-        HashSet<String> fieldSet = new HashSet<>(fieldList);
-        List<String> additionalFields = new ArrayList<>();
-        for (String field : kv.getValue().getFieldToEnrichmentTypes().keySet()) {
-          if (!fieldSet.contains(field)) {
-            additionalFields.add(field);
-          }
-        }
-        //adding only the ones that we don't already have to the field list
-        if (additionalFields.size() > 0) {
-          _LOG.debug("Adding additional fields: " + Joiner.on(',').join(additionalFields));
-          fieldList.addAll(additionalFields);
-          sourceConfigsChanged.put(kv.getKey(), config);
-        }
-      }
-      //Add the additional enrichment types to the mapping between the fields
-      {
-        for(Map.Entry<String, List<String>> fieldToType : kv.getValue().getFieldToEnrichmentTypes().entrySet()) {
-          String field = fieldToType.getKey();
-          final HashSet<String> types = new HashSet<>(fieldToType.getValue());
-          int sizeBefore = 0;
-          if(fieldToTypeMap.containsKey(field)) {
-            List<String> typeList = fieldToTypeMap.get(field);
-            sizeBefore = new HashSet<>(typeList).size();
-            types.addAll(typeList);
-          }
-          int sizeAfter = types.size();
-          boolean changed = sizeBefore != sizeAfter;
-          if(changed) {
-            fieldToTypeMap.put(field, new ArrayList<String>() {{
-                addAll(types);
-              }});
-            sourceConfigsChanged.put(kv.getKey(), config);
-          }
-        }
-      }
-    }
-    for(Map.Entry<String, SensorEnrichmentConfig> kv : sourceConfigsChanged.entrySet()) {
-      scHandler.persistConfig(kv.getKey(), kv.getValue());
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
deleted file mode 100644
index 6a45ec9..0000000
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/SensorEnrichmentConfig.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.metron.common.utils.JSONUtils;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class SensorEnrichmentConfig {
-
-  private String index;
-  private Map<String, List<String>> enrichmentFieldMap = new HashMap<>();
-  private Map<String, List<String>> threatIntelFieldMap = new HashMap<>();
-  private Map<String, List<String>> fieldToEnrichmentTypeMap = new HashMap<>();
-  private Map<String, List<String>> fieldToThreatIntelTypeMap = new HashMap<>();
-  private int batchSize;
-
-  public String getIndex() {
-    return index;
-  }
-
-  public void setIndex(String index) {
-    this.index = index;
-  }
-
-  public Map<String, List<String>> getEnrichmentFieldMap() {
-    return enrichmentFieldMap;
-  }
-
-  public void setEnrichmentFieldMap(Map<String, List<String>> enrichmentFieldMap) {
-    this.enrichmentFieldMap = enrichmentFieldMap;
-  }
-
-  public Map<String, List<String>> getThreatIntelFieldMap() {
-    return threatIntelFieldMap;
-  }
-
-  public void setThreatIntelFieldMap(Map<String, List<String>> threatIntelFieldMap) {
-    this.threatIntelFieldMap = threatIntelFieldMap;
-  }
-
-  public Map<String, List<String>> getFieldToEnrichmentTypeMap() {
-    return fieldToEnrichmentTypeMap;
-  }
-
-  public Map<String, List<String>> getFieldToThreatIntelTypeMap() {
-    return fieldToThreatIntelTypeMap;
-  }
-  public void setFieldToEnrichmentTypeMap(Map<String, List<String>> fieldToEnrichmentTypeMap) {
-    this.fieldToEnrichmentTypeMap = fieldToEnrichmentTypeMap;
-  }
-
-  public void setFieldToThreatIntelTypeMap(Map<String, List<String>> fieldToThreatIntelTypeMap) {
-    this.fieldToThreatIntelTypeMap= fieldToThreatIntelTypeMap;
-  }
-  public int getBatchSize() {
-    return batchSize;
-  }
-
-  public void setBatchSize(int batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  public static SensorEnrichmentConfig fromBytes(byte[] config) throws IOException {
-    return JSONUtils.INSTANCE.load(new String(config), SensorEnrichmentConfig.class);
-  }
-
-  public String toJSON() throws JsonProcessingException {
-    return JSONUtils.INSTANCE.toJSON(this, true);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    SensorEnrichmentConfig that = (SensorEnrichmentConfig) o;
-
-    if (getBatchSize() != that.getBatchSize()) return false;
-    if (getIndex() != null ? !getIndex().equals(that.getIndex()) : that.getIndex() != null) return false;
-    if (getEnrichmentFieldMap() != null ? !getEnrichmentFieldMap().equals(that.getEnrichmentFieldMap()) : that.getEnrichmentFieldMap() != null)
-      return false;
-    if (getThreatIntelFieldMap() != null ? !getThreatIntelFieldMap().equals(that.getThreatIntelFieldMap()) : that.getThreatIntelFieldMap() != null)
-      return false;
-    if (getFieldToEnrichmentTypeMap() != null ? !getFieldToEnrichmentTypeMap().equals(that.getFieldToEnrichmentTypeMap()) : that.getFieldToEnrichmentTypeMap() != null)
-      return false;
-    return getFieldToThreatIntelTypeMap() != null ? getFieldToThreatIntelTypeMap().equals(that.getFieldToThreatIntelTypeMap()) : that.getFieldToThreatIntelTypeMap() == null;
-
-  }
-
-  @Override
-  public String toString() {
-    return "{index=" + index + ", batchSize=" + batchSize +
-            ", enrichmentFieldMap=" + enrichmentFieldMap +
-            ", threatIntelFieldMap" + threatIntelFieldMap +
-            ", fieldToEnrichmentTypeMap=" + fieldToEnrichmentTypeMap +
-            ", fieldToThreatIntelTypeMap=" + fieldToThreatIntelTypeMap + "}";
-  }
-
-  @Override
-  public int hashCode() {
-    int result = getIndex() != null ? getIndex().hashCode() : 0;
-    result = 31 * result + (getEnrichmentFieldMap() != null ? getEnrichmentFieldMap().hashCode() : 0);
-    result = 31 * result + (getThreatIntelFieldMap() != null ? getThreatIntelFieldMap().hashCode() : 0);
-    result = 31 * result + (getFieldToEnrichmentTypeMap() != null ? getFieldToEnrichmentTypeMap().hashCode() : 0);
-    result = 31 * result + (getFieldToThreatIntelTypeMap() != null ? getFieldToThreatIntelTypeMap().hashCode() : 0);
-    result = 31 * result + getBatchSize();
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/EnrichmentConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/EnrichmentConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/EnrichmentConfig.java
new file mode 100644
index 0000000..af6c148
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/enrichment/EnrichmentConfig.java
@@ -0,0 +1,71 @@
+/**
+ * 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;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class EnrichmentConfig {
+  private Map<String, List<String>> fieldMap = new HashMap<>();
+  private Map<String, List<String>> fieldToTypeMap = new HashMap<>();
+
+  public Map<String, List<String>> getFieldMap() {
+    return fieldMap;
+  }
+
+  public void setFieldMap(Map<String, List<String>> fieldMap) {
+    this.fieldMap = fieldMap;
+  }
+
+  public Map<String, List<String>> getFieldToTypeMap() {
+    return fieldToTypeMap;
+  }
+
+  public void setFieldToTypeMap(Map<String, List<String>> fieldToTypeMap) {
+    this.fieldToTypeMap = fieldToTypeMap;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    EnrichmentConfig that = (EnrichmentConfig) o;
+
+    if (getFieldMap() != null ? !getFieldMap().equals(that.getFieldMap()) : that.getFieldMap() != null) return false;
+    return getFieldToTypeMap() != null ? getFieldToTypeMap().equals(that.getFieldToTypeMap()) : that.getFieldToTypeMap() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getFieldMap() != null ? getFieldMap().hashCode() : 0;
+    result = 31 * result + (getFieldToTypeMap() != null ? getFieldToTypeMap().hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "EnrichmentConfig{" +
+            "fieldMap=" + fieldMap +
+            ", fieldToTypeMap=" + fieldToTypeMap +
+            '}';
+  }
+}


[3/5] incubator-metron git commit: METRON-141: The ability to do threat triage closes apache/incubator-metron#108

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateParser.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateParser.java
new file mode 100644
index 0000000..3586f01
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/generated/PredicateParser.java
@@ -0,0 +1,1108 @@
+// Generated from org/apache/metron/common/query/generated/Predicate.g4 by ANTLR 4.5
+package org.apache.metron.common.query.generated;
+
+//CHECKSTYLE:OFF
+/**
+ * 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.
+ */
+
+import org.antlr.v4.runtime.atn.*;
+import org.antlr.v4.runtime.dfa.DFA;
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.misc.*;
+import org.antlr.v4.runtime.tree.*;
+import java.util.List;
+import java.util.Iterator;
+import java.util.ArrayList;
+
+@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
+public class PredicateParser extends Parser {
+	static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+
+	protected static final DFA[] _decisionToDFA;
+	protected static final PredictionContextCache _sharedContextCache =
+		new PredictionContextCache();
+	public static final int
+		AND=1, OR=2, NOT=3, TRUE=4, FALSE=5, EQ=6, NEQ=7, COMMA=8, LBRACKET=9, 
+		RBRACKET=10, LPAREN=11, RPAREN=12, IN=13, NIN=14, EXISTS=15, IDENTIFIER=16, 
+		STRING_LITERAL=17, SEMI=18, COMMENT=19, WS=20;
+	public static final int
+		RULE_single_rule = 0, RULE_logical_expr = 1, RULE_comparison_expr = 2, 
+		RULE_logical_entity = 3, RULE_list_entity = 4, RULE_func_args = 5, RULE_op_list = 6, 
+		RULE_identifier_operand = 7, RULE_comparison_operand = 8, RULE_comp_operator = 9;
+	public static final String[] ruleNames = {
+		"single_rule", "logical_expr", "comparison_expr", "logical_entity", "list_entity", 
+		"func_args", "op_list", "identifier_operand", "comparison_operand", "comp_operator"
+	};
+
+	private static final String[] _LITERAL_NAMES = {
+		null, null, null, null, null, null, "'=='", "'!='", "','", "'['", "']'", 
+		"'('", "')'", "'in'", "'not in'", "'exists'", null, null, "';'"
+	};
+	private static final String[] _SYMBOLIC_NAMES = {
+		null, "AND", "OR", "NOT", "TRUE", "FALSE", "EQ", "NEQ", "COMMA", "LBRACKET", 
+		"RBRACKET", "LPAREN", "RPAREN", "IN", "NIN", "EXISTS", "IDENTIFIER", "STRING_LITERAL", 
+		"SEMI", "COMMENT", "WS"
+	};
+	public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES);
+
+	/**
+	 * @deprecated Use {@link #VOCABULARY} instead.
+	 */
+	@Deprecated
+	public static final String[] tokenNames;
+	static {
+		tokenNames = new String[_SYMBOLIC_NAMES.length];
+		for (int i = 0; i < tokenNames.length; i++) {
+			tokenNames[i] = VOCABULARY.getLiteralName(i);
+			if (tokenNames[i] == null) {
+				tokenNames[i] = VOCABULARY.getSymbolicName(i);
+			}
+
+			if (tokenNames[i] == null) {
+				tokenNames[i] = "<INVALID>";
+			}
+		}
+	}
+
+	@Override
+	@Deprecated
+	public String[] getTokenNames() {
+		return tokenNames;
+	}
+
+	@Override
+
+	public Vocabulary getVocabulary() {
+		return VOCABULARY;
+	}
+
+	@Override
+	public String getGrammarFileName() { return "Predicate.g4"; }
+
+	@Override
+	public String[] getRuleNames() { return ruleNames; }
+
+	@Override
+	public String getSerializedATN() { return _serializedATN; }
+
+	@Override
+	public ATN getATN() { return _ATN; }
+
+	public PredicateParser(TokenStream input) {
+		super(input);
+		_interp = new ParserATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache);
+	}
+	public static class Single_ruleContext extends ParserRuleContext {
+		public Logical_exprContext logical_expr() {
+			return getRuleContext(Logical_exprContext.class,0);
+		}
+		public TerminalNode EOF() { return getToken(PredicateParser.EOF, 0); }
+		public Single_ruleContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_single_rule; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterSingle_rule(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitSingle_rule(this);
+		}
+	}
+
+	public final Single_ruleContext single_rule() throws RecognitionException {
+		Single_ruleContext _localctx = new Single_ruleContext(_ctx, getState());
+		enterRule(_localctx, 0, RULE_single_rule);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(20);
+			logical_expr(0);
+			setState(21);
+			match(EOF);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Logical_exprContext extends ParserRuleContext {
+		public Logical_exprContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_logical_expr; }
+	 
+		public Logical_exprContext() { }
+		public void copyFrom(Logical_exprContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class LogicalEntityContext extends Logical_exprContext {
+		public Logical_entityContext logical_entity() {
+			return getRuleContext(Logical_entityContext.class,0);
+		}
+		public LogicalEntityContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalEntity(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalEntity(this);
+		}
+	}
+	public static class ComparisonExpressionContext extends Logical_exprContext {
+		public Comparison_exprContext comparison_expr() {
+			return getRuleContext(Comparison_exprContext.class,0);
+		}
+		public ComparisonExpressionContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterComparisonExpression(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitComparisonExpression(this);
+		}
+	}
+	public static class LogicalExpressionInParenContext extends Logical_exprContext {
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public Logical_exprContext logical_expr() {
+			return getRuleContext(Logical_exprContext.class,0);
+		}
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public LogicalExpressionInParenContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalExpressionInParen(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalExpressionInParen(this);
+		}
+	}
+	public static class NotFuncContext extends Logical_exprContext {
+		public TerminalNode NOT() { return getToken(PredicateParser.NOT, 0); }
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public Logical_exprContext logical_expr() {
+			return getRuleContext(Logical_exprContext.class,0);
+		}
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public NotFuncContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterNotFunc(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitNotFunc(this);
+		}
+	}
+	public static class LogicalExpressionAndContext extends Logical_exprContext {
+		public List<Logical_exprContext> logical_expr() {
+			return getRuleContexts(Logical_exprContext.class);
+		}
+		public Logical_exprContext logical_expr(int i) {
+			return getRuleContext(Logical_exprContext.class,i);
+		}
+		public TerminalNode AND() { return getToken(PredicateParser.AND, 0); }
+		public LogicalExpressionAndContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalExpressionAnd(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalExpressionAnd(this);
+		}
+	}
+	public static class LogicalExpressionOrContext extends Logical_exprContext {
+		public List<Logical_exprContext> logical_expr() {
+			return getRuleContexts(Logical_exprContext.class);
+		}
+		public Logical_exprContext logical_expr(int i) {
+			return getRuleContext(Logical_exprContext.class,i);
+		}
+		public TerminalNode OR() { return getToken(PredicateParser.OR, 0); }
+		public LogicalExpressionOrContext(Logical_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalExpressionOr(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalExpressionOr(this);
+		}
+	}
+
+	public final Logical_exprContext logical_expr() throws RecognitionException {
+		return logical_expr(0);
+	}
+
+	private Logical_exprContext logical_expr(int _p) throws RecognitionException {
+		ParserRuleContext _parentctx = _ctx;
+		int _parentState = getState();
+		Logical_exprContext _localctx = new Logical_exprContext(_ctx, _parentState);
+		Logical_exprContext _prevctx = _localctx;
+		int _startState = 2;
+		enterRecursionRule(_localctx, 2, RULE_logical_expr, _p);
+		try {
+			int _alt;
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(35);
+			switch ( getInterpreter().adaptivePredict(_input,0,_ctx) ) {
+			case 1:
+				{
+				_localctx = new ComparisonExpressionContext(_localctx);
+				_ctx = _localctx;
+				_prevctx = _localctx;
+
+				setState(24);
+				comparison_expr();
+				}
+				break;
+			case 2:
+				{
+				_localctx = new LogicalExpressionInParenContext(_localctx);
+				_ctx = _localctx;
+				_prevctx = _localctx;
+				setState(25);
+				match(LPAREN);
+				setState(26);
+				logical_expr(0);
+				setState(27);
+				match(RPAREN);
+				}
+				break;
+			case 3:
+				{
+				_localctx = new NotFuncContext(_localctx);
+				_ctx = _localctx;
+				_prevctx = _localctx;
+				setState(29);
+				match(NOT);
+				setState(30);
+				match(LPAREN);
+				setState(31);
+				logical_expr(0);
+				setState(32);
+				match(RPAREN);
+				}
+				break;
+			case 4:
+				{
+				_localctx = new LogicalEntityContext(_localctx);
+				_ctx = _localctx;
+				_prevctx = _localctx;
+				setState(34);
+				logical_entity();
+				}
+				break;
+			}
+			_ctx.stop = _input.LT(-1);
+			setState(45);
+			_errHandler.sync(this);
+			_alt = getInterpreter().adaptivePredict(_input,2,_ctx);
+			while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
+				if ( _alt==1 ) {
+					if ( _parseListeners!=null ) triggerExitRuleEvent();
+					_prevctx = _localctx;
+					{
+					setState(43);
+					switch ( getInterpreter().adaptivePredict(_input,1,_ctx) ) {
+					case 1:
+						{
+						_localctx = new LogicalExpressionAndContext(new Logical_exprContext(_parentctx, _parentState));
+						pushNewRecursionContext(_localctx, _startState, RULE_logical_expr);
+						setState(37);
+						if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)");
+						setState(38);
+						match(AND);
+						setState(39);
+						logical_expr(7);
+						}
+						break;
+					case 2:
+						{
+						_localctx = new LogicalExpressionOrContext(new Logical_exprContext(_parentctx, _parentState));
+						pushNewRecursionContext(_localctx, _startState, RULE_logical_expr);
+						setState(40);
+						if (!(precpred(_ctx, 5))) throw new FailedPredicateException(this, "precpred(_ctx, 5)");
+						setState(41);
+						match(OR);
+						setState(42);
+						logical_expr(6);
+						}
+						break;
+					}
+					} 
+				}
+				setState(47);
+				_errHandler.sync(this);
+				_alt = getInterpreter().adaptivePredict(_input,2,_ctx);
+			}
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			unrollRecursionContexts(_parentctx);
+		}
+		return _localctx;
+	}
+
+	public static class Comparison_exprContext extends ParserRuleContext {
+		public Comparison_exprContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_comparison_expr; }
+	 
+		public Comparison_exprContext() { }
+		public void copyFrom(Comparison_exprContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class ComparisonExpressionParensContext extends Comparison_exprContext {
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public Comparison_exprContext comparison_expr() {
+			return getRuleContext(Comparison_exprContext.class,0);
+		}
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public ComparisonExpressionParensContext(Comparison_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterComparisonExpressionParens(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitComparisonExpressionParens(this);
+		}
+	}
+	public static class InExpressionContext extends Comparison_exprContext {
+		public Identifier_operandContext identifier_operand() {
+			return getRuleContext(Identifier_operandContext.class,0);
+		}
+		public TerminalNode IN() { return getToken(PredicateParser.IN, 0); }
+		public List_entityContext list_entity() {
+			return getRuleContext(List_entityContext.class,0);
+		}
+		public InExpressionContext(Comparison_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterInExpression(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitInExpression(this);
+		}
+	}
+	public static class ComparisonExpressionWithOperatorContext extends Comparison_exprContext {
+		public List<Comparison_operandContext> comparison_operand() {
+			return getRuleContexts(Comparison_operandContext.class);
+		}
+		public Comparison_operandContext comparison_operand(int i) {
+			return getRuleContext(Comparison_operandContext.class,i);
+		}
+		public Comp_operatorContext comp_operator() {
+			return getRuleContext(Comp_operatorContext.class,0);
+		}
+		public ComparisonExpressionWithOperatorContext(Comparison_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterComparisonExpressionWithOperator(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitComparisonExpressionWithOperator(this);
+		}
+	}
+	public static class NInExpressionContext extends Comparison_exprContext {
+		public Identifier_operandContext identifier_operand() {
+			return getRuleContext(Identifier_operandContext.class,0);
+		}
+		public TerminalNode NIN() { return getToken(PredicateParser.NIN, 0); }
+		public List_entityContext list_entity() {
+			return getRuleContext(List_entityContext.class,0);
+		}
+		public NInExpressionContext(Comparison_exprContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterNInExpression(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitNInExpression(this);
+		}
+	}
+
+	public final Comparison_exprContext comparison_expr() throws RecognitionException {
+		Comparison_exprContext _localctx = new Comparison_exprContext(_ctx, getState());
+		enterRule(_localctx, 4, RULE_comparison_expr);
+		try {
+			setState(64);
+			switch ( getInterpreter().adaptivePredict(_input,3,_ctx) ) {
+			case 1:
+				_localctx = new ComparisonExpressionWithOperatorContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(48);
+				comparison_operand();
+				setState(49);
+				comp_operator();
+				setState(50);
+				comparison_operand();
+				}
+				break;
+			case 2:
+				_localctx = new InExpressionContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(52);
+				identifier_operand();
+				setState(53);
+				match(IN);
+				setState(54);
+				list_entity();
+				}
+				break;
+			case 3:
+				_localctx = new NInExpressionContext(_localctx);
+				enterOuterAlt(_localctx, 3);
+				{
+				setState(56);
+				identifier_operand();
+				setState(57);
+				match(NIN);
+				setState(58);
+				list_entity();
+				}
+				break;
+			case 4:
+				_localctx = new ComparisonExpressionParensContext(_localctx);
+				enterOuterAlt(_localctx, 4);
+				{
+				setState(60);
+				match(LPAREN);
+				setState(61);
+				comparison_expr();
+				setState(62);
+				match(RPAREN);
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Logical_entityContext extends ParserRuleContext {
+		public Logical_entityContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_logical_entity; }
+	 
+		public Logical_entityContext() { }
+		public void copyFrom(Logical_entityContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class LogicalFuncContext extends Logical_entityContext {
+		public TerminalNode IDENTIFIER() { return getToken(PredicateParser.IDENTIFIER, 0); }
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public Func_argsContext func_args() {
+			return getRuleContext(Func_argsContext.class,0);
+		}
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public LogicalFuncContext(Logical_entityContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalFunc(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalFunc(this);
+		}
+	}
+	public static class LogicalConstContext extends Logical_entityContext {
+		public TerminalNode TRUE() { return getToken(PredicateParser.TRUE, 0); }
+		public TerminalNode FALSE() { return getToken(PredicateParser.FALSE, 0); }
+		public LogicalConstContext(Logical_entityContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalConst(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalConst(this);
+		}
+	}
+	public static class ExistsFuncContext extends Logical_entityContext {
+		public TerminalNode EXISTS() { return getToken(PredicateParser.EXISTS, 0); }
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public TerminalNode IDENTIFIER() { return getToken(PredicateParser.IDENTIFIER, 0); }
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public ExistsFuncContext(Logical_entityContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterExistsFunc(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitExistsFunc(this);
+		}
+	}
+
+	public final Logical_entityContext logical_entity() throws RecognitionException {
+		Logical_entityContext _localctx = new Logical_entityContext(_ctx, getState());
+		enterRule(_localctx, 6, RULE_logical_entity);
+		int _la;
+		try {
+			setState(76);
+			switch (_input.LA(1)) {
+			case TRUE:
+			case FALSE:
+				_localctx = new LogicalConstContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(66);
+				_la = _input.LA(1);
+				if ( !(_la==TRUE || _la==FALSE) ) {
+				_errHandler.recoverInline(this);
+				} else {
+					consume();
+				}
+				}
+				break;
+			case EXISTS:
+				_localctx = new ExistsFuncContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(67);
+				match(EXISTS);
+				setState(68);
+				match(LPAREN);
+				setState(69);
+				match(IDENTIFIER);
+				setState(70);
+				match(RPAREN);
+				}
+				break;
+			case IDENTIFIER:
+				_localctx = new LogicalFuncContext(_localctx);
+				enterOuterAlt(_localctx, 3);
+				{
+				setState(71);
+				match(IDENTIFIER);
+				setState(72);
+				match(LPAREN);
+				setState(73);
+				func_args();
+				setState(74);
+				match(RPAREN);
+				}
+				break;
+			default:
+				throw new NoViableAltException(this);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class List_entityContext extends ParserRuleContext {
+		public TerminalNode LBRACKET() { return getToken(PredicateParser.LBRACKET, 0); }
+		public Op_listContext op_list() {
+			return getRuleContext(Op_listContext.class,0);
+		}
+		public TerminalNode RBRACKET() { return getToken(PredicateParser.RBRACKET, 0); }
+		public List_entityContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_list_entity; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterList_entity(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitList_entity(this);
+		}
+	}
+
+	public final List_entityContext list_entity() throws RecognitionException {
+		List_entityContext _localctx = new List_entityContext(_ctx, getState());
+		enterRule(_localctx, 8, RULE_list_entity);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(78);
+			match(LBRACKET);
+			setState(79);
+			op_list(0);
+			setState(80);
+			match(RBRACKET);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Func_argsContext extends ParserRuleContext {
+		public Op_listContext op_list() {
+			return getRuleContext(Op_listContext.class,0);
+		}
+		public Func_argsContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_func_args; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterFunc_args(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitFunc_args(this);
+		}
+	}
+
+	public final Func_argsContext func_args() throws RecognitionException {
+		Func_argsContext _localctx = new Func_argsContext(_ctx, getState());
+		enterRule(_localctx, 10, RULE_func_args);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(82);
+			op_list(0);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Op_listContext extends ParserRuleContext {
+		public Identifier_operandContext identifier_operand() {
+			return getRuleContext(Identifier_operandContext.class,0);
+		}
+		public Op_listContext op_list() {
+			return getRuleContext(Op_listContext.class,0);
+		}
+		public TerminalNode COMMA() { return getToken(PredicateParser.COMMA, 0); }
+		public Op_listContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_op_list; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterOp_list(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitOp_list(this);
+		}
+	}
+
+	public final Op_listContext op_list() throws RecognitionException {
+		return op_list(0);
+	}
+
+	private Op_listContext op_list(int _p) throws RecognitionException {
+		ParserRuleContext _parentctx = _ctx;
+		int _parentState = getState();
+		Op_listContext _localctx = new Op_listContext(_ctx, _parentState);
+		Op_listContext _prevctx = _localctx;
+		int _startState = 12;
+		enterRecursionRule(_localctx, 12, RULE_op_list, _p);
+		try {
+			int _alt;
+			enterOuterAlt(_localctx, 1);
+			{
+			{
+			setState(85);
+			identifier_operand();
+			}
+			_ctx.stop = _input.LT(-1);
+			setState(92);
+			_errHandler.sync(this);
+			_alt = getInterpreter().adaptivePredict(_input,5,_ctx);
+			while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
+				if ( _alt==1 ) {
+					if ( _parseListeners!=null ) triggerExitRuleEvent();
+					_prevctx = _localctx;
+					{
+					{
+					_localctx = new Op_listContext(_parentctx, _parentState);
+					pushNewRecursionContext(_localctx, _startState, RULE_op_list);
+					setState(87);
+					if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
+					setState(88);
+					match(COMMA);
+					setState(89);
+					identifier_operand();
+					}
+					} 
+				}
+				setState(94);
+				_errHandler.sync(this);
+				_alt = getInterpreter().adaptivePredict(_input,5,_ctx);
+			}
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			unrollRecursionContexts(_parentctx);
+		}
+		return _localctx;
+	}
+
+	public static class Identifier_operandContext extends ParserRuleContext {
+		public Identifier_operandContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_identifier_operand; }
+	 
+		public Identifier_operandContext() { }
+		public void copyFrom(Identifier_operandContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class LogicalVariableContext extends Identifier_operandContext {
+		public TerminalNode IDENTIFIER() { return getToken(PredicateParser.IDENTIFIER, 0); }
+		public LogicalVariableContext(Identifier_operandContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalVariable(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalVariable(this);
+		}
+	}
+	public static class StringLiteralContext extends Identifier_operandContext {
+		public TerminalNode STRING_LITERAL() { return getToken(PredicateParser.STRING_LITERAL, 0); }
+		public StringLiteralContext(Identifier_operandContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterStringLiteral(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitStringLiteral(this);
+		}
+	}
+	public static class StringFuncContext extends Identifier_operandContext {
+		public TerminalNode IDENTIFIER() { return getToken(PredicateParser.IDENTIFIER, 0); }
+		public TerminalNode LPAREN() { return getToken(PredicateParser.LPAREN, 0); }
+		public Func_argsContext func_args() {
+			return getRuleContext(Func_argsContext.class,0);
+		}
+		public TerminalNode RPAREN() { return getToken(PredicateParser.RPAREN, 0); }
+		public StringFuncContext(Identifier_operandContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterStringFunc(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitStringFunc(this);
+		}
+	}
+
+	public final Identifier_operandContext identifier_operand() throws RecognitionException {
+		Identifier_operandContext _localctx = new Identifier_operandContext(_ctx, getState());
+		enterRule(_localctx, 14, RULE_identifier_operand);
+		try {
+			setState(102);
+			switch ( getInterpreter().adaptivePredict(_input,6,_ctx) ) {
+			case 1:
+				_localctx = new StringLiteralContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(95);
+				match(STRING_LITERAL);
+				}
+				break;
+			case 2:
+				_localctx = new LogicalVariableContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(96);
+				match(IDENTIFIER);
+				}
+				break;
+			case 3:
+				_localctx = new StringFuncContext(_localctx);
+				enterOuterAlt(_localctx, 3);
+				{
+				setState(97);
+				match(IDENTIFIER);
+				setState(98);
+				match(LPAREN);
+				setState(99);
+				func_args();
+				setState(100);
+				match(RPAREN);
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Comparison_operandContext extends ParserRuleContext {
+		public Comparison_operandContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_comparison_operand; }
+	 
+		public Comparison_operandContext() { }
+		public void copyFrom(Comparison_operandContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class LogicalConstComparisonContext extends Comparison_operandContext {
+		public Logical_entityContext logical_entity() {
+			return getRuleContext(Logical_entityContext.class,0);
+		}
+		public LogicalConstComparisonContext(Comparison_operandContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterLogicalConstComparison(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitLogicalConstComparison(this);
+		}
+	}
+	public static class IdentifierOperandContext extends Comparison_operandContext {
+		public Identifier_operandContext identifier_operand() {
+			return getRuleContext(Identifier_operandContext.class,0);
+		}
+		public IdentifierOperandContext(Comparison_operandContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterIdentifierOperand(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitIdentifierOperand(this);
+		}
+	}
+
+	public final Comparison_operandContext comparison_operand() throws RecognitionException {
+		Comparison_operandContext _localctx = new Comparison_operandContext(_ctx, getState());
+		enterRule(_localctx, 16, RULE_comparison_operand);
+		try {
+			setState(106);
+			switch ( getInterpreter().adaptivePredict(_input,7,_ctx) ) {
+			case 1:
+				_localctx = new IdentifierOperandContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(104);
+				identifier_operand();
+				}
+				break;
+			case 2:
+				_localctx = new LogicalConstComparisonContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(105);
+				logical_entity();
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Comp_operatorContext extends ParserRuleContext {
+		public Comp_operatorContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_comp_operator; }
+	 
+		public Comp_operatorContext() { }
+		public void copyFrom(Comp_operatorContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class ComparisonOpContext extends Comp_operatorContext {
+		public TerminalNode EQ() { return getToken(PredicateParser.EQ, 0); }
+		public TerminalNode NEQ() { return getToken(PredicateParser.NEQ, 0); }
+		public ComparisonOpContext(Comp_operatorContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).enterComparisonOp(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof PredicateListener ) ((PredicateListener)listener).exitComparisonOp(this);
+		}
+	}
+
+	public final Comp_operatorContext comp_operator() throws RecognitionException {
+		Comp_operatorContext _localctx = new Comp_operatorContext(_ctx, getState());
+		enterRule(_localctx, 18, RULE_comp_operator);
+		int _la;
+		try {
+			_localctx = new ComparisonOpContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(108);
+			_la = _input.LA(1);
+			if ( !(_la==EQ || _la==NEQ) ) {
+			_errHandler.recoverInline(this);
+			} else {
+				consume();
+			}
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) {
+		switch (ruleIndex) {
+		case 1:
+			return logical_expr_sempred((Logical_exprContext)_localctx, predIndex);
+		case 6:
+			return op_list_sempred((Op_listContext)_localctx, predIndex);
+		}
+		return true;
+	}
+	private boolean logical_expr_sempred(Logical_exprContext _localctx, int predIndex) {
+		switch (predIndex) {
+		case 0:
+			return precpred(_ctx, 6);
+		case 1:
+			return precpred(_ctx, 5);
+		}
+		return true;
+	}
+	private boolean op_list_sempred(Op_listContext _localctx, int predIndex) {
+		switch (predIndex) {
+		case 2:
+			return precpred(_ctx, 1);
+		}
+		return true;
+	}
+
+	public static final String _serializedATN =
+		"\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\26q\4\2\t\2\4\3\t"+
+		"\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13\t\13\3"+
+		"\2\3\2\3\2\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3&\n\3\3"+
+		"\3\3\3\3\3\3\3\3\3\3\3\7\3.\n\3\f\3\16\3\61\13\3\3\4\3\4\3\4\3\4\3\4\3"+
+		"\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\5\4C\n\4\3\5\3\5\3\5\3\5\3"+
+		"\5\3\5\3\5\3\5\3\5\3\5\5\5O\n\5\3\6\3\6\3\6\3\6\3\7\3\7\3\b\3\b\3\b\3"+
+		"\b\3\b\3\b\7\b]\n\b\f\b\16\b`\13\b\3\t\3\t\3\t\3\t\3\t\3\t\3\t\5\ti\n"+
+		"\t\3\n\3\n\5\nm\n\n\3\13\3\13\3\13\2\4\4\16\f\2\4\6\b\n\f\16\20\22\24"+
+		"\2\4\3\2\6\7\3\2\b\tt\2\26\3\2\2\2\4%\3\2\2\2\6B\3\2\2\2\bN\3\2\2\2\n"+
+		"P\3\2\2\2\fT\3\2\2\2\16V\3\2\2\2\20h\3\2\2\2\22l\3\2\2\2\24n\3\2\2\2\26"+
+		"\27\5\4\3\2\27\30\7\2\2\3\30\3\3\2\2\2\31\32\b\3\1\2\32&\5\6\4\2\33\34"+
+		"\7\r\2\2\34\35\5\4\3\2\35\36\7\16\2\2\36&\3\2\2\2\37 \7\5\2\2 !\7\r\2"+
+		"\2!\"\5\4\3\2\"#\7\16\2\2#&\3\2\2\2$&\5\b\5\2%\31\3\2\2\2%\33\3\2\2\2"+
+		"%\37\3\2\2\2%$\3\2\2\2&/\3\2\2\2\'(\f\b\2\2()\7\3\2\2).\5\4\3\t*+\f\7"+
+		"\2\2+,\7\4\2\2,.\5\4\3\b-\'\3\2\2\2-*\3\2\2\2.\61\3\2\2\2/-\3\2\2\2/\60"+
+		"\3\2\2\2\60\5\3\2\2\2\61/\3\2\2\2\62\63\5\22\n\2\63\64\5\24\13\2\64\65"+
+		"\5\22\n\2\65C\3\2\2\2\66\67\5\20\t\2\678\7\17\2\289\5\n\6\29C\3\2\2\2"+
+		":;\5\20\t\2;<\7\20\2\2<=\5\n\6\2=C\3\2\2\2>?\7\r\2\2?@\5\6\4\2@A\7\16"+
+		"\2\2AC\3\2\2\2B\62\3\2\2\2B\66\3\2\2\2B:\3\2\2\2B>\3\2\2\2C\7\3\2\2\2"+
+		"DO\t\2\2\2EF\7\21\2\2FG\7\r\2\2GH\7\22\2\2HO\7\16\2\2IJ\7\22\2\2JK\7\r"+
+		"\2\2KL\5\f\7\2LM\7\16\2\2MO\3\2\2\2ND\3\2\2\2NE\3\2\2\2NI\3\2\2\2O\t\3"+
+		"\2\2\2PQ\7\13\2\2QR\5\16\b\2RS\7\f\2\2S\13\3\2\2\2TU\5\16\b\2U\r\3\2\2"+
+		"\2VW\b\b\1\2WX\5\20\t\2X^\3\2\2\2YZ\f\3\2\2Z[\7\n\2\2[]\5\20\t\2\\Y\3"+
+		"\2\2\2]`\3\2\2\2^\\\3\2\2\2^_\3\2\2\2_\17\3\2\2\2`^\3\2\2\2ai\7\23\2\2"+
+		"bi\7\22\2\2cd\7\22\2\2de\7\r\2\2ef\5\f\7\2fg\7\16\2\2gi\3\2\2\2ha\3\2"+
+		"\2\2hb\3\2\2\2hc\3\2\2\2i\21\3\2\2\2jm\5\20\t\2km\5\b\5\2lj\3\2\2\2lk"+
+		"\3\2\2\2m\23\3\2\2\2no\t\3\2\2o\25\3\2\2\2\n%-/BN^hl";
+	public static final ATN _ATN =
+		new ATNDeserializer().deserialize(_serializedATN.toCharArray());
+	static {
+		_decisionToDFA = new DFA[_ATN.getNumberOfDecisions()];
+		for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) {
+			_decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh b/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
index bd9e9eb..ff64e3d 100755
--- a/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
+++ b/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
@@ -30,4 +30,4 @@ export METRON_VERSION=${project.version}
 export METRON_HOME=/usr/metron/$METRON_VERSION
 export PARSERS_JAR=${project.artifactId}-$METRON_VERSION.jar
 export ZK_HOME=${ZK_HOME:-/usr/hdp/current/zookeeper-client}
-java -cp $METRON_HOME/lib/$PARSERS_JAR:$ZK_HOME/lib/* org.apache.metron.common.cli.ConfigurationsUtils "$@"
+java -cp $METRON_HOME/lib/$PARSERS_JAR:$ZK_HOME/lib/* org.apache.metron.common.cli.ConfigurationManager "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
index a791086..6df930b 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/bolt/ConfiguredBoltTest.java
@@ -24,8 +24,8 @@ import org.apache.metron.common.Constants;
 import org.apache.metron.TestConstants;
 import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
 import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -119,12 +119,12 @@ public class ConfiguredBoltTest extends BaseEnrichmentBoltTest {
     enrichmentFieldMap.put("enrichmentTest", new ArrayList<String>() {{
       add("enrichmentField");
     }});
-    testSensorConfig.setEnrichmentFieldMap(enrichmentFieldMap);
+    testSensorConfig.getEnrichment().setFieldMap(enrichmentFieldMap);
     Map<String, List<String>> threatIntelFieldMap = new HashMap<>();
     threatIntelFieldMap.put("threatIntelTest", new ArrayList<String>() {{
       add("threatIntelField");
     }});
-    testSensorConfig.setThreatIntelFieldMap(threatIntelFieldMap);
+    testSensorConfig.getThreatIntel().setFieldMap(threatIntelFieldMap);
     sampleConfigurations.updateSensorEnrichmentConfig(sensorType, testSensorConfig);
     ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, testSensorConfig, zookeeperUrl);
     waitForConfigUpdate(sensorType);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
new file mode 100644
index 0000000..e17b877
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationManagerIntegrationTest.java
@@ -0,0 +1,176 @@
+/**
+ * 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.cli;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Iterables;
+import org.apache.commons.cli.PosixParser;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.metron.TestConstants;
+import org.apache.metron.common.configuration.ConfigurationType;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.*;
+
+public class ConfigurationManagerIntegrationTest {
+  private TestingServer testZkServer;
+  private CuratorFramework client;
+  private String zookeeperUrl;
+  private String outDir = "target/configs";
+  private Set<String> sensors = new HashSet<>();
+
+  private void cleanDir(File rootDir) throws IOException {
+    if(rootDir.isDirectory()) {
+      try {
+        Files.delete(Paths.get(rootDir.toURI()));
+      } catch (DirectoryNotEmptyException dne) {
+        for(File f : rootDir.listFiles()) {
+          cleanDir(f);
+        }
+        rootDir.delete();
+      }
+    }
+    else {
+      rootDir.delete();
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    testZkServer = new TestingServer(true);
+    zookeeperUrl = testZkServer.getConnectString();
+    client = ConfigurationsUtils.getClient(zookeeperUrl);
+    client.start();
+    File sensorDir = new File(new File(TestConstants.SAMPLE_CONFIG_PATH), ConfigurationType.SENSOR.getDirectory());
+    sensors.addAll(Collections2.transform(
+             Arrays.asList(sensorDir.list())
+            ,s -> Iterables.getFirst(Splitter.on('.').split(s), "null")
+                                         )
+                  );
+    pushConfigs();
+  }
+
+  private void pushConfigs() throws Exception {
+    String[] args = new String[]{
+            "-z", zookeeperUrl
+            , "--mode", "PUSH"
+            , "--input_dir", TestConstants.SAMPLE_CONFIG_PATH
+    };
+    ConfigurationManager manager = new ConfigurationManager();
+    manager.run(ConfigurationManager.ConfigurationOptions.parse(new PosixParser(), args));
+  }
+
+  public void pullConfigs(boolean force) throws Exception {
+    String[] args = null;
+    if(force) {
+      args = new String[]{
+              "-z", zookeeperUrl
+              , "--mode", "PULL"
+              , "--output_dir", outDir
+              , "--force"
+      };
+    }
+    else {
+      args = new String[]{
+              "-z", zookeeperUrl
+              , "--mode", "PULL"
+              , "--output_dir", outDir
+      };
+    }
+    ConfigurationManager manager = new ConfigurationManager();
+    manager.run(ConfigurationManager.ConfigurationOptions.parse(new PosixParser(), args));
+  }
+
+  public void validateConfigsOnDisk(File configDir) throws IOException {
+    File globalConfigFile = new File(configDir, "global.json");
+    Assert.assertTrue("Global config does not exist", globalConfigFile.exists());
+    validateConfig("global", ConfigurationType.GLOBAL, new String(Files.readAllBytes(Paths.get(globalConfigFile.toURI()))));
+    for(String sensor : sensors) {
+      File sensorFile = new File(configDir, ConfigurationType.SENSOR.getDirectory() + "/" + sensor + ".json");
+      Assert.assertTrue(sensor + " config does not exist", sensorFile.exists());
+      validateConfig(sensor, ConfigurationType.SENSOR, new String(Files.readAllBytes(Paths.get(sensorFile.toURI()))));
+    }
+  }
+
+  @Test
+  public void testPull() throws Exception {
+    cleanDir(new File(outDir));
+    pullConfigs(false);
+    validateConfigsOnDisk(new File(outDir));
+    try {
+      //second time without force should
+      pullConfigs(false);
+      Assert.fail("Should have failed to pull configs in a directory structure that already exists.");
+    }
+    catch(IllegalStateException t) {
+      //make sure we didn't bork anything
+      validateConfigsOnDisk(new File(outDir));
+    }
+    pullConfigs(true);
+    validateConfigsOnDisk(new File(outDir));
+  }
+  public void validateConfig(String name, ConfigurationType type, String data)
+  {
+      try {
+        type.deserialize(data);
+      } catch (Exception e) {
+        Assert.fail("Unable to load config " + name + ": " + data);
+      }
+  }
+  @Test
+  public void testPush() throws Exception {
+    pushConfigs();
+    final Set<String> sensorsInZookeeper = new HashSet<>();
+    final BooleanWritable foundGlobal = new BooleanWritable(false);
+    ConfigurationsUtils.visitConfigs(client, new ConfigurationsUtils.ConfigurationVisitor() {
+      @Override
+      public void visit(ConfigurationType configurationType, String name, String data) {
+        Assert.assertTrue(data.length() > 0);
+        validateConfig(name, configurationType, data);
+        if(configurationType == ConfigurationType.GLOBAL) {
+          validateConfig(name, configurationType, data);
+          foundGlobal.set(true);
+        }
+        else {
+          sensorsInZookeeper.add(name);
+        }
+      }
+    });
+    Assert.assertEquals(sensorsInZookeeper, sensors);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    client.close();
+    testZkServer.close();
+    testZkServer.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
index bee4af7..3393b41 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/cli/ConfigurationsUtilsTest.java
@@ -21,7 +21,7 @@ import junit.framework.Assert;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
 import org.apache.metron.TestConstants;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.utils.JSONUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -74,17 +74,6 @@ public class ConfigurationsUtilsTest {
 
   }
 
-  @Test
-  public void testCmdLine() throws Exception {
-    String[] args = {"-z", zookeeperUrl, "-p", TestConstants.SAMPLE_CONFIG_PATH};
-    ConfigurationsUtils.main(args);
-    byte[] readGlobalConfigBytes = ConfigurationsUtils.readGlobalConfigBytesFromZookeeper(client);
-    Assert.assertTrue(Arrays.equals(testGlobalConfig, readGlobalConfigBytes));
-    for(String sensorType: testSensorConfigMap.keySet()) {
-      byte[] readSensorConfigBytes = ConfigurationsUtils.readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client);
-      Assert.assertTrue(Arrays.equals(testSensorConfigMap.get(sensorType), readSensorConfigBytes));
-    }
-  }
 
   @After
   public void tearDown() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/EnrichmentConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/EnrichmentConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/EnrichmentConfigTest.java
deleted file mode 100644
index b7e3e02..0000000
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/EnrichmentConfigTest.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * 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;
-
-import org.adrianwalker.multilinestring.Multiline;
-import org.apache.metron.common.Constants;
-import org.apache.metron.common.utils.JSONUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class EnrichmentConfigTest {
-  /**
-   {
-      "index": "bro",
-      "batchSize": 5,
-      "enrichmentFieldMap": {
-        "geo": ["ip_dst_addr", "ip_src_addr"],
-        "host": ["host"]
-                            },
-      "threatIntelFieldMap": {
-        "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-                             },
-      "fieldToThreatIntelTypeMap": {
-        "ip_dst_addr" : [ "malicious_ip" ]
-       ,"ip_src_addr" : [ "malicious_ip" ]
-                                   }
-    }
-   */
-  @Multiline
-  public static String sourceConfigStr;
-
-  /**
-{
-  "zkQuorum" : "localhost:2181"
- ,"sensorToFieldList" : {
-  "bro" : {
-           "type" : "THREAT_INTEL"
-          ,"fieldToEnrichmentTypes" : {
-            "ip_src_addr" : [ "playful" ]
-           ,"ip_dst_addr" : [ "playful" ]
-                                      }
-          }
-                        }
-}
-     */
-  @Multiline
-  public static String threatIntelConfigStr;
-
-  @Test
-  public void testThreatIntel() throws Exception {
-
-    SensorEnrichmentConfig broSc = JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
-
-
-    EnrichmentConfig config = JSONUtils.INSTANCE.load(threatIntelConfigStr, EnrichmentConfig.class);
-    final Map<String, SensorEnrichmentConfig> outputScs = new HashMap<>();
-    EnrichmentConfig.SourceConfigHandler scHandler = new EnrichmentConfig.SourceConfigHandler() {
-      @Override
-      public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
-        if(sensor.equals("bro")) {
-          return JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
-        }
-        else {
-          throw new IllegalStateException("Tried to retrieve an unexpected sensor: " + sensor);
-        }
-      }
-
-      @Override
-      public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
-        outputScs.put(sensor, config);
-      }
-    };
-    EnrichmentConfig.updateSensorConfigs(scHandler, config.getSensorToFieldList());
-    Assert.assertNotNull(outputScs.get("bro"));
-    Assert.assertNotSame(outputScs.get("bro"), broSc);
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getThreatIntelFieldMap().get(Constants.SIMPLE_HBASE_THREAT_INTEL).size()
-                       , 2
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getThreatIntelFieldMap()
-                                  .get(Constants.SIMPLE_HBASE_THREAT_INTEL)
-                                  .contains("ip_src_addr")
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getThreatIntelFieldMap()
-                                  .get(Constants.SIMPLE_HBASE_THREAT_INTEL)
-                                  .contains("ip_dst_addr")
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().keySet().size()
-                       , 2
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_src_addr").size()
-                       , 2
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_src_addr").contains("playful")
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_src_addr").contains("malicious_ip")
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_dst_addr").size()
-                       , 2
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_dst_addr").contains("playful")
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToThreatIntelTypeMap().get("ip_dst_addr").contains("malicious_ip")
-                       );
-  }
-
-  /**
-   {
-  "zkQuorum" : "localhost:2181"
- ,"sensorToFieldList" : {
-  "bro" : {
-           "type" : "ENRICHMENT"
-          ,"fieldToEnrichmentTypes" : {
-            "ip_src_addr" : [ "playful" ]
-           ,"ip_dst_addr" : [ "playful" ]
-                                      }
-          }
-                        }
-   }
-   */
-  @Multiline
-  public static String enrichmentConfigStr;
-  @Test
-  public void testEnrichment() throws Exception {
-
-    SensorEnrichmentConfig broSc = JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
-
-    EnrichmentConfig config = JSONUtils.INSTANCE.load(enrichmentConfigStr, EnrichmentConfig.class);
-    final Map<String, SensorEnrichmentConfig> outputScs = new HashMap<>();
-    EnrichmentConfig.SourceConfigHandler scHandler = new EnrichmentConfig.SourceConfigHandler() {
-      @Override
-      public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
-        if(sensor.equals("bro")) {
-          return JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
-        }
-        else {
-          throw new IllegalStateException("Tried to retrieve an unexpected sensor: " + sensor);
-        }
-      }
-
-      @Override
-      public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
-        outputScs.put(sensor, config);
-      }
-    };
-    EnrichmentConfig.updateSensorConfigs(scHandler, config.getSensorToFieldList());
-    Assert.assertNotNull(outputScs.get("bro"));
-    Assert.assertNotSame(outputScs.get("bro"), broSc);
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getEnrichmentFieldMap().get(Constants.SIMPLE_HBASE_ENRICHMENT).size()
-                       , 2
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getEnrichmentFieldMap()
-                                  .get(Constants.SIMPLE_HBASE_ENRICHMENT)
-                                  .contains("ip_src_addr")
-                       );
-    Assert.assertTrue( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getEnrichmentFieldMap()
-                                  .get(Constants.SIMPLE_HBASE_ENRICHMENT)
-                                  .contains("ip_dst_addr")
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToEnrichmentTypeMap().keySet().size()
-                       , 2
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToEnrichmentTypeMap().get("ip_src_addr").size()
-                       , 1
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToEnrichmentTypeMap().get("ip_src_addr").get(0)
-                       , "playful"
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToEnrichmentTypeMap().get("ip_dst_addr").size()
-                       , 1
-                       );
-    Assert.assertEquals( outputScs.get("bro").toJSON()
-                       , outputScs.get("bro").getFieldToEnrichmentTypeMap().get("ip_dst_addr").get(0)
-                       , "playful"
-                       );
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
index 93a3d54..418ee9b 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentConfigTest.java
@@ -21,7 +21,7 @@ import junit.framework.Assert;
 import nl.jqno.equalsverifier.EqualsVerifier;
 import nl.jqno.equalsverifier.Warning;
 import org.apache.metron.TestConstants;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.junit.Test;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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
new file mode 100644
index 0000000..009fc22
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/SensorEnrichmentUpdateConfigTest.java
@@ -0,0 +1,224 @@
+/**
+ * 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;
+
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentUpdateConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.utils.JSONUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SensorEnrichmentUpdateConfigTest {
+  /**
+   {
+      "index": "bro",
+      "batchSize": 5,
+      "enrichment" : {
+        "fieldMap": {
+        "geo": ["ip_dst_addr", "ip_src_addr"],
+        "host": ["host"]
+                    }
+      },
+      "threatIntel": {
+        "fieldMap": {
+          "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+                             },
+        "fieldToTypeMap": {
+          "ip_dst_addr" : [ "malicious_ip" ]
+         ,"ip_src_addr" : [ "malicious_ip" ]
+                          },
+        "triageConfig" : {
+          "riskLevelRules" : {
+            "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
+                             },
+          "aggregator" : "MAX"
+                        }
+      }
+    }
+   */
+  @Multiline
+  public static String sourceConfigStr;
+
+  /**
+{
+  "zkQuorum" : "localhost:2181"
+ ,"sensorToFieldList" : {
+  "bro" : {
+           "type" : "THREAT_INTEL"
+          ,"fieldToEnrichmentTypes" : {
+            "ip_src_addr" : [ "playful" ]
+           ,"ip_dst_addr" : [ "playful" ]
+                                      }
+          }
+                        }
+}
+     */
+  @Multiline
+  public static String threatIntelConfigStr;
+
+  @Test
+  public void testThreatIntel() throws Exception {
+
+    SensorEnrichmentConfig broSc = (SensorEnrichmentConfig) ConfigurationType.SENSOR.deserialize(sourceConfigStr);
+
+
+    SensorEnrichmentUpdateConfig config = JSONUtils.INSTANCE.load(threatIntelConfigStr, SensorEnrichmentUpdateConfig.class);
+    final Map<String, SensorEnrichmentConfig> outputScs = new HashMap<>();
+    SensorEnrichmentUpdateConfig.SourceConfigHandler scHandler = new SensorEnrichmentUpdateConfig.SourceConfigHandler() {
+      @Override
+      public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
+        if(sensor.equals("bro")) {
+          return JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
+        }
+        else {
+          throw new IllegalStateException("Tried to retrieve an unexpected sensor: " + sensor);
+        }
+      }
+
+      @Override
+      public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
+        outputScs.put(sensor, config);
+      }
+    };
+    SensorEnrichmentUpdateConfig.updateSensorConfigs(scHandler, config.getSensorToFieldList());
+    Assert.assertNotNull(outputScs.get("bro"));
+    Assert.assertNotSame(outputScs.get("bro"), broSc);
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldMap().get(Constants.SIMPLE_HBASE_THREAT_INTEL).size()
+                       , 2
+                       );
+    Assert.assertEquals(1, outputScs.get("bro").getThreatIntel().getTriageConfig().getRiskLevelRules().size());
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldMap()
+                                  .get(Constants.SIMPLE_HBASE_THREAT_INTEL)
+                                  .contains("ip_src_addr")
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldMap()
+                                  .get(Constants.SIMPLE_HBASE_THREAT_INTEL)
+                                  .contains("ip_dst_addr")
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().keySet().size()
+                       , 2
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_src_addr").size()
+                       , 2
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_src_addr").contains("playful")
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_src_addr").contains("malicious_ip")
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_dst_addr").size()
+                       , 2
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_dst_addr").contains("playful")
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getThreatIntel().getFieldToTypeMap().get("ip_dst_addr").contains("malicious_ip")
+                       );
+  }
+
+  /**
+   {
+  "zkQuorum" : "localhost:2181"
+ ,"sensorToFieldList" : {
+  "bro" : {
+           "type" : "ENRICHMENT"
+          ,"fieldToEnrichmentTypes" : {
+            "ip_src_addr" : [ "playful" ]
+           ,"ip_dst_addr" : [ "playful" ]
+                                      }
+          }
+                        }
+   }
+   */
+  @Multiline
+  public static String enrichmentConfigStr;
+  @Test
+  public void testEnrichment() throws Exception {
+
+    SensorEnrichmentConfig broSc = JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
+
+    SensorEnrichmentUpdateConfig config = JSONUtils.INSTANCE.load(enrichmentConfigStr, SensorEnrichmentUpdateConfig.class);
+    final Map<String, SensorEnrichmentConfig> outputScs = new HashMap<>();
+    SensorEnrichmentUpdateConfig.SourceConfigHandler scHandler = new SensorEnrichmentUpdateConfig.SourceConfigHandler() {
+      @Override
+      public SensorEnrichmentConfig readConfig(String sensor) throws Exception {
+        if(sensor.equals("bro")) {
+          return JSONUtils.INSTANCE.load(sourceConfigStr, SensorEnrichmentConfig.class);
+        }
+        else {
+          throw new IllegalStateException("Tried to retrieve an unexpected sensor: " + sensor);
+        }
+      }
+
+      @Override
+      public void persistConfig(String sensor, SensorEnrichmentConfig config) throws Exception {
+        outputScs.put(sensor, config);
+      }
+    };
+    SensorEnrichmentUpdateConfig.updateSensorConfigs(scHandler, config.getSensorToFieldList());
+    Assert.assertNotNull(outputScs.get("bro"));
+    Assert.assertNotSame(outputScs.get("bro"), broSc);
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldMap().get(Constants.SIMPLE_HBASE_ENRICHMENT).size()
+                       , 2
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldMap()
+                                  .get(Constants.SIMPLE_HBASE_ENRICHMENT)
+                                  .contains("ip_src_addr")
+                       );
+    Assert.assertTrue( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldMap()
+                                  .get(Constants.SIMPLE_HBASE_ENRICHMENT)
+                                  .contains("ip_dst_addr")
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldToTypeMap().keySet().size()
+                       , 2
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldToTypeMap().get("ip_src_addr").size()
+                       , 1
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldToTypeMap().get("ip_src_addr").get(0)
+                       , "playful"
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldToTypeMap().get("ip_dst_addr").size()
+                       , 1
+                       );
+    Assert.assertEquals( outputScs.get("bro").toJSON()
+                       , outputScs.get("bro").getEnrichment().getFieldToTypeMap().get("ip_dst_addr").get(0)
+                       , "playful"
+                       );
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/java/org/apache/metron/common/query/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/query/QueryParserTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/query/QueryParserTest.java
new file mode 100644
index 0000000..ad798e2
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/query/QueryParserTest.java
@@ -0,0 +1,152 @@
+/**
+ * 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.query;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class QueryParserTest {
+
+  @Test
+  public void testValidation() throws Exception {
+    PredicateProcessor processor = new PredicateProcessor();
+    try {
+      processor.validate("'foo'");
+      Assert.fail("Invalid rule found to be valid.");
+    }
+    catch(ParseException e) {
+
+    }
+    try {
+      processor.validate("enrichedField1 == 'enrichedValue1");
+      Assert.fail("Invalid rule found to be valid.");
+    }
+    catch(ParseException e) {
+
+    }
+  }
+
+  private static boolean run(String rule, VariableResolver resolver) {
+    PredicateProcessor processor = new PredicateProcessor();
+    Assert.assertTrue(rule + " not valid.", processor.validate(rule));
+    return processor.parse(rule, resolver);
+  }
+
+  @Test
+  public void testSimpleOps() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("empty", "");
+      put("spaced", "metron is great");
+      put("foo.bar", "casey");
+    }};
+    Assert.assertTrue(run("'casey' == foo.bar", v -> variableMap.get(v)));
+    Assert.assertTrue(run("'casey' == foo", v -> variableMap.get(v)));
+    Assert.assertFalse(run("'casey' != foo", v -> variableMap.get(v)));
+    Assert.assertTrue(run("'stella' == 'stella'", v -> variableMap.get(v)));
+    Assert.assertFalse(run("'stella' == foo", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo== foo", v -> variableMap.get(v)));
+    Assert.assertTrue(run("empty== ''", v -> variableMap.get(v)));
+    Assert.assertTrue(run("spaced == 'metron is great'", v -> variableMap.get(v)));
+  }
+
+  @Test
+  public void testBooleanOps() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("empty", "");
+      put("spaced", "metron is great");
+    }};
+    Assert.assertFalse(run("not('casey' == foo and true)", v -> variableMap.get(v)));
+    Assert.assertTrue(run("not(not('casey' == foo and true))", v -> variableMap.get(v)));
+    Assert.assertTrue(run("('casey' == foo) && ( false != true )", v -> variableMap.get(v)));
+    Assert.assertFalse(run("('casey' == foo) and (FALSE == TRUE)", v -> variableMap.get(v)));
+    Assert.assertFalse(run("'casey' == foo and FALSE", v -> variableMap.get(v)));
+    Assert.assertTrue(run("'casey' == foo and true", v -> variableMap.get(v)));
+  }
+  @Test
+  public void testList() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("empty", "");
+      put("spaced", "metron is great");
+    }};
+    Assert.assertTrue(run("foo in [ 'casey', 'david' ]", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo in [ foo, 'david' ]", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo in [ 'casey', 'david' ] and 'casey' == foo", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo in [ 'casey', 'david' ] and foo == 'casey'", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo in [ 'casey' ]", v -> variableMap.get(v)));
+    Assert.assertFalse(run("foo not in [ 'casey', 'david' ]", v -> variableMap.get(v)));
+    Assert.assertFalse(run("foo not in [ 'casey', 'david' ] and 'casey' == foo", v -> variableMap.get(v)));
+  }
+  @Test
+  public void testExists() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("empty", "");
+      put("spaced", "metron is great");
+    }};
+    Assert.assertTrue(run("exists(foo)", v -> variableMap.get(v)));
+    Assert.assertFalse(run("exists(bar)", v -> variableMap.get(v)));
+    Assert.assertTrue(run("exists(bar) or true", v -> variableMap.get(v)));
+  }
+
+  @Test
+  public void testStringFunctions() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("ip", "192.168.0.1");
+      put("empty", "");
+      put("spaced", "metron is great");
+    }};
+    Assert.assertTrue(run("true and TO_UPPER(foo) == 'CASEY'", v -> variableMap.get(v)));
+    Assert.assertTrue(run("foo in [ TO_LOWER('CASEY'), 'david' ]", v -> variableMap.get(v)));
+    Assert.assertTrue(run("TO_UPPER(foo) in [ TO_UPPER('casey'), 'david' ] and IN_SUBNET(ip, '192.168.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertFalse(run("TO_LOWER(foo) in [ TO_UPPER('casey'), 'david' ]", v -> variableMap.get(v)));
+  }
+  @Test
+  public void testLogicalFunctions() throws Exception {
+    final Map<String, String> variableMap = new HashMap<String, String>() {{
+      put("foo", "casey");
+      put("ip", "192.168.0.1");
+      put("ip_src_addr", "192.168.0.1");
+      put("ip_dst_addr", "10.0.0.1");
+      put("other_ip", "10.168.0.1");
+      put("empty", "");
+      put("spaced", "metron is great");
+    }};
+    Assert.assertTrue(run("IN_SUBNET(ip, '192.168.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertTrue(run("IN_SUBNET(ip, '192.168.0.0/24', '11.0.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertFalse(run("IN_SUBNET(ip_dst_addr, '192.168.0.0/24', '11.0.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertFalse(run("IN_SUBNET(other_ip, '192.168.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertFalse(run("IN_SUBNET(blah, '192.168.0.0/24')", v -> variableMap.get(v)));
+    Assert.assertTrue(run("true and STARTS_WITH(foo, 'ca')", v -> variableMap.get(v)));
+    Assert.assertTrue(run("true and STARTS_WITH(TO_UPPER(foo), 'CA')", v -> variableMap.get(v)));
+    Assert.assertTrue(run("(true and STARTS_WITH(TO_UPPER(foo), 'CA')) || true", v -> variableMap.get(v)));
+    Assert.assertTrue(run("true and ENDS_WITH(foo, 'sey')", v -> variableMap.get(v)));
+    Assert.assertTrue(run("not(IN_SUBNET(ip_src_addr, '192.168.0.0/24') and IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))", v-> variableMap.get(v)));
+    Assert.assertTrue(run("IN_SUBNET(ip_src_addr, '192.168.0.0/24')", v-> variableMap.get(v)));
+    Assert.assertFalse(run("not(IN_SUBNET(ip_src_addr, '192.168.0.0/24'))", v-> variableMap.get(v)));
+    Assert.assertFalse(run("IN_SUBNET(ip_dst_addr, '192.168.0.0/24')", v-> variableMap.get(v)));
+    Assert.assertTrue(run("not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))", v-> variableMap.get(v)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-common/src/test/resources/config/sensors/bro.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/resources/config/sensors/bro.json b/metron-platform/metron-common/src/test/resources/config/sensors/bro.json
index 8886495..c8f2a54 100644
--- a/metron-platform/metron-common/src/test/resources/config/sensors/bro.json
+++ b/metron-platform/metron-common/src/test/resources/config/sensors/bro.json
@@ -1,19 +1,27 @@
 {
   "index": "bro",
   "batchSize": 5,
-  "enrichmentFieldMap":
-  {
-    "geo": ["ip_dst_addr", "ip_src_addr"],
-    "host": ["host"]
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_dst_addr",
+        "ip_src_addr"
+      ],
+      "host": [
+        "host"
+      ]
+    }
   },
-  "threatIntelFieldMap":
-  {
-    "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-  },
-  "fieldToThreatIntelTypeMap":
-  {
-    "ip_dst_addr" : [ "malicious_ip" ]
+  "threatIntel":{
+    "fieldMap":
+    {
+      "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+    },
+    "fieldToTypeMap":
+    {
+      "ip_dst_addr" : [ "malicious_ip" ]
     ,"ip_src_addr" : [ "malicious_ip" ]
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/assembly/assembly.xml b/metron-platform/metron-data-management/src/main/assembly/assembly.xml
index c2c384b..0b36f9d 100644
--- a/metron-platform/metron-data-management/src/main/assembly/assembly.xml
+++ b/metron-platform/metron-data-management/src/main/assembly/assembly.xml
@@ -19,7 +19,7 @@
   <includeBaseDirectory>false</includeBaseDirectory>
   <fileSets>
     <fileSet>
-      <directory>${project.basedir}/src/main/bash</directory>
+      <directory>${project.basedir}/src/main/scripts</directory>
       <outputDirectory>/bin</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>
       <excludes>


[2/5] incubator-metron git commit: METRON-141: The ability to do threat triage closes apache/incubator-metron#108

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/Whois_CSV_to_JSON.py
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/Whois_CSV_to_JSON.py b/metron-platform/metron-data-management/src/main/bash/Whois_CSV_to_JSON.py
deleted file mode 100755
index 2091418..0000000
--- a/metron-platform/metron-data-management/src/main/bash/Whois_CSV_to_JSON.py
+++ /dev/null
@@ -1,208 +0,0 @@
-#!/usr/bin/python
-
-"""
-Copyright 2014 Cisco Systems, Inc.
-
-Licensed 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.
-"""
-
-import sys
-import os
-import csv
-import json
-import multiprocessing
-import logging
-logging.basicConfig(level=logging.DEBUG)
-
-
-def is_field_excluded(fieldname=None):
-    """
-    Checks to see if a field name is a member of a list of names to exclude. Modify to suit your own list.
-
-    :param fieldname: A string representing a field name
-    :return: True or False
-    """
-    import re
-
-    # List of fields names to exclude
-    excluded_fields = [
-        'Audit_auditUpdatedDate',
-        #'domainName'
-    ]
-
-    if fieldname in excluded_fields:
-        return True
-
-    # Regexes to match for exclusion
-    excluded_regexes = [
-        ['_rawText$', re.IGNORECASE],
-    ]
-
-    for regex in excluded_regexes:
-        if re.search(regex[0], fieldname, regex[1]):
-            return True
-
-    return False
-
-
-def process_csv(in_filename, out_filename):
-    """
-    Processes a CSV file of WHOIS data and converts each line to a JSON element, skipping specific fields that
-    are not deemed necessary (domainName, *_rawText, Audit_auditUpdatedDate)
-
-    :param in_filename: Input CSV filename with full path
-    :param out_filename: Output JSON filename with full path
-    :return: None
-    """
-    if out_filename:
-        out_fh = open(out_filename, 'wb')
-        logging.debug('%s: Converting %s to %s' % (multiprocessing.current_process().name, in_filename, out_filename))
-    else:
-        logging.debug('%s: Analyzing %s' % (multiprocessing.current_process().name, in_filename))
-
-    with open(in_filename, 'rb') as f:
-        reader = csv.DictReader(f, delimiter=',', quotechar='"')
-        line_num = 0
-        try:
-            for row in reader:
-                line_num += 1
-                try:
-                    if out_filename:
-                        # json conversion and output
-                        new_row = {}
-                        for field in reader.fieldnames:
-                            # fields we don't want include these + anything with rawText
-                            #if field not in ['Audit_auditUpdatedDate', 'domainName'] and not field.endswith('_rawText'):
-                            if not is_field_excluded(field):
-                                new_row[field] = row.get(field)
-                        json.dump(new_row, out_fh)
-                        out_fh.write('\n')
-                    else:
-                        # analysis .. check to be sure fileheader and csv row counts match
-                        if len(row) != len(reader.fieldnames):
-                            raise Exception('Field count mismatch: row: %s / fields: %s' % (len(row), len(reader.fieldnames)))
-                except Exception, e:
-                    logging.warn("Error with file %s, line %s: %s" % (in_filename, line_num, e))
-
-            if not out_filename:
-                logging.info('Analyzed %s: OK' % in_filename)
-        except Exception, e:
-            logging.warn(e)
-
-        out_fh.close()
-
-
-##-------------------------------------------------------------------------
-
-def process_files(source_dir, output_dir, max_processes=10, overwrite=False):
-    """
-    Generates a multiprocessing.Pool() queue with a list of input and output files to be processed with processCSV.
-    Files are added by walking the source_dir and adding any file with a CSV extension. Output is placed into a single
-    directory for processing. Output filenames are generated using the first part of the directory name so a file
-    named source_dir/com/1.csv would become outputDir/com_1.json
-
-    :param source_dir: Source directory of CSV files
-    :param output_dir: Output directory for resultant JSON files
-    :param max_processes: Maximum number of processes run
-    :return:
-    """
-    logging.info("Processing Whois files from %s" % source_dir)
-
-    if output_dir and not os.path.exists(output_dir):
-        logging.debug("Creating output directory %s" % output_dir)
-        os.makedirs(output_dir)
-
-    logging.info("Starting %s pool workers" % max_processes)
-
-    if sys.version.startswith('2.6'):
-        # no maxtaskperchild in 2.6
-        pool = multiprocessing.Pool(processes=max_processes)
-    else:
-        pool = multiprocessing.Pool(processes=max_processes, maxtasksperchild=4)
-
-    filecount = 0
-    for dirname, dirnames, filenames in os.walk(source_dir):
-        for filename in filenames:
-            if filename.endswith('.csv'):
-                # output files go to outputDir and are named using the last subdirectory from the dirname
-                if output_dir:
-                    out_filename = filename.replace('csv', 'json')
-                    out_filename = os.path.join(output_dir, '%s_%s' % (os.path.split(dirname)[-1], out_filename))
-
-                    # if file does not exist or if overwrite is true, add file process to the pool
-                    if not os.path.isfile(out_filename) or overwrite:
-                        pool.apply_async(process_csv, args=(os.path.join(dirname, filename), out_filename))
-                        filecount += 1
-                    else:
-                        logging.info("Skipping %s, %s exists and overwrite is false" % (filename, out_filename))
-                else:
-                    # no outputdir so we just analyze the files
-                    pool.apply_async(process_csv, args=(os.path.join(dirname, filename), None))
-                    filecount += 1
-
-    try:
-        pool.close()
-        logging.info("Starting activities on %s CSV files" % filecount)
-        pool.join()
-    except KeyboardInterrupt:
-        logging.info("Aborting")
-        pool.terminate()
-
-    logging.info("Completed")
-
-
-##-------------------------------------------------------------------------
-
-if __name__ == "__main__":
-
-    max_cpu = multiprocessing.cpu_count()
-
-    from optparse import OptionParser
-    parser = OptionParser()
-    parser.add_option('-s', '--source', dest='source_dir', action='store',
-                      help='Source directory to walk for CSV files')
-    parser.add_option('-o', '--output', dest='out_dir', action='store',
-                      help='Output directory for JSON files')
-    parser.add_option('-O', '--overwrite', dest='overwrite', action='store_true',
-                      help='Overwrite existing files in output directory')
-    parser.add_option('-p', '--processes', dest='max_processes', action='store', default=max_cpu, type='int',
-                      help='Max number of processes to spawn')
-    parser.add_option('-a', '--analyze', dest='analyze', action='store_true',
-                      help='Analyze CSV files for validity, no file output')
-    parser.add_option('-d', '--debug', dest='debug', action='store_true',
-                      help='Enable debug messages')
-
-    (options, args) = parser.parse_args()
-
-    if not options.source_dir:
-        logging.error("Source directory required")
-        sys.exit(-1)
-
-    if not options.out_dir or options.analyze:
-        out_dir = None
-    elif not options.out_dir:
-        logging.error("Ouput directory or analysis option required")
-        sys.exit(-1)
-    else:
-        out_dir = options.out_dir
-
-    if options.max_processes > max_cpu:
-        logging.warn('Max Processes (%s) is greater than available Processors (%s)' % (options.max_processes, max_cpu))
-
-    if options.debug:
-        # enable debug level and multiprocessing debugging
-        logging.basicConfig(level=logging.DEBUG)
-        multiprocessing.log_to_stderr(logging.DEBUG)
-
-    process_files(options.source_dir, options.out_dir, options.max_processes, options.overwrite)
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/flatfile_loader.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/flatfile_loader.sh b/metron-platform/metron-data-management/src/main/bash/flatfile_loader.sh
deleted file mode 100755
index 03fd641..0000000
--- a/metron-platform/metron-data-management/src/main/bash/flatfile_loader.sh
+++ /dev/null
@@ -1,42 +0,0 @@
-#!/bin/bash
-# 
-# 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.
-# 
-
-BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
-[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
-
-# Autodetect JAVA_HOME if not defined
-if [ -e /usr/libexec/bigtop-detect-javahome ]; then
-  . /usr/libexec/bigtop-detect-javahome
-elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
-  . /usr/lib/bigtop-utils/bigtop-detect-javahome
-fi
-
-export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
-CP=/usr/metron/0.1BETA/lib/metron-data-management-0.1BETA.jar:/usr/metron/0.1BETA/lib/taxii-1.1.0.1.jar:`${HBASE_HOME}/bin/hbase classpath`
-HADOOP_CLASSPATH=$(echo $CP )
-for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
-  if [ -f $jar ];then
-    LIBJARS="$jar,$LIBJARS"
-  fi
-done
-export HADOOP_CLASSPATH
-export METRON_VERSION=${project.version}
-export METRON_HOME=/usr/metron/$METRON_VERSION
-export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
-hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.nonbulk.flatfile.SimpleEnrichmentFlatFileLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/prune_elasticsearch_indices.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/prune_elasticsearch_indices.sh b/metron-platform/metron-data-management/src/main/bash/prune_elasticsearch_indices.sh
deleted file mode 100644
index c3f1d05..0000000
--- a/metron-platform/metron-data-management/src/main/bash/prune_elasticsearch_indices.sh
+++ /dev/null
@@ -1,21 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-yarn jar /usr/metron/${project.version}/lib/${project.artifactId}-${project.version}.jar org.apache.metron.dataloads.bulk.ElasticsearchDataPrunerRunner "$@"
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/prune_hdfs_files.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/prune_hdfs_files.sh b/metron-platform/metron-data-management/src/main/bash/prune_hdfs_files.sh
deleted file mode 100644
index b8d9372..0000000
--- a/metron-platform/metron-data-management/src/main/bash/prune_hdfs_files.sh
+++ /dev/null
@@ -1,21 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-yarn jar /usr/metron/${project.version}/lib/${project.artifactId}-${project.version}.jar org.apache.metron.dataloads.bulk.HDFSDataPruner "$@"
-

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_load.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_load.sh b/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_load.sh
deleted file mode 100755
index 865d0ad..0000000
--- a/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_load.sh
+++ /dev/null
@@ -1,41 +0,0 @@
-#!/bin/bash
-# 
-# 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.
-# 
-
-BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
-[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
-
-# Autodetect JAVA_HOME if not defined
-if [ -e /usr/libexec/bigtop-detect-javahome ]; then
-  . /usr/libexec/bigtop-detect-javahome
-elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
-  . /usr/lib/bigtop-utils/bigtop-detect-javahome
-fi
-
-export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
-HADOOP_CLASSPATH=${HBASE_HOME}/lib/hbase-server.jar:`${HBASE_HOME}/bin/hbase classpath`
-for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
-  if [ -f $jar ];then
-    LIBJARS="$jar,$LIBJARS"
-  fi
-done
-export HADOOP_CLASSPATH
-export METRON_VERSION=${project.version}
-export METRON_HOME=/usr/metron/$METRON_VERSION
-export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
-hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.bulk.ThreatIntelBulkLoader -libjars ${LIBJARS} "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_prune.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_prune.sh b/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_prune.sh
deleted file mode 100755
index 6156027..0000000
--- a/metron-platform/metron-data-management/src/main/bash/threatintel_bulk_prune.sh
+++ /dev/null
@@ -1,40 +0,0 @@
-#!/bin/bash
-#
-# 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.
-# 
-BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
-[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
-
-# Autodetect JAVA_HOME if not defined
-if [ -e /usr/libexec/bigtop-detect-javahome ]; then
-  . /usr/libexec/bigtop-detect-javahome
-elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
-  . /usr/lib/bigtop-utils/bigtop-detect-javahome
-fi
-
-export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
-HADOOP_CLASSPATH=${HBASE_HOME}/lib/hbase-server.jar:`${HBASE_HOME}/bin/hbase classpath`
-for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
-  if [ -f $jar ];then
-    LIBJARS="$jar,$LIBJARS"
-  fi
-done
-export HADOOP_CLASSPATH
-export METRON_VERSION=${project.version}
-export METRON_HOME=/usr/metron/$METRON_VERSION
-export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
-hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.bulk.LeastRecentlyUsedPruner -libjars ${LIBJARS} "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/bash/threatintel_taxii_load.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/bash/threatintel_taxii_load.sh b/metron-platform/metron-data-management/src/main/bash/threatintel_taxii_load.sh
deleted file mode 100755
index 23d09ba..0000000
--- a/metron-platform/metron-data-management/src/main/bash/threatintel_taxii_load.sh
+++ /dev/null
@@ -1,42 +0,0 @@
-#!/bin/bash
-# 
-# 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.
-# 
-
-BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
-[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
-
-# Autodetect JAVA_HOME if not defined
-if [ -e /usr/libexec/bigtop-detect-javahome ]; then
-  . /usr/libexec/bigtop-detect-javahome
-elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
-  . /usr/lib/bigtop-utils/bigtop-detect-javahome
-fi
-
-export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
-CP=/usr/metron/0.1BETA/lib/metron-data-management-0.1BETA.jar:/usr/metron/0.1BETA/lib/taxii-1.1.0.1.jar:`${HBASE_HOME}/bin/hbase classpath`
-HADOOP_CLASSPATH=$(echo $CP )
-for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
-  if [ -f $jar ];then
-    LIBJARS="$jar,$LIBJARS"
-  fi
-done
-export HADOOP_CLASSPATH
-export METRON_VERSION=${project.version}
-export METRON_HOME=/usr/metron/$METRON_VERSION
-export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
-hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.nonbulk.taxii.TaxiiLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
index 2e63ef2..4e2da61 100644
--- a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
+++ b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/bulk/ThreatIntelBulkLoader.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
 import org.apache.metron.dataloads.hbase.mr.BulkLoadMapper;
-import org.apache.metron.common.configuration.EnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentUpdateConfig;
 import org.apache.metron.enrichment.converter.HbaseConverter;
 import org.apache.metron.enrichment.converter.EnrichmentConverter;
 import org.apache.metron.common.utils.JSONUtils;
@@ -238,10 +238,10 @@ public class ThreatIntelBulkLoader  {
     if(BulkLoadOptions.CONVERTER.has(cli)) {
       converterClass = BulkLoadOptions.CONVERTER.get(cli);
     }
-    EnrichmentConfig enrichmentConfig = null;
+    SensorEnrichmentUpdateConfig sensorEnrichmentUpdateConfig = null;
     if(BulkLoadOptions.ENRICHMENT_CONFIG.has(cli)) {
-      enrichmentConfig = JSONUtils.INSTANCE.load( new File(BulkLoadOptions.ENRICHMENT_CONFIG.get(cli))
-              , EnrichmentConfig.class
+      sensorEnrichmentUpdateConfig = JSONUtils.INSTANCE.load( new File(BulkLoadOptions.ENRICHMENT_CONFIG.get(cli))
+              , SensorEnrichmentUpdateConfig.class
       );
     }
 
@@ -252,8 +252,8 @@ public class ThreatIntelBulkLoader  {
     if(!jobRet) {
       System.exit(1);
     }
-    if(enrichmentConfig != null) {
-        enrichmentConfig.updateSensorConfigs();
+    if(sensorEnrichmentUpdateConfig != null) {
+        sensorEnrichmentUpdateConfig.updateSensorConfigs();
     }
     System.exit(0);
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
index cbd3beb..0c7501a 100644
--- a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
+++ b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/flatfile/SimpleEnrichmentFlatFileLoader.java
@@ -31,7 +31,7 @@ import org.apache.log4j.PropertyConfigurator;
 import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
 import org.apache.metron.dataloads.extractor.inputformat.WholeFileFormat;
-import org.apache.metron.common.configuration.EnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentUpdateConfig;
 import org.apache.metron.hbase.HTableProvider;
 import org.apache.metron.enrichment.converter.HbaseConverter;
 import org.apache.metron.enrichment.converter.EnrichmentConverter;
@@ -239,10 +239,10 @@ public class SimpleEnrichmentFlatFileLoader {
     );
     boolean lineByLine = !handler.getInputFormatHandler().getClass().equals(WholeFileFormat.class);
     Extractor e = handler.getExtractor();
-    EnrichmentConfig enrichmentConfig = null;
+    SensorEnrichmentUpdateConfig sensorEnrichmentUpdateConfig = null;
     if(LoadOptions.ENRICHMENT_CONFIG.has(cli)) {
-      enrichmentConfig = JSONUtils.INSTANCE.load( new File(LoadOptions.ENRICHMENT_CONFIG.get(cli))
-              , EnrichmentConfig.class
+      sensorEnrichmentUpdateConfig = JSONUtils.INSTANCE.load( new File(LoadOptions.ENRICHMENT_CONFIG.get(cli))
+              , SensorEnrichmentUpdateConfig.class
       );
     }
     HbaseConverter converter = new EnrichmentConverter();
@@ -254,8 +254,8 @@ public class SimpleEnrichmentFlatFileLoader {
     for (File f : inputFiles) {
       loader.loadFile(f, e, table, LoadOptions.HBASE_CF.get(cli), converter, lineByLine);
     }
-    if(enrichmentConfig != null) {
-      enrichmentConfig.updateSensorConfigs();
+    if(sensorEnrichmentUpdateConfig != null) {
+      sensorEnrichmentUpdateConfig.updateSensorConfigs();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
index df803c1..689a08f 100644
--- a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
+++ b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiLoader.java
@@ -31,7 +31,7 @@ import org.apache.log4j.PropertyConfigurator;
 import org.apache.metron.dataloads.extractor.Extractor;
 import org.apache.metron.dataloads.extractor.ExtractorHandler;
 import org.apache.metron.dataloads.extractor.stix.StixExtractor;
-import org.apache.metron.common.configuration.EnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentUpdateConfig;
 import org.apache.metron.common.utils.JSONUtils;
 
 import javax.annotation.Nullable;
@@ -176,12 +176,12 @@ public class TaxiiLoader {
     }
     ExtractorHandler handler = ExtractorHandler.load(FileUtils.readFileToString(new File(TaxiiOptions.EXTRACTOR_CONFIG.get(cli))));
     Extractor e = handler.getExtractor();
-    EnrichmentConfig enrichmentConfig = null;
+    SensorEnrichmentUpdateConfig sensorEnrichmentUpdateConfig = null;
     if(TaxiiOptions.ENRICHMENT_CONFIG.has(cli)) {
-      enrichmentConfig = JSONUtils.INSTANCE.load( new File(TaxiiOptions.ENRICHMENT_CONFIG.get(cli))
-              , EnrichmentConfig.class
+      sensorEnrichmentUpdateConfig = JSONUtils.INSTANCE.load( new File(TaxiiOptions.ENRICHMENT_CONFIG.get(cli))
+              , SensorEnrichmentUpdateConfig.class
       );
-      enrichmentConfig.updateSensorConfigs();
+      sensorEnrichmentUpdateConfig.updateSensorConfigs();
     }
 
     Timer timer = new Timer();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/Whois_CSV_to_JSON.py
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/Whois_CSV_to_JSON.py b/metron-platform/metron-data-management/src/main/scripts/Whois_CSV_to_JSON.py
new file mode 100755
index 0000000..2091418
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/Whois_CSV_to_JSON.py
@@ -0,0 +1,208 @@
+#!/usr/bin/python
+
+"""
+Copyright 2014 Cisco Systems, Inc.
+
+Licensed 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.
+"""
+
+import sys
+import os
+import csv
+import json
+import multiprocessing
+import logging
+logging.basicConfig(level=logging.DEBUG)
+
+
+def is_field_excluded(fieldname=None):
+    """
+    Checks to see if a field name is a member of a list of names to exclude. Modify to suit your own list.
+
+    :param fieldname: A string representing a field name
+    :return: True or False
+    """
+    import re
+
+    # List of fields names to exclude
+    excluded_fields = [
+        'Audit_auditUpdatedDate',
+        #'domainName'
+    ]
+
+    if fieldname in excluded_fields:
+        return True
+
+    # Regexes to match for exclusion
+    excluded_regexes = [
+        ['_rawText$', re.IGNORECASE],
+    ]
+
+    for regex in excluded_regexes:
+        if re.search(regex[0], fieldname, regex[1]):
+            return True
+
+    return False
+
+
+def process_csv(in_filename, out_filename):
+    """
+    Processes a CSV file of WHOIS data and converts each line to a JSON element, skipping specific fields that
+    are not deemed necessary (domainName, *_rawText, Audit_auditUpdatedDate)
+
+    :param in_filename: Input CSV filename with full path
+    :param out_filename: Output JSON filename with full path
+    :return: None
+    """
+    if out_filename:
+        out_fh = open(out_filename, 'wb')
+        logging.debug('%s: Converting %s to %s' % (multiprocessing.current_process().name, in_filename, out_filename))
+    else:
+        logging.debug('%s: Analyzing %s' % (multiprocessing.current_process().name, in_filename))
+
+    with open(in_filename, 'rb') as f:
+        reader = csv.DictReader(f, delimiter=',', quotechar='"')
+        line_num = 0
+        try:
+            for row in reader:
+                line_num += 1
+                try:
+                    if out_filename:
+                        # json conversion and output
+                        new_row = {}
+                        for field in reader.fieldnames:
+                            # fields we don't want include these + anything with rawText
+                            #if field not in ['Audit_auditUpdatedDate', 'domainName'] and not field.endswith('_rawText'):
+                            if not is_field_excluded(field):
+                                new_row[field] = row.get(field)
+                        json.dump(new_row, out_fh)
+                        out_fh.write('\n')
+                    else:
+                        # analysis .. check to be sure fileheader and csv row counts match
+                        if len(row) != len(reader.fieldnames):
+                            raise Exception('Field count mismatch: row: %s / fields: %s' % (len(row), len(reader.fieldnames)))
+                except Exception, e:
+                    logging.warn("Error with file %s, line %s: %s" % (in_filename, line_num, e))
+
+            if not out_filename:
+                logging.info('Analyzed %s: OK' % in_filename)
+        except Exception, e:
+            logging.warn(e)
+
+        out_fh.close()
+
+
+##-------------------------------------------------------------------------
+
+def process_files(source_dir, output_dir, max_processes=10, overwrite=False):
+    """
+    Generates a multiprocessing.Pool() queue with a list of input and output files to be processed with processCSV.
+    Files are added by walking the source_dir and adding any file with a CSV extension. Output is placed into a single
+    directory for processing. Output filenames are generated using the first part of the directory name so a file
+    named source_dir/com/1.csv would become outputDir/com_1.json
+
+    :param source_dir: Source directory of CSV files
+    :param output_dir: Output directory for resultant JSON files
+    :param max_processes: Maximum number of processes run
+    :return:
+    """
+    logging.info("Processing Whois files from %s" % source_dir)
+
+    if output_dir and not os.path.exists(output_dir):
+        logging.debug("Creating output directory %s" % output_dir)
+        os.makedirs(output_dir)
+
+    logging.info("Starting %s pool workers" % max_processes)
+
+    if sys.version.startswith('2.6'):
+        # no maxtaskperchild in 2.6
+        pool = multiprocessing.Pool(processes=max_processes)
+    else:
+        pool = multiprocessing.Pool(processes=max_processes, maxtasksperchild=4)
+
+    filecount = 0
+    for dirname, dirnames, filenames in os.walk(source_dir):
+        for filename in filenames:
+            if filename.endswith('.csv'):
+                # output files go to outputDir and are named using the last subdirectory from the dirname
+                if output_dir:
+                    out_filename = filename.replace('csv', 'json')
+                    out_filename = os.path.join(output_dir, '%s_%s' % (os.path.split(dirname)[-1], out_filename))
+
+                    # if file does not exist or if overwrite is true, add file process to the pool
+                    if not os.path.isfile(out_filename) or overwrite:
+                        pool.apply_async(process_csv, args=(os.path.join(dirname, filename), out_filename))
+                        filecount += 1
+                    else:
+                        logging.info("Skipping %s, %s exists and overwrite is false" % (filename, out_filename))
+                else:
+                    # no outputdir so we just analyze the files
+                    pool.apply_async(process_csv, args=(os.path.join(dirname, filename), None))
+                    filecount += 1
+
+    try:
+        pool.close()
+        logging.info("Starting activities on %s CSV files" % filecount)
+        pool.join()
+    except KeyboardInterrupt:
+        logging.info("Aborting")
+        pool.terminate()
+
+    logging.info("Completed")
+
+
+##-------------------------------------------------------------------------
+
+if __name__ == "__main__":
+
+    max_cpu = multiprocessing.cpu_count()
+
+    from optparse import OptionParser
+    parser = OptionParser()
+    parser.add_option('-s', '--source', dest='source_dir', action='store',
+                      help='Source directory to walk for CSV files')
+    parser.add_option('-o', '--output', dest='out_dir', action='store',
+                      help='Output directory for JSON files')
+    parser.add_option('-O', '--overwrite', dest='overwrite', action='store_true',
+                      help='Overwrite existing files in output directory')
+    parser.add_option('-p', '--processes', dest='max_processes', action='store', default=max_cpu, type='int',
+                      help='Max number of processes to spawn')
+    parser.add_option('-a', '--analyze', dest='analyze', action='store_true',
+                      help='Analyze CSV files for validity, no file output')
+    parser.add_option('-d', '--debug', dest='debug', action='store_true',
+                      help='Enable debug messages')
+
+    (options, args) = parser.parse_args()
+
+    if not options.source_dir:
+        logging.error("Source directory required")
+        sys.exit(-1)
+
+    if not options.out_dir or options.analyze:
+        out_dir = None
+    elif not options.out_dir:
+        logging.error("Ouput directory or analysis option required")
+        sys.exit(-1)
+    else:
+        out_dir = options.out_dir
+
+    if options.max_processes > max_cpu:
+        logging.warn('Max Processes (%s) is greater than available Processors (%s)' % (options.max_processes, max_cpu))
+
+    if options.debug:
+        # enable debug level and multiprocessing debugging
+        logging.basicConfig(level=logging.DEBUG)
+        multiprocessing.log_to_stderr(logging.DEBUG)
+
+    process_files(options.source_dir, options.out_dir, options.max_processes, options.overwrite)
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/flatfile_loader.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/flatfile_loader.sh b/metron-platform/metron-data-management/src/main/scripts/flatfile_loader.sh
new file mode 100755
index 0000000..03fd641
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/flatfile_loader.sh
@@ -0,0 +1,42 @@
+#!/bin/bash
+# 
+# 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.
+# 
+
+BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
+CP=/usr/metron/0.1BETA/lib/metron-data-management-0.1BETA.jar:/usr/metron/0.1BETA/lib/taxii-1.1.0.1.jar:`${HBASE_HOME}/bin/hbase classpath`
+HADOOP_CLASSPATH=$(echo $CP )
+for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
+  if [ -f $jar ];then
+    LIBJARS="$jar,$LIBJARS"
+  fi
+done
+export HADOOP_CLASSPATH
+export METRON_VERSION=${project.version}
+export METRON_HOME=/usr/metron/$METRON_VERSION
+export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
+hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.nonbulk.flatfile.SimpleEnrichmentFlatFileLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/prune_elasticsearch_indices.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/prune_elasticsearch_indices.sh b/metron-platform/metron-data-management/src/main/scripts/prune_elasticsearch_indices.sh
new file mode 100644
index 0000000..c3f1d05
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/prune_elasticsearch_indices.sh
@@ -0,0 +1,21 @@
+#!/bin/bash
+#
+# 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.
+#
+
+yarn jar /usr/metron/${project.version}/lib/${project.artifactId}-${project.version}.jar org.apache.metron.dataloads.bulk.ElasticsearchDataPrunerRunner "$@"
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/prune_hdfs_files.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/prune_hdfs_files.sh b/metron-platform/metron-data-management/src/main/scripts/prune_hdfs_files.sh
new file mode 100644
index 0000000..b8d9372
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/prune_hdfs_files.sh
@@ -0,0 +1,21 @@
+#!/bin/bash
+#
+# 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.
+#
+
+yarn jar /usr/metron/${project.version}/lib/${project.artifactId}-${project.version}.jar org.apache.metron.dataloads.bulk.HDFSDataPruner "$@"
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_load.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_load.sh b/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_load.sh
new file mode 100755
index 0000000..865d0ad
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_load.sh
@@ -0,0 +1,41 @@
+#!/bin/bash
+# 
+# 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.
+# 
+
+BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
+HADOOP_CLASSPATH=${HBASE_HOME}/lib/hbase-server.jar:`${HBASE_HOME}/bin/hbase classpath`
+for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
+  if [ -f $jar ];then
+    LIBJARS="$jar,$LIBJARS"
+  fi
+done
+export HADOOP_CLASSPATH
+export METRON_VERSION=${project.version}
+export METRON_HOME=/usr/metron/$METRON_VERSION
+export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
+hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.bulk.ThreatIntelBulkLoader -libjars ${LIBJARS} "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_prune.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_prune.sh b/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_prune.sh
new file mode 100755
index 0000000..6156027
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/threatintel_bulk_prune.sh
@@ -0,0 +1,40 @@
+#!/bin/bash
+#
+# 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.
+# 
+BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
+HADOOP_CLASSPATH=${HBASE_HOME}/lib/hbase-server.jar:`${HBASE_HOME}/bin/hbase classpath`
+for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
+  if [ -f $jar ];then
+    LIBJARS="$jar,$LIBJARS"
+  fi
+done
+export HADOOP_CLASSPATH
+export METRON_VERSION=${project.version}
+export METRON_HOME=/usr/metron/$METRON_VERSION
+export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
+hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.bulk.LeastRecentlyUsedPruner -libjars ${LIBJARS} "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-data-management/src/main/scripts/threatintel_taxii_load.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/scripts/threatintel_taxii_load.sh b/metron-platform/metron-data-management/src/main/scripts/threatintel_taxii_load.sh
new file mode 100755
index 0000000..23d09ba
--- /dev/null
+++ b/metron-platform/metron-data-management/src/main/scripts/threatintel_taxii_load.sh
@@ -0,0 +1,42 @@
+#!/bin/bash
+# 
+# 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.
+# 
+
+BIGTOP_DEFAULTS_DIR=${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "${BIGTOP_DEFAULTS_DIR}" -a -r ${BIGTOP_DEFAULTS_DIR}/hbase ] && . ${BIGTOP_DEFAULTS_DIR}/hbase
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
+CP=/usr/metron/0.1BETA/lib/metron-data-management-0.1BETA.jar:/usr/metron/0.1BETA/lib/taxii-1.1.0.1.jar:`${HBASE_HOME}/bin/hbase classpath`
+HADOOP_CLASSPATH=$(echo $CP )
+for jar in $(echo $HADOOP_CLASSPATH | sed 's/:/ /g');do
+  if [ -f $jar ];then
+    LIBJARS="$jar,$LIBJARS"
+  fi
+done
+export HADOOP_CLASSPATH
+export METRON_VERSION=${project.version}
+export METRON_HOME=/usr/metron/$METRON_VERSION
+export DM_JAR=${project.artifactId}-$METRON_VERSION.jar
+hadoop jar $METRON_HOME/lib/$DM_JAR org.apache.metron.dataloads.nonbulk.taxii.TaxiiLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
index 6b54fec..989abfb 100644
--- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java
@@ -19,7 +19,7 @@ package org.apache.metron.elasticsearch.writer;
 
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.configuration.Configurations;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.interfaces.BulkMessageWriter;
 import org.elasticsearch.action.bulk.BulkRequestBuilder;
 import org.elasticsearch.action.bulk.BulkResponse;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/pom.xml b/metron-platform/metron-enrichment/pom.xml
index 756a0b4..87e5096 100644
--- a/metron-platform/metron-enrichment/pom.xml
+++ b/metron-platform/metron-enrichment/pom.xml
@@ -159,6 +159,7 @@
                     </systemProperties>
                 </configuration>
             </plugin>
+
             <!-- Normally, dependency report takes time, skip it -->
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
index f2e0113..22629a4 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapter.java
@@ -64,7 +64,7 @@ public class SimpleHBaseAdapter implements EnrichmentAdapter<CacheKey>,Serializa
   public JSONObject enrich(CacheKey value) {
     JSONObject enriched = new JSONObject();
     List<String> enrichmentTypes = value.getConfig()
-                                        .getFieldToEnrichmentTypeMap()
+                                        .getEnrichment().getFieldToTypeMap()
                                         .get(EnrichmentUtils.toTopLevelField(value.getField()));
     if(enrichmentTypes != null && value.getValue() != null) {
       try {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
index dbdf6ec..ee5636b 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapter.java
@@ -54,7 +54,7 @@ public class ThreatIntelAdapter implements EnrichmentAdapter<CacheKey>,Serializa
 
   @Override
   public void logAccess(CacheKey value) {
-    List<String> enrichmentTypes = value.getConfig().getFieldToThreatIntelTypeMap().get(value.getField());
+    List<String> enrichmentTypes = value.getConfig().getThreatIntel().getFieldToTypeMap().get(value.getField());
     if(enrichmentTypes != null) {
       for(String enrichmentType : enrichmentTypes) {
         lookup.getAccessTracker().logAccess(new EnrichmentKey(enrichmentType, value.getValue()));
@@ -67,7 +67,7 @@ public class ThreatIntelAdapter implements EnrichmentAdapter<CacheKey>,Serializa
   public JSONObject enrich(CacheKey value) {
     JSONObject enriched = new JSONObject();
     List<String> enrichmentTypes = value.getConfig()
-                                        .getFieldToThreatIntelTypeMap()
+                                        .getThreatIntel().getFieldToTypeMap()
                                         .get(EnrichmentUtils.toTopLevelField(value.getField()));
     if(enrichmentTypes != null) {
       int i = 0;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
index f3e742d..7cfa34d 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBolt.java
@@ -24,7 +24,7 @@ import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.bolt.ConfiguredBolt;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.utils.ErrorUtils;
 import org.apache.metron.common.utils.MessageUtils;
 import org.apache.metron.common.interfaces.BulkMessageWriter;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
index 1338b44..4dbd33e 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/CacheKey.java
@@ -17,7 +17,7 @@
  */
 package org.apache.metron.enrichment.bolt;
 
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 
 public class CacheKey {
   private String field;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
index 8ef44d0..48e09f8 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentJoinBolt.java
@@ -18,7 +18,7 @@
 package org.apache.metron.enrichment.bolt;
 
 import backtype.storm.task.TopologyContext;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.utils.MessageUtils;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
@@ -48,6 +48,10 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
   public Set<String> getStreamIds(JSONObject message) {
     Set<String> streamIds = new HashSet<>();
     String sourceType = MessageUtils.getSensorType(message);
+    if(sourceType == null) {
+      String errorMessage = "Unable to find source type for message: " + message;
+      throw new IllegalStateException(errorMessage);
+    }
     Map<String, List<String>>  fieldMap = getFieldMap(sourceType);
     if(fieldMap != null) {
       for (String enrichmentType : getFieldMap(sourceType).keySet()) {
@@ -83,8 +87,8 @@ public class EnrichmentJoinBolt extends JoinBolt<JSONObject> {
   public Map<String, List<String>> getFieldMap(String sourceType) {
     if(sourceType != null) {
       SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
-      if (config != null) {
-        return config.getEnrichmentFieldMap();
+      if (config != null && config.getEnrichment() != null) {
+        return config.getEnrichment().getFieldMap();
       }
       else {
         LOG.error("Unable to retrieve a sensor enrichment config of " + sourceType);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
index 6b49edb..c367173 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/EnrichmentSplitterBolt.java
@@ -21,7 +21,7 @@ import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
 import org.apache.metron.common.Constants;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.configuration.Enrichment;
 import org.apache.metron.enrichment.utils.EnrichmentUtils;
 import org.apache.metron.common.utils.MessageUtils;
@@ -127,7 +127,7 @@ public class EnrichmentSplitterBolt extends SplitBolt<JSONObject> {
         if(sensorType != null) {
             SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sensorType);
             if (config != null) {
-                return config.getEnrichmentFieldMap();
+                return config.getEnrichment().getFieldMap();
             } else {
                 LOG.error("Unable to retrieve a sensor enrichment config of " + sensorType);
             }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
index e5b8ca6..b79d6c7 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
@@ -31,7 +31,7 @@ import org.apache.metron.common.Constants;
 import org.apache.metron.common.bolt.ConfiguredBolt;
 import org.apache.metron.common.configuration.Configurations;
 import org.apache.metron.enrichment.configuration.Enrichment;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
 import org.apache.metron.common.utils.ErrorUtils;
 import org.json.simple.JSONObject;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
index 68e56ed..1964961 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java
@@ -29,6 +29,8 @@ import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Sets;
 import org.apache.metron.common.bolt.ConfiguredBolt;
+import org.apache.metron.common.utils.ErrorUtils;
+import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.util.HashMap;
@@ -66,10 +68,12 @@ public abstract class JoinBolt<V> extends ConfiguredBolt {
   public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) {
     super.prepare(map, topologyContext, outputCollector);
     this.collector = outputCollector;
-    if (this.maxCacheSize == null)
+    if (this.maxCacheSize == null) {
       throw new IllegalStateException("maxCacheSize must be specified");
-    if (this.maxTimeRetain == null)
+    }
+    if (this.maxTimeRetain == null) {
       throw new IllegalStateException("maxTimeRetain must be specified");
+    }
     loader = new CacheLoader<String, Map<String, V>>() {
       public Map<String, V> load(String key) throws Exception {
         return new HashMap<>();
@@ -96,13 +100,18 @@ public abstract class JoinBolt<V> extends ConfiguredBolt {
       streamMessageMap.put(streamId, message);
       Set<String> streamIds = getStreamIds(message);
       Set<String> streamMessageKeys = streamMessageMap.keySet();
-      if (streamMessageKeys.size() == streamIds.size() && Sets.symmetricDifference
-              (streamMessageKeys, streamIds)
-              .isEmpty()) {
-        collector.emit("message", tuple, new Values(key, joinMessages
-                (streamMessageMap)));
-        collector.ack(tuple);
+      if ( streamMessageKeys.size() == streamIds.size()
+        && Sets.symmetricDifference(streamMessageKeys, streamIds)
+               .isEmpty()
+         ) {
+        collector.emit( "message"
+                      , tuple
+                      , new Values( key
+                                  , joinMessages(streamMessageMap)
+                                  )
+                      );
         cache.invalidate(key);
+        collector.ack(tuple);
       } else {
         cache.put(key, streamMessageMap);
         if(LOG.isDebugEnabled()) {
@@ -111,15 +120,19 @@ public abstract class JoinBolt<V> extends ConfiguredBolt {
                    );
         }
       }
-    } catch (ExecutionException e) {
+    } catch (Exception e) {
+      LOG.error("[Metron] Unable to join messages: " + message, e);
+      JSONObject error = ErrorUtils.generateErrorMessage("Joining problem: " + message, e);
+      collector.ack(tuple);
+      collector.emit("error", new Values(error));
       collector.reportError(e);
-      LOG.error(e.getMessage(), e);
     }
   }
 
   @Override
   public void declareOutputFields(OutputFieldsDeclarer declarer) {
     declarer.declareStream("message", new Fields("key", "message"));
+    declarer.declareStream("error", new Fields("message"));
   }
 
   public abstract void prepare(Map map, TopologyContext topologyContext);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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 a2b0e78..c08bd0d 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,7 +17,12 @@
  */
 package org.apache.metron.enrichment.bolt;
 
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import com.google.common.base.Joiner;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.threatintel.triage.ThreatTriageProcessor;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,7 +43,7 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
   public Map<String, List<String>> getFieldMap(String sourceType) {
     SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
     if(config != null) {
-      return config.getThreatIntelFieldMap();
+      return config.getThreatIntel().getFieldMap();
     }
     else {
       LOG.error("Unable to retrieve sensor config: " + sourceType);
@@ -49,12 +54,54 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
   @Override
   public JSONObject joinMessages(Map<String, JSONObject> streamMessageMap) {
     JSONObject ret = super.joinMessages(streamMessageMap);
-    for(Object key : ret.keySet()) {
-      if(key.toString().startsWith("threatintels") && !key.toString().endsWith(".ts")) {
-        ret.put("is_alert" , "true");
-        break;
+    boolean isAlert = ret.containsKey("is_alert");
+    if(!isAlert) {
+      for (Object key : ret.keySet()) {
+        if (key.toString().startsWith("threatintels") && !key.toString().endsWith(".ts")) {
+          isAlert = true;
+          break;
+        }
       }
     }
+    if(isAlert) {
+      ret.put("is_alert" , "true");
+      String sourceType = MessageUtils.getSensorType(ret);
+      SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sourceType);
+      ThreatTriageConfig triageConfig = null;
+      if(config != null) {
+        triageConfig = config.getThreatIntel().getTriageConfig();
+        if(LOG.isDebugEnabled()) {
+          LOG.debug(sourceType + ": Found sensor enrichment config.");
+        }
+      }
+      else {
+        LOG.debug(sourceType + ": Unable to find threat config.");
+      }
+      if(triageConfig != null) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug(sourceType + ": Found threat triage config: " + triageConfig);
+        }
+
+        if(LOG.isDebugEnabled() && (triageConfig.getRiskLevelRules() == null || triageConfig.getRiskLevelRules().isEmpty())) {
+          LOG.debug(sourceType + ": Empty rules!");
+        }
+
+        ThreatTriageProcessor threatTriageProcessor = new ThreatTriageProcessor(triageConfig);
+        Double triageLevel = threatTriageProcessor.apply(ret);
+        if(LOG.isDebugEnabled()) {
+          String rules = Joiner.on('\n').join(triageConfig.getRiskLevelRules().entrySet());
+          LOG.debug("Marked " + sourceType + " as triage level " + triageLevel + " with rules " + rules);
+        }
+        if(triageLevel != null && triageLevel > 0) {
+          ret.put("threat.triage.level", triageLevel);
+        }
+      }
+      else {
+        LOG.debug(sourceType + ": Unable to find threat triage config!");
+      }
+
+    }
+
     return ret;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
index 1429b2c..3cd1780 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelSplitterBolt.java
@@ -17,7 +17,7 @@
  */
 package org.apache.metron.enrichment.bolt;
 
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.utils.ThreatIntelUtils;
 
 import java.util.HashMap;
@@ -35,7 +35,7 @@ public class ThreatIntelSplitterBolt extends EnrichmentSplitterBolt {
     if (sensorType != null) {
       SensorEnrichmentConfig config = configurations.getSensorEnrichmentConfig(sensorType);
       if (config != null) {
-        return config.getThreatIntelFieldMap();
+        return config.getThreatIntel().getFieldMap();
       } else {
         LOG.error("Unable to retrieve sensor config: " + sensorType);
       }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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
new file mode 100644
index 0000000..bb4d387
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/threatintel/triage/ThreatTriageProcessor.java
@@ -0,0 +1,51 @@
+/**
+ * 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.threatintel.triage;
+
+import com.google.common.base.Function;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
+import org.apache.metron.common.query.MapVariableResolver;
+import org.apache.metron.common.query.PredicateProcessor;
+import org.apache.metron.common.query.VariableResolver;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class ThreatTriageProcessor implements Function<Map, Double> {
+  private ThreatTriageConfig config;
+  public ThreatTriageProcessor(ThreatTriageConfig config) {
+    this.config = config;
+  }
+
+  @Nullable
+  @Override
+  public Double apply(@Nullable Map input) {
+    List<Number> scores = new ArrayList<>();
+    PredicateProcessor predicateProcessor = new PredicateProcessor();
+    VariableResolver resolver = new MapVariableResolver(input);
+    for(Map.Entry<String, Number> kv : config.getRiskLevelRules().entrySet()) {
+      if(predicateProcessor.parse(kv.getKey(), resolver)) {
+        scores.add(kv.getValue());
+      }
+    }
+    return config.getAggregator().aggregate(scores, config.getAggregationConfig());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java
index 1c79f12..e2c6309 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java
@@ -19,7 +19,7 @@ package org.apache.metron.enrichment.adapters.simplehbase;
 
 
 import org.adrianwalker.multilinestring.Multiline;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.enrichment.converter.EnrichmentKey;
 import org.apache.metron.enrichment.converter.EnrichmentValue;
@@ -52,26 +52,28 @@ public class SimpleHBaseAdapterTest {
   }};
 
   /**
-   * {
-   * "10.0.2.3.orientation":"north"
-   * }
+    {
+    "10.0.2.3.orientation":"north"
+    }
    */
   @Multiline
   private String expectedMessageString;
 
   /**
-   * {
-   * "index": "bro",
-   * "batchSize": 5,
-   * "enrichmentFieldMap": {
-   * "geo": ["ip_dst_addr", "ip_src_addr"],
-   * "host": ["host"]
-   * },
-   * "fieldToEnrichmentTypeMap": {
-   * "ip_dst_addr" : [ "10.0.2.3" ],
-   * "ip_src_addr" : [ "10.3.30.120" ]
-   * }
-   * }
+    {
+      "index": "bro",
+      "batchSize": 5,
+      "enrichment": {
+        "fieldMap": {
+          "geo": ["ip_dst_addr", "ip_src_addr"],
+          "host": ["host"]
+        },
+      "fieldToTypeMap": {
+        "ip_dst_addr" : [ "10.0.2.3" ],
+        "ip_src_addr" : [ "10.3.30.120" ]
+        }
+      }
+   }
    */
   @Multiline
   private String sourceConfigStr;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java
index 62c8b43..2afeb5b 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java
@@ -20,7 +20,7 @@ package org.apache.metron.enrichment.adapters.threatintel;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.bolt.CacheKey;
 import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.enrichment.converter.EnrichmentKey;
@@ -62,29 +62,33 @@ public class ThreatIntelAdapterTest {
   private EnrichmentLookup lookup;
 
   /**
-   * {
-   * "10.0.2.3":"alert"
-   * }
+    {
+    "10.0.2.3":"alert"
+    }
    */
   @Multiline
   private String expectedMessageString;
 
   /**
-   * {
-   * "index": "bro",
-   * "batchSize": 5,
-   * "enrichmentFieldMap": {
-   * "geo": ["ip_dst_addr", "ip_src_addr"],
-   * "host": ["host"]
-   * },
-   * "threatIntelFieldMap": {
-   * "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
-   * },
-   * "fieldToThreatIntelTypeMap": {
-   * "ip_dst_addr" : [ "10.0.2.3" ],
-   * "ip_src_addr" : [ "malicious_ip" ]
-   * }
-   * }
+    {
+      "index": "bro",
+      "batchSize": 5,
+      "enrichment": {
+        "fieldMap": {
+          "geo": ["ip_dst_addr", "ip_src_addr"],
+          "host": ["host"]
+        }
+      },
+      "threatIntel" : {
+        "fieldMap": {
+          "hbaseThreatIntel": ["ip_dst_addr", "ip_src_addr"]
+        },
+        "fieldToTypeMap": {
+          "ip_dst_addr" : [ "10.0.2.3" ],
+          "ip_src_addr" : [ "malicious_ip" ]
+        }
+      }
+    }
    */
   @Multiline
   private static String sourceConfigStr;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
index d5a90fb..5a09f79 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java
@@ -22,9 +22,9 @@ import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.TestConstants;
 import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
 import org.apache.metron.enrichment.configuration.Enrichment;
-import org.apache.metron.common.configuration.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
-import org.apache.metron.common.cli.ConfigurationsUtils;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.hamcrest.Description;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/deed21e6/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 306c3e1..d3a5414 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
@@ -17,8 +17,12 @@
  */
 package org.apache.metron.enrichment.bolt;
 
+import com.fasterxml.jackson.databind.JsonMappingException;
 import junit.framework.Assert;
 import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
+import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.test.bolt.BaseEnrichmentBoltTest;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
@@ -78,12 +82,37 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
     alertMessage = (JSONObject) parser.parse(alertMessageString);
   }
 
-  @Test
-  public void test() throws IOException {
+  /**
+    {
+    "riskLevelRules" : {
+        "enrichedField1 == 'enrichedValue1'" : 10
+                          }
+   ,"aggregator" : "MAX"
+   }
+   */
+  @Multiline
+  private static String threatTriageConfigStr;
+
+  public void test(String threatTriageConfig, boolean badConfig) throws IOException {
     ThreatIntelJoinBolt threatIntelJoinBolt = new ThreatIntelJoinBolt("zookeeperUrl");
     threatIntelJoinBolt.setCuratorFramework(client);
     threatIntelJoinBolt.setTreeCache(cache);
-    threatIntelJoinBolt.getConfigurations().updateSensorEnrichmentConfig(sensorType, new FileInputStream(sampleSensorEnrichmentConfigPath));
+    SensorEnrichmentConfig enrichmentConfig = JSONUtils.INSTANCE.load(new FileInputStream(sampleSensorEnrichmentConfigPath), SensorEnrichmentConfig.class);
+    boolean withThreatTriage = threatTriageConfig != null;
+    if(withThreatTriage) {
+      try {
+        enrichmentConfig.getThreatIntel().setTriageConfig(JSONUtils.INSTANCE.load(threatTriageConfig, ThreatTriageConfig.class));
+        if(badConfig) {
+          Assert.fail(threatTriageConfig + "\nThis should not parse!");
+        }
+      }
+      catch(JsonMappingException pe) {
+        if(!badConfig) {
+          throw pe;
+        }
+      }
+    }
+    threatIntelJoinBolt.getConfigurations().updateSensorEnrichmentConfig(sensorType, enrichmentConfig);
     threatIntelJoinBolt.withMaxCacheSize(100);
     threatIntelJoinBolt.withMaxTimeRetain(10000);
     threatIntelJoinBolt.prepare(new HashMap<>(), topologyContext, outputCollector);
@@ -101,5 +130,35 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
     streamMessageMap.put("message", alertMessage);
     joinedMessage = threatIntelJoinBolt.joinMessages(streamMessageMap);
     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);
+    }
+    else {
+      Assert.assertFalse(joinedMessage.containsKey("threat.triage.level"));
+    }
+  }
+  /**
+    {
+    "riskLevelRules" : {
+        "enrichedField1 == 'enrichedValue1" : 10
+                          }
+   ,"aggregator" : "MAX"
+   }
+   */
+  @Multiline
+  private static String badRuleThreatTriageConfigStr;
+
+
+  @Test
+  public void testWithTriage() throws IOException {
+    test(threatTriageConfigStr, false);
+  }
+  @Test
+  public void testWithBadTriageRule() throws IOException {
+    test(badRuleThreatTriageConfigStr, true);
+  }
+  @Test
+  public void testWithoutTriage() throws IOException {
+    test(null, false);
   }
 }